You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by Ben-Zvi <gi...@git.apache.org> on 2017/05/02 04:24:03 UTC

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

GitHub user Ben-Zvi opened a pull request:

    https://github.com/apache/drill/pull/822

    DRILL-5457: Spill implementation for Hash Aggregate

    This pull-request is for the work on enabling memory spill for the Hash Aggregate Operator.
    
    To assist in reviewing this extensive code change, listed below are various topics/issues that describe the implementation decisions and give some code pointers. The reviewer can read these items and peek into their relevant code, or read all the items first (and comment on the design decisions as well).
    
    The last topic is not "least": It describes many issues and solutions related to the need to estimate the memory size of batches (and hash tables, etc.) This work took a significant amount of time, and will need some more to get better.
    
    (Most of the code changes are in HashAggTemplate.java, hence this file is not mentioned specifically below)
    
    ### Aggregation phase:
       The code was changed to pass the aggregation phase information (whether this is a Single phase, or 1st of two phase, or 2nd of two phase) from the planner to the HAG operator code.
    (See HashAggregate.java, AggPrelBase.java, HashAggPrel.java )
    
    ### Partitioning:
      The data (rows/groups) coming into the HAG is partitioned into (a power of 2) number of partitions, based on the N least significant bits of the hash value (computed out of the row's key columns).
      Each partition can be handled independently of the others. Ideally each partition should fit into the available memory. The number of partitions is initialized from the option "drill.exec.hashagg.num_partitions", and scaled down if the available memory seems too small (each partition needs to hold at least one batch in memory).
      The scaling down uses the formula:  AVAIL_MEMORY >  NUM_PARTITIONS * ( K * EST_BATCH_SIZE + 8M )
    (see delayedSetup() ) where K is the option drill.exec.hashagg.min_batches_per_partition -- see below).
      Computing the number of partitions is delayed till actuall data arrives on incoming (in order to get an accurate sizing on varchars). See delayedSetup(). There is also special code for cases data never arrives (empty batches) hence no partitions (see beginning of outputCurrentBatch(), cleanUp(), delayedSetup() ).
      Many of the code changes made in order to implement multi-partitions follow the original code, only changing scalar members (of HashAggTemplate) into arrays, like "htable" becomes "htables[]".
      Each partition has its own hash table. After each time it is spilled, its hash table is freed and reallocated.
    
    ### Hash Code:
      The hash code computation result was extracted from the HashTable (needed for the partitions), and added as a parameter to the put() method. Thus for each new incoming row, first the hash code is computed, and the low N bits are used to select the partition, then the hash code is right shifted by N, and the result is passed back to the put() method.
      After spilling, the hash codes are not kept. When reading the rows (groups) from the spill, the hash codes are computed again (and right shifted before use - once per each cycle of spilling - thus repartitioning).
    (See more about "spilling cycle" below).
    
    ### Hash Table put():
      The put() method for the hash table was rewriten and simplified. In addition to the hash-code parameter change, it was changed to return the PutStatus, with two new states: NEW_BATCH_ADDED notifies the caller that a new batch was created internally, hence a new batch (only needed for Hash Agg) is needed (prior code was getting this from comparing the returned index against the prior number of batches).
      A second new state is KEY_ADDED_LAST, which notifies that a batch was just filled, hence it is time for checking memory availability (because a new batch would be allocated soon).
      Similar rewriting was done for the hash table containsKey() method (and addBatchifNeeded() ).
    
    ### Memory pressure check:
      Logically the place to check for a memory pressure is when a new memory is needed (i.e., when a new group needs to be created.) However the code structure does not let this easily (e.g., a new batch is allocated inside the hash table object when a new group is detected, or the hash table structure is doubled in size),  thus instead the check is done AFTER a new group was added, in case this was the last group added to that batch  (see in checkGroupAndAggrValues() - checking for a new status KEY_ADDED_LAST  )
       This memory availability check checks if there is enough memory left between the allocated so far and the limit.
     Spill is initiated when:  MEMORY_USED + MAX_MEMORY_NEEDED > MEMORY_LIMIT   (see checkGroupAndAggrValues() )
     where the memory needed is:  (EST_BATCH_SIZE + 64K * (4+4)) * K * PLANNED_BATCHES + MAX_HASH_TABLES_RESIZE
    (See K above, under Partitioning, and the rest well below, under memory estimations).
    
    ### When can not spill:
      Single phase HAG can not spill. Also under memory duress 2nd phase may end up with only a single partition, which can not allow spilling (no progress is made). In these two cases, the memory check is skipped, and the operator functions like the old code -- if runs out of memory then it will OOM. A try-catch was added into the code to provide more detail on the OOM (see getOOMErrorMsg() ).
       Also in case of a single partition the allocator's memory limit is set to 10GB, to be compatible with the prior code.
       Another "can't spill" situation is when choosing a partition to spill, but no partition has more than 1 batch  (hence memory can not be gained, as after spilling 1 batch need to reinitialize that partition with a new batch). See chooseAPartitionToFlush(). In such a case the code "crosses its fingers" and continues without spilling.
    
    ### 1st phase - Early return:
      The 1st phase HAG does not spill to disk. When the 1st detects a memory pressure it picks the current partition (the one whose last batch just got full) and returns that partition downstream (just like regular return, only early). Afterwards that partition is (deallocated and) initialized. Note the boolean "earlyOutput" in the code which controls special processing in this case - when turned on the code switches to output (e.g., innerNext() in HashAggBatch.java), and turned off when done (see  outputCurrentBatch() ).
    
    ### Spilling: 
      Using the SpillSet (like the External Sort does) for the actual IO.  Each partition spills into a single
    file.  Changes to SpillSet: Generalize it for any kind of "buffered memory" operator (pass in the operator's type). Also small changes to the spill file name.
    
    ### 2nd phase - Flushing/spilling: 
      When a memory pressure is detected (while reading and processing the incoming), one of the
    partitions is selected ( see chooseAPartitionToFlush() ) and flushed ( spillAPartition() ), and then its memory is freed and that partition is re-initialized ( reinitPartition() ). The choice of a partition gives some small priority to the current partition (since its last batch is full, unlike the others), and priority by a factor of 4 to partitions that are already spilled (i.e., a spilled partition with 10 batches would be chosen vs a pristine/non-spilled with 39 batches.)
    
    Partition spilling (spillAPartition() ):  For each batch in the partition - Allocate an outgoing container, link the values and the keys into this container, and write it to the file/stream. 
    
    2nd phase - End of incoming: After the last batch was read from the incoming - the original code ( next() ) returned a status of NONE. The new code - after spilling can't return NONE, so instead returning a special status of RESTART (see outputCurrentBatch() ). This RESTART is captured by the caller of the next() ( innerNext() in HashAggBatch.java ) which continues to drive the aggregation (instead of returning).
    
    After the end of the incoming, all the (partially) spilled partitions finish spilling all their remaining in-memory batches to disk (see outputCurrentBatch() ). This is done to simplify the later processing of each spilled partition, as well as freeing memory which may be needed as partitions are processed. The spilled partitions are added into a list (spilledPartitionsList) to allow for later processing.
    
    ### 2nd phase reading of the spill: 
    Reading of each spilled partition/file is performed like reading the incoming. For this purpose, a new class was added: SpilledRecordbatch. The main method there is next() which reads a batch from the stream -- first time it uses the original readFromStream() method, which creates a new container; subsequent calls use the new readFromStreamWithContainer() method, which is similar - only reuses the prior container. (This was done because many places in the code have references into the container).
    
    ### Spilling cycles: 
    Reading a spilled partition "just like incoming" allows for that input to spill again (and again ...);
    this was termed SECONDARY spilling (and TERTIARY ...). As the spilled partitions are kept in a FIFO list, processing of SECONDARY partitions would start only after all the regular spilled ones, etc. Hence a member "cycleNum" was created, incremented every time that processing the spilled list advances to another "cycle" (see outputCurrentBatch() ).
      The "cycleNum" is used for the hash-code computation; the same hash-code is computed at every cycle, but the cycle tells how much to right-shift that code so that different bits would be used (for partitioning and hash-table bucket).
    
    ### Configuration options:
    - drill.exec.hashagg.num_partitions: Initial number of partitions in each HAG operator (the number may be down adjusted in case too little memory is available). Default value: 32 , allowed range 1 - 128 , where a value of 1 means "No spilling" (and thus setting 10GB limit).
    - drill.exec.hashagg.min_batches_per_partition: Range 2--5. Default 3. Used for internal initial estimate of number of partitions, and later when predicting memory needed (to avoid a spill).
      (A value of 2 may be better, but it evokes some bug which would be addressed separately).
    
    Also using options common to all the "buffered" operators (can be overriden, per operator):
    - drill.exe.spill.fs: File system for spilling into.
    - drill.exec.spill.directories: (Array of) directories to spill into.
    (To override, per-operator: for the (managed) External Sort: "drill.exec.sort.external.spill.fs" and
     "drill.exec.sort.external.spill.directories", and for the Hash Aggregate:
     "drill.exec.hashagg.spill.fs" and "drill.exec.hashagg.spill.directories")
    
    For testing:
    - drill.exec.hashagg.mem_limit: Limit the memory for each HAG operator (also sets this number in the allocator, hence this is a "hard limit").
    
    Also for testing (or for a customer workaround ??):
    - planner.force_2phase_aggr: Forces the aggregation to be two phase.
    
    ### Stats and metrics:
      The hash-table stats were modified to sum the stats across all the partitions' hash tables. (This only applies to the first spilling cycle; does not count for SECONDARY, TERTIARY spilling etc.).
      New metrics added:
     - "num partitions" (actual number; may have been scaled down due to memory pressure)
     - "spilled partitions" (number that has spilled)
     - "MB spilled" (in case of 1st phase - that's the total data returned early).
     All the above three refer to the end of input into the HAG (does not include handling of spills, secondary spills, etc.)
     - "cycle": 0 - no spill (or 1st phase), 1 - regular spill, 2 - Secondary, 3 - Tertiary ...)
     
    ### Memory Allocation Utilities:
       Extended for all "buffered operators", not only for Sort. (Hash Join will be added later as well, etc.)
    
    ###Changes to Unit tests:
    - New TestHashAggSpill : Runs two hash agg queries - One spills some of its partitions (1 out of 2), and the other test forces a smaller memory hence gets into a Secondary and Tertiary spills.
    - TestBugFixes.testDRILL4884: This test implicitly relied on rows returned in order (the old Hash agg, plus the Parquet file).
        With the new division into partitions, that order was broken. Fix: added an "order by".
    - TestTpchDistributedConcurrent.testConcurrentQueries: Needed longer timeout (probably spilled).
    
    ### MISC
    - After a spill, check again if enough memory was freed, else spill (another partition) again. (Not sure if needed.)
    - Suggestion not implemented: Scaling down the initial hash-table sizes by the number of partitions (e.g. when 4 partitions, each hash-table starts with 1/4 of the initial size). Reason for not changing: starting with a small size immediately causes doubling and another doubling etc. Better allocate a little more and save that work.
    - The RecordBatchSizer had a recent change to handle MAPs (recursively). Merged this change with the modified measureColumn() which returns an int (the est size).
    
    ### MEMORY SIZE ESTIMATIONS
      As described above, we need to get good estimate of the memory needs in order to decide initially on the number of partitions, and later to decide each time (a batch gets filled) wheter to spill or not.
     These estimates are complicated due to:
    (1) Possible changes in the incoming data batches (e.g., varchar(12) in the first batch becomes varchar(200) in the second incoming batch). This may invalidate prior estimates.
    (2) Arbitrary setting of length 50 for varchar type (when sizing the allocation of DrillBufs)
    (3) Allocation size aligned up to nearest power of 2 (DrillBufs for varchars)
    (4) When an internal batch gets filled, and estimation shows ample memory -- a second batch may get filled before the first one's partition allocated a new batch (hence may cause "double booking").
    (5) Inserting a single value may cause the "start indices" (the real actual "hash table") to double in size. This structure can get pretty large (few MB).
    (6) Does the size of the incoming batch being charged against the HAG's memory allocator limit ? (Not sure; usually not a problem as the prior batch is deallocated before the next one comes; unless the next one is "much bigger")
    (7) For varchars: The memory is allocated as a power of 2 (e.g. doubled via setSafe()). This can cause a big memory waste, like if the total memory needed for 64k varchars is ~5MB, then 8MB is allocated, wasting 3MB).
    (8) The varchar value vector uses an internal "offset vector" that allocates "size+1", hence for 64K it allocates 512kb, of which 256kb are wasted (see DRILL-5446).
    
    ### Solutions for the memory estimation issues:
    (1)+(6) above: Monitor the size of each incoming batch. Resize batch size estimate if the incoming batch is bigger (see doWork() )
    
    (5) When estimating memory needs, take into account hash table size doubling in all partitions (using the new hash table method extraMemoryNeededForResize() ).
    
    (4) Track "plannedBatches"; when "promising" few partitions a new batch each, take this into account when checking for available memory. (Though "more than 1" situation seems very rare).
    
    (2)+(3) Idealy tracking the size of EACH varchar column could work better, but not simple to implement. Instead -- just find the maximum size of any of the incoming columns (for simplicity - not only varchars), and use this value (capped at 50, min value of 8; rounded up to the next power of 2 if needed).  This addresses the common situation of multiple short varchar key columns but not the (very rare) situation of a huge varchar KEY column, plus few short ones.
    
    (7) Update RecordBatchSizer.java -- added a method  netRowWidthCap50()  which takes into account the rounding up (per each column in a row), plus nulls arrays as needed, for each row (will multiply that by 64K in updateEstMaxBatchSize() ).
    
    ==== END ====


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/Ben-Zvi/drill hashagg-spill

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/822.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #822
    
----
commit 1e436cef2dd8c4f519e584fbe18d233eab26468f
Author: Boaz Ben-Zvi <bo...@bbenzvi-e754-mbp13.local>
Date:   2017-05-02T02:59:49Z

    Spill implementation for Hash Aggregate

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117103383
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -86,6 +92,19 @@
     
       BooleanValidator EXTERNAL_SORT_DISABLE_MANAGED_OPTION = new BooleanValidator("exec.sort.disable_managed", false);
     
    +  // Hash Aggregate Options
    +
    +  String HASHAGG_NUM_PARTITIONS_KEY = "drill.exec.hashagg.num_partitions";
    +  LongValidator HASHAGG_NUM_PARTITIONS = new RangeLongValidator("exec.hashagg.num_partitions", 1, 128, 32); // 1 means - no spilling
    +  String HASHAGG_MAX_MEMORY_KEY = "drill.exec.hashagg.mem_limit";
    +  LongValidator HASHAGG_MAX_MEMORY = new RangeLongValidator("exec.hashagg.mem_limit", 0, Integer.MAX_VALUE, 0);
    +  // min batches is used for tuning (each partition needs so many batches when planning the number of partitions,
    +  // or reserve this number when calculating whether the remaining available memory is too small and requires a spill.)
    +  // Low value may OOM (e.g., when incoming rows become wider), higher values use fewer partitions but are safer
    +  String HASHAGG_MIN_BATCHES_PER_PARTITION_KEY = "drill.exec.hashagg.min_batches_per_partition";
    +  LongValidator HASHAGG_MIN_BATCHES_PER_PARTITION = new RangeLongValidator("exec.hashagg.min_batches_per_partition", 2, 5, 3);
    +  String HASHAGG_SPILL_DIRS = "drill.exec.hashagg.spill.directories";
    --- End diff --
    
    Maybe label which are config options and which session/system options.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119974724
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    --- End diff --
    
    Yes, the code for spilling (a multi batch) partition was started as a copy of the code that outputs a batch (to the downstream). However there are some different debugging and logging code added inside, which would complicate combining the two.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122310373
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -154,7 +152,7 @@
       private int cycleNum = 0; // primary, secondary, tertiary, etc.
       private int originalPartition = -1; // the partition a secondary reads from
     
    -  private class SpilledPartition { public int spilledBatches; public String spillFile /* Path filePath */; int cycleNum; int origPartn; int prevOrigPartn; }
    +  private class SpilledPartition { public int spilledBatches; public String spillFile; int cycleNum; int origPartn; int prevOrigPartn; }
    --- End diff --
    
    `private static class` since you don't have any methods and so have no use for the "inner this" pointer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117079841
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    --- End diff --
    
    Why ignore this exception ? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117105048
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java ---
    @@ -99,17 +100,31 @@ public void setCost(double cost) {
         this.cost = cost;
       }
     
    -  // Not available. Presumably because Drill does not currently use
    -  // this value, though it does appear in some test physical plans.
    -//  public void setMaxAllocation(long alloc) {
    -//    maxAllocation = alloc;
    -//  }
    -
       @Override
       public long getMaxAllocation() {
         return maxAllocation;
       }
     
    +  /**
    +   * Any operator that supports spilling should override this method
    +   * @param maxAllocation The max memory allocation to be set
    +   */
    +  @Override
    +  public void setMaxAllocation(long maxAllocation) {
    +    this.maxAllocation = maxAllocation;
    +    /*throw new DrillRuntimeException("Unsupported method: setMaxAllocation()");*/
    +  }
    +
    +  /**
    +   * Any operator that supports spilling should override this method (and return true)
    +   * @return false
    +   */
    +  @Override
    +  public boolean getBufferedOperator() { return false; }
    --- End diff --
    
    Actually, whether an operator is buffered is an attribute of the operator itself, not a property of an operator instance. So, shouldn't this method:
    
    * Return false in the base class.
    * Return true in operators that are buffered.
    * Marked as \@JsonIgnore so it is not serialized (see [this post|http://stackoverflow.com/questions/7421474/how-can-i-tell-jackson-to-ignore-a-property-for-which-i-dont-have-control-over])
    * Should have no matching set method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117804950
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java ---
    @@ -497,6 +510,21 @@ public int numResizing() {
         return numResizing;
       }
     
    +  /**
    +   *
    +   * @return Size of extra memory needed if the HT (i.e. startIndices) is doubled
    +   */
    +  @Override
    +  public int extraMemoryNeededForResize() {
    --- End diff --
    
    Good observation !! Fixed .....


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122858065
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java ---
    @@ -149,14 +149,24 @@ public IterOutcome innerNext() {
           if ( aggOut == HashAggregator.AggIterOutcome.AGG_OK ) { return IterOutcome.OK; }
           // if RESTART - continue below with doWork() - read some spilled partition, just like reading incoming
           incoming = aggregator.getNewIncoming(); // Restart - incoming was just changed
    -      if ( wasKilled ) { // if kill() was called before, then finish up
    -        aggregator.cleanup();
    -        incoming.kill(false);
    -        return IterOutcome.NONE;
    -      }
         }
     
    -    AggOutcome out = aggregator.doWork();
    +    if ( wasKilled ) { // if kill() was called before, then finish up
    +      aggregator.cleanup();
    +      incoming.kill(false);
    +      return IterOutcome.NONE;
    +    }
    +
    +    // Read and aggregate records
    +    // ( may need to run again if the spilled partition that was read
    +    //   generated new partitions that were all spilled )
    +    AggOutcome out = AggOutcome.CALL_WORK_AGAIN;
    +    while ( out == AggOutcome.CALL_WORK_AGAIN) {
    +      //
    +      //  Read incoming batches and process their records
    +      //
    +      out = aggregator.doWork();
    +    }
    --- End diff --
    
    ```
    while (aggregator.doWork() == AggOutcome.CALL_WORK_AGAIN) {
      // Nothing to do
    }
    ```
    ?
    
    In one of your reviews you said you didn't like empty loops, but sometimes they are handy...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117103575
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java ---
    @@ -127,6 +129,60 @@ public void readFromStream(InputStream input) throws IOException {
         va = container;
       }
     
    +  // Like above, only preserve the original container and list of value-vectors
    --- End diff --
    
    This class was refactored in the external sort unit test PR. Let's coordinate on merging the two sets of changes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812287
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    --- End diff --
    
    Maybe log the problem to help track down issues on a production system?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119257032
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    +    private VectorContainer container = null;
    --- End diff --
    
    No -- the next() method reassigns this container (when reading the next batch from the spill file).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119260746
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813958
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    --- End diff --
    
    Wrong assumption! Does the code set schema to null on receipt of a `NONE` status from upstream? Or, does this count on the upstream having a null schema?
    
    I wonder, would it make sense to introduce an explicit set of states to keep track of things? `START, HAS_SCHEMA, LOADING, SPILLING, RELOADING, EOF`? It is easier to reason about states (with a nice story about state transitions) than to keep track of all the indicator flags and variables that, in effect indicate states.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118814014
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
         }
     
    -    // get the number of records in the batch holder that are pending output
    -    int numPendingOutput = batchHolders.get(outBatchIndex).getNumPendingOutput();
    +    // Initialization (covers the case of early output)
    +    ArrayList<BatchHolder> currPartition = batchHolders[earlyPartition];
    +    int currOutBatchIndex = outBatchIndex[earlyPartition];
    +    int partitionToReturn = earlyPartition;
    +
    +    if ( ! earlyOutput ) {
    +      // Update the next partition to return (if needed)
    +      // skip fully returned (or spilled) partitions
    +      while (nextPartitionToReturn < numPartitions) {
    +        //
    +        // If this partition was spilled - spill the rest of it and skip it
    +        //
    +        if ( isSpilled(nextPartitionToReturn) ) {
    +          spillAPartition(nextPartitionToReturn); // spill the rest
    +          SpilledPartition sp = new SpilledPartition();
    --- End diff --
    
    Can `SpilledPartition` take on some of the behavior that is inline here? Can it, for example, take a partition state instance and do the spilling? Can it handle the reading later?
    
    Not yet entirely clear how this works, so take this suggestion with a grain of salt...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118814000
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
         }
     
    -    // get the number of records in the batch holder that are pending output
    -    int numPendingOutput = batchHolders.get(outBatchIndex).getNumPendingOutput();
    +    // Initialization (covers the case of early output)
    +    ArrayList<BatchHolder> currPartition = batchHolders[earlyPartition];
    +    int currOutBatchIndex = outBatchIndex[earlyPartition];
    +    int partitionToReturn = earlyPartition;
    +
    +    if ( ! earlyOutput ) {
    +      // Update the next partition to return (if needed)
    +      // skip fully returned (or spilled) partitions
    +      while (nextPartitionToReturn < numPartitions) {
    +        //
    +        // If this partition was spilled - spill the rest of it and skip it
    +        //
    +        if ( isSpilled(nextPartitionToReturn) ) {
    +          spillAPartition(nextPartitionToReturn); // spill the rest
    +          SpilledPartition sp = new SpilledPartition();
    +          sp.spillFile = spillFiles[nextPartitionToReturn];
    +          sp.spilledBatches = spilledBatchesCount[nextPartitionToReturn];
    +          sp.cycleNum = cycleNum; // remember the current cycle
    +          sp.origPartn = nextPartitionToReturn; // for debugging / filename
    +          sp.prevOrigPartn = originalPartition; // for debugging / filename
    +          spilledPartitionsList.add(sp);
    +          try {
    +            reinitPartition(nextPartitionToReturn); // free the memory
    +          } catch (Exception e) {throw new RuntimeException(e);}
    --- End diff --
    
    See note above about caching specific exceptions and mapping to the proper `UserException`. Else, it is really hard to know what went wrong by looking at logs. And, the user has no clue: just "something went wrong."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117106759
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/util/MemoryAllocationUtilities.java ---
    @@ -40,7 +39,7 @@
        * @param plan
        * @param queryContext
        */
    -  public static void setupSortMemoryAllocations(final PhysicalPlan plan, final QueryContext queryContext) {
    +  public static void setupBufferedOpsMemoryAllocations(final PhysicalPlan plan, final QueryContext queryContext) {
    --- End diff --
    
    This change looks good. Very clean.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119256961
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    +    private VectorContainer container = null;
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812433
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    --- End diff --
    
    again, move all the `foo[i]` setup to a method on the partition state class suggested above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117056991
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -546,44 +1254,204 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
          holder.value = vv0.getAccessor().get(incomingRowIdx) ;
          }
          */
    +    /*
    +    if ( handlingSpills && ( incomingRowIdx == 0 ) ) {
    --- End diff --
    
    You could enclose this using the EXTRA_DEBUG_1 flag that is used elsewhere. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117806316
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    --- End diff --
    
    There was a user intervention - the setting of num_partitions to 1 !! Though for clarity, the message was changed now to "Spilling was disabled due to configuration setting of num_partitions to 1"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118154367
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java ---
    @@ -188,6 +188,18 @@
         NOT_YET,
     
         /**
    +     * Restart reading incoming batches.
    +     * <p>
    +     *     Prior calls to {@link #next()} returned data after all incoming batches were processed.
    +     *     But with this status, need to restart reading incoming RecordBatches.
    +     * </p>
    +     * <p>
    +     *     Currently only used internally by the Hash Aggregate.
    +     * </p>
    +     */
    --- End diff --
    
    The whole iterator driven next() model is very limiting. This change was the simplest - re-invoking the next next() so that spilled partitions could be processed from then on. 
    Trying to switch without returning from the next() first seems too complex.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117105800
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPrelBase.java ---
    @@ -51,7 +51,7 @@
     
     public abstract class AggPrelBase extends DrillAggregateRelBase implements Prel {
     
    -  protected static enum OperatorPhase {PHASE_1of1, PHASE_1of2, PHASE_2of2};
    +  public static enum OperatorPhase {PHASE_1of1, PHASE_1of2, PHASE_2of2};
    --- End diff --
    
    Nit, but enum values are constants and so convention is all upper case: maybe: PHASE1OF2 or PHASE1_OF_2 or PHASE_1_OF_2...
    
    Not a big deal; the current form is a bit easier to read...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122324389
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -512,122 +509,122 @@ private void updateEstMaxBatchSize(RecordBatch incoming) {
         }
       }
     
    +  /**
    +   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
    +   *  Once complete, get the incoming NEXT batch and process it as well, etc.
    +   *  For 1st phase, may return when an early output needs to be performed.
    +   *
    +   * @return Agg outcome status
    +   */
       @Override
       public AggOutcome doWork() {
    -    try {
    -      // Note: Keeping the outer and inner try blocks here to maintain some similarity with
    -      // StreamingAggregate which does somethings conditionally in the outer try block.
    -      // 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) {
    +    while (true) {
     
    -        // This would be called only once - after actual data arrives on incoming
    -        if ( schema == null && incoming.getRecordCount() > 0 ) {
    -          this.schema = incoming.getSchema();
    -          // Calculate the number of partitions based on actual incoming data
    -          delayedSetup();
    -        }
    +      // This would be called only once - first time actual data arrives on incoming
    +      if ( schema == null && incoming.getRecordCount() > 0 ) {
    +        this.schema = incoming.getSchema();
    +        currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
    +        // Calculate the number of partitions based on actual incoming data
    +        delayedSetup();
    +      }
     
    -        // loop through existing records, aggregating the values as necessary.
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Starting outer loop of doWork()...");
    +      //
    +      //  loop through existing records in this batch, aggregating the values as necessary.
    +      //
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Starting outer loop of doWork()...");
    +      }
    +      for (; underlyingIndex < currentBatchRecordCount; incIndex()) {
    +        if (EXTRA_DEBUG_2) {
    +          logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
             }
    -        for (; underlyingIndex < incoming.getRecordCount(); incIndex()) {
    -          if (EXTRA_DEBUG_2) {
    -            logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
    -          }
    -          checkGroupAndAggrValues(currentIndex);
    -          // If adding a group discovered a memory pressure during 1st phase, then start
    -          // outputing some partition to free memory.
    -          if ( earlyOutput ) {
    -            outputCurrentBatch();
    -            incIndex(); // next time continue with the next incoming row
    -            return AggOutcome.RETURN_OUTCOME;
    -          }
    +        checkGroupAndAggrValues(currentIndex);
    +        // If adding a group discovered a memory pressure during 1st phase, then start
    +        // outputing some partition downstream in order to free memory.
    +        if ( earlyOutput ) {
    +          outputCurrentBatch();
    +          incIndex(); // next time continue with the next incoming row
    +          return AggOutcome.RETURN_OUTCOME;
             }
    +      }
    +
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Processed {} records", underlyingIndex);
    +      }
     
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Processed {} records", underlyingIndex);
    +      // Cleanup the previous batch since we are done processing it.
    +      for (VectorWrapper<?> v : incoming) {
    --- End diff --
    
    If this is original code, then clearly it does the right thing else we'd have seen problems by now. So, fine to leave as-is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117865368
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -546,44 +1254,204 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
          holder.value = vv0.getAccessor().get(incomingRowIdx) ;
          }
          */
    +    /*
    +    if ( handlingSpills && ( incomingRowIdx == 0 ) ) {
    +      // for debugging -- show the first row from a spilled batch
    +      Object tmp0 = (incoming).getValueAccessorById(NullableVarCharVector.class, 0).getValueVector();
    +      Object tmp1 = (incoming).getValueAccessorById(NullableVarCharVector.class, 1).getValueVector();
    +      Object tmp2 = (incoming).getValueAccessorById(NullableBigIntVector.class, 2).getValueVector();
    +
    +      if (tmp0 != null && tmp1 != null && tmp2 != null) {
    +        NullableVarCharVector vv0 = ((NullableVarCharVector) tmp0);
    +        NullableVarCharVector vv1 = ((NullableVarCharVector) tmp1);
    +        NullableBigIntVector  vv2 = ((NullableBigIntVector) tmp2);
    +        logger.debug("The first row = {} , {} , {}", vv0.getAccessor().get(incomingRowIdx), vv1.getAccessor().get(incomingRowIdx), vv2.getAccessor().get(incomingRowIdx));
    +      }
    +    }
    +    */
    +    // The hash code is computed once, then its lower bits are used to determine the
    +    // partition to use, and the higher bits determine the location in the hash table.
    +    int hashCode;
    +    try {
    +      htables[0].updateBatches();
    +      hashCode = htables[0].getHashCode(incomingRowIdx);
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
     
    -    htable.put(incomingRowIdx, htIdxHolder, 1 /* retry count */);
    +    // right shift hash code for secondary (or tertiary...) spilling
    +    for (int i = 0; i < cycleNum; i++) { hashCode >>>= bitsInMask; }
     
    +    int currentPartition = hashCode & partitionMask ;
    +    hashCode >>>= bitsInMask;
    +    HashTable.PutStatus putStatus = null;
    +    long allocatedBefore = allocator.getAllocatedMemory();
    +
    +    // Insert the key columns into the hash table
    +    try {
    +      putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode);
    +    } catch (OutOfMemoryException exc) {
    +      throw new OutOfMemoryException(getOOMErrorMsg(), exc); // may happen when can not spill
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
         int currentIdx = htIdxHolder.value;
     
    -    // get the batch index and index within the batch
    -    if (currentIdx >= batchHolders.size() * HashTable.BATCH_SIZE) {
    -      addBatchHolder();
    +    long addedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +    if ( addedMem > 0 ) {
    +      logger.trace("MEMORY CHECK HT: allocated {}  added {} partition {}",allocatedBefore,addedMem,currentPartition);
         }
    -    BatchHolder bh = batchHolders.get((currentIdx >>> 16) & HashTable.BATCH_MASK);
    +
    +    // Check if put() added a new batch (for the keys) inside the hash table, hence a matching batch
    +    // (for the aggregate columns) needs to be created
    +    if ( putStatus == HashTable.PutStatus.NEW_BATCH_ADDED ) {
    +      try {
    +        long allocatedBeforeAggCol = allocator.getAllocatedMemory();
    +
    +        addBatchHolder(currentPartition);
    +
    +        if ( plannedBatches > 0 ) { plannedBatches--; } // just allocated a planned batch
    +        long totalAddedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +        logger.trace("MEMORY CHECK AGG: added {}  total (with HT) added {}",allocator.getAllocatedMemory()-allocatedBeforeAggCol,totalAddedMem);
    +        // resize the batch estimate if needed (e.g., varchars may take more memory than estimated)
    +        if ( totalAddedMem > estMaxBatchSize ) {
    +          logger.trace("Adjusting Batch size estimate from {} to {}",estMaxBatchSize,totalAddedMem);
    +          estMaxBatchSize = totalAddedMem;
    +        }
    +      } catch (OutOfMemoryException exc) {
    --- End diff --
    
    addBatchHolder() calls newBatchHolder(), which should allocate the new batch using the allocator, hence (I think) may OOM if not enough memory left.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119256700
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    --- End diff --
    
    Moved out the inner class into an independent class. (The "static inner class" approach failed due to a possible bug with Drill bytecode fixup -- the first parameter in the constructor was eliminated in the generated code).
      Extending the record batch interface was intentional - this leads to a very neat handling of the batch in the Hash Agg -- no code change, just like handling the batches incoming from upstream.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122320539
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -130,6 +127,7 @@
       private int currentIndex = 0;
       private IterOutcome outcome;
       private int numGroupedRecords = 0;
    +  private int currentBatchRecordCount = 0; // Performance: Avoid repeated calls to getRecordCount()
    --- End diff --
    
    The getRecordCount() virtual method is called **per each record** ! And in some cases this method performs several checks. Unfortunately other inefficiencies indeed dwarf this savings. A local variable won't work, as execution may return and come back (e.g. spill) midway processing the incoming batch.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813975
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
         }
     
    -    // get the number of records in the batch holder that are pending output
    -    int numPendingOutput = batchHolders.get(outBatchIndex).getNumPendingOutput();
    +    // Initialization (covers the case of early output)
    +    ArrayList<BatchHolder> currPartition = batchHolders[earlyPartition];
    +    int currOutBatchIndex = outBatchIndex[earlyPartition];
    +    int partitionToReturn = earlyPartition;
    +
    +    if ( ! earlyOutput ) {
    --- End diff --
    
    This is another overly-large method that would greatly benefit from being divided up into conceptual chunks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812566
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    +    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
    +    logger.trace("Incoming sizer: {}",sizer);
    +    // An empty batch only has the schema, can not tell actual length of varchars
    +    // else use the actual varchars length, each capped at 50 (to match the space allocation)
    +    estRowWidth = sizer.rowCount() == 0 ? sizer.stdRowWidth() : sizer.netRowWidthCap50();
    +    estMaxBatchSize = estRowWidth * MAX_BATCH_SIZE;
    +
    +    // Get approx max (varchar) column width to get better memory allocation
    +    maxColumnWidth = Math.max(sizer.maxSize(), VARIABLE_MIN_WIDTH_VALUE_SIZE);
    +    maxColumnWidth = Math.min(maxColumnWidth, VARIABLE_MAX_WIDTH_VALUE_SIZE);
    --- End diff --
    
    Is this saying that we'll use actual data width (via an estimate), unless it is less than 8 or more than 50? So, if the input consists of rows of columns of width 250, we'll assume that they are actually 50? What will that do to the memory calcs?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by rchallapalli <gi...@git.apache.org>.
Github user rchallapalli commented on the issue:

    https://github.com/apache/drill/pull/822
  
    Based on the current design, if the code senses that there is not sufficient memory then it goes back to the old code. Now I have encountered a case where this happened and the old agg did not respect the memory constraints imposed by me. I gave 116MB memory and the old hash agg code consumed ~130MB and completed the query. This doesn't play well with the overall resource management plan


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119747147
  
    --- 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 ) {
    +          this.schema = incoming.getSchema();
    +          // Calculate the number of partitions based on actual incoming data
    +          delayedSetup();
    +        }
    +
    --- End diff --
    
    Annotate history points to @amansinha100 as the creator .... And the trace level logging is already used heavily for "normal monitoring" (e.g., number of partitions chosen, size estimates). Need some new levels below trace, which unfortunately cannot be created in LogBack ..... 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119954158
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    --- End diff --
    
    Concurrent open files: While spilling, there is one per each (non-pristine) spilling partition (yes, can be as high as 16, or even 32). Afterwards, they are all closed; then for reading, each one gets opened; and though we process one partition at a time, closing of all is postponed to the end, as the processing code is unaware that the "incoming" actually comes from a spill file. 
        About the limits: Seems that current defaults (e.g. 64K open files per process) can serve us well for the foreseeable future. Intel just announced the i9, where the top of the line CPU has 18 cores. Hence 1000s of concurrent active same-process threads are not feasible anytime soon (think about context switching). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118811987
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    --- End diff --
    
    This class extends the record batch interface. That interface is *VERY* confusing. It sounds like it is just a "bundle of vectors" that holds records. But, it is actually the definition of the Drill Volcano-like iterator protocol: it defines the methods needed to use your Spilled Record Batch class as an operator. Since this is not an operator, you don't need to extend that class.
    
    In fact, it is not clear you even need a superclass. To hold the vectors this class has a container member. This class does not need most of the vector-access methods as this is not an operator; any that are needed can be called on the container itself.
    
    Clearly a spilled batch need not follow the `next()` iterator protocol.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119947814
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    --- End diff --
    
    This is HashAgg closing time, so GC probably does it; anyway won't hurt -- added a close call. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119245339
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -166,20 +252,23 @@ public BatchHolder() {
         }
     
         private boolean updateAggrValues(int incomingRowIdx, int idxWithinBatch) {
    -      updateAggrValuesInternal(incomingRowIdx, idxWithinBatch);
    +      try { updateAggrValuesInternal(incomingRowIdx, idxWithinBatch); }
    +      catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc); }
           maxOccupiedIdx = Math.max(maxOccupiedIdx, idxWithinBatch);
           return true;
         }
     
         private void setup() {
    -      setupInterior(incoming, outgoing, aggrValuesContainer);
    +      try { setupInterior(incoming, outgoing, aggrValuesContainer); }
    +      catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
         }
     
         private void outputValues(IndexPointer outStartIdxHolder, IndexPointer outNumRecordsHolder) {
           outStartIdxHolder.value = batchOutputCount;
           outNumRecordsHolder.value = 0;
           for (int i = batchOutputCount; i <= maxOccupiedIdx; i++) {
    -        outputRecordValues(i, batchOutputCount);
    +        try { outputRecordValues(i, batchOutputCount); }
    +        catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
    --- End diff --
    
    Done !



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118616162
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -205,10 +225,10 @@ drill.exec: {
             // Deprecated for managed xsort; used only by legacy xsort
             threshold: 40000,
             // File system to use. Local file system by default.
    -        fs: "file:///"
    +        fs: ${drill.exec.spill.fs},
    --- End diff --
    
    Done. Added:
    
            // -- The two options below can be used to override the options common
            // -- for all spilling operators (see "spill" above).
            // -- This is done for backward compatibility; in the future they
            // -- would be deprecated (you should be using only the common ones)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812353
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    --- End diff --
    
    This is a bit old-school. Might as well use the power of OO to simplify the code. Rather than six arrays which must be kept in sync, perhaps one array that contains instances of a class, where the class members are the various per-partition state variables.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119258161
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    +    private VectorContainer container = null;
    +    private InputStream spillStream;
    +    private int spilledBatches;
    +    private FragmentContext context;
    +    private BatchSchema schema;
    +    private OperatorContext oContext;
    +    // Path spillStreamPath;
    +    private String spillFile;
    +    VectorAccessibleSerializable vas;
    +
    +    public SpilledRecordbatch(String spillFile,/* Path spillStreamPath,*/ int spilledBatches, FragmentContext context, BatchSchema schema, OperatorContext oContext) {
    +      this.context = context;
    +      this.schema = schema;
    +      this.spilledBatches = spilledBatches;
    +      this.oContext = oContext;
    +      //this.spillStreamPath = spillStreamPath;
    +      this.spillFile = spillFile;
    +      vas = new VectorAccessibleSerializable(allocator);
    +      container = vas.get();
    +
    +      try {
    +        this.spillStream = spillSet.openForInput(spillFile);
    +      } catch (IOException e) { throw new RuntimeException(e);}
    +
    +      next(); // initialize the container
    +    }
    +
    +    @Override
    +    public SelectionVector2 getSelectionVector2() {
    +      throw new UnsupportedOperationException();
    +    }
    +
    +    @Override
    +    public SelectionVector4 getSelectionVector4() {
    +      throw new UnsupportedOperationException();
    +    }
    +
    +    @Override
    +    public TypedFieldId getValueVectorId(SchemaPath path) {
    +      return container.getValueVectorId(path);
    +    }
    +
    +    @Override
    +    public VectorWrapper<?> getValueAccessorById(Class<?> clazz, int... ids) {
    +      return container.getValueAccessorById(clazz, ids);
    +    }
    +
    +    @Override
    +    public Iterator<VectorWrapper<?>> iterator() {
    +      return container.iterator();
    +    }
    +
    +    @Override
    +    public FragmentContext getContext() { return context; }
    +
    +    @Override
    +    public BatchSchema getSchema() { return schema; }
    +
    +    @Override
    +    public WritableBatch getWritableBatch() {
    +      return WritableBatch.get(this);
    +    }
    +
    +    @Override
    +    public VectorContainer getOutgoingContainer() { return container; }
    +
    +    @Override
    +    public int getRecordCount() { return container.getRecordCount(); }
    +
    +    @Override
    +    public void kill(boolean sendUpstream) {
    +      this.close(); // delete the current spill file
    +    }
    +
    +    /**
    +     * Read the next batch from the spill file
    +     *
    +     * @return IterOutcome
    +     */
    +    @Override
    +    public IterOutcome next() {
    --- End diff --
    
    HashAgg is unique in the way it reads (and processes) the spilled batches exactly like reading (and processing) the incoming batches.  Its actual code footprint is quite small (mostly in the next() method).
      SpilledRun extends BatchGroup, and seems to have more logic. The two seem to have too many differences to bother in combining them.... 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119745935
  
    --- 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 --
    
    There is no code for OK_FIRST_NON_EMPTY; and the local field "schema" is used here as a flag to note "setup not yet performed" ( not always matched with OK_NEW_SCHEMA; sometimes the second batch with an OK is the first non empty batch).
       And next() is a FINAL method (in AbstractRecordBatch), which in turn invokes other next() methods of other classes extending RecordBatch (like the new SpilledRecordBatch).  Should we put there the code to perform delayed setup for the HashAgg ?
      Even if the next() is modified to return a new flag like OK_FIRST_NON_EMPTY -- these flags are checked in the code below, starting from the second batch and on. Not sure where the code reading the first incoming batch is ....
      


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812385
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    --- End diff --
    
    Here, rather than doing a loop that keeps arrays in sync, if we go with the partition state class, we simply call, say, `setup()` for each partition. Maybe even just call the constructor and let it set up the per-partition state.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118811819
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -166,20 +252,23 @@ public BatchHolder() {
         }
     
         private boolean updateAggrValues(int incomingRowIdx, int idxWithinBatch) {
    -      updateAggrValuesInternal(incomingRowIdx, idxWithinBatch);
    +      try { updateAggrValuesInternal(incomingRowIdx, idxWithinBatch); }
    +      catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc); }
           maxOccupiedIdx = Math.max(maxOccupiedIdx, idxWithinBatch);
           return true;
         }
     
         private void setup() {
    -      setupInterior(incoming, outgoing, aggrValuesContainer);
    +      try { setupInterior(incoming, outgoing, aggrValuesContainer); }
    +      catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
         }
     
         private void outputValues(IndexPointer outStartIdxHolder, IndexPointer outNumRecordsHolder) {
           outStartIdxHolder.value = batchOutputCount;
           outNumRecordsHolder.value = 0;
           for (int i = batchOutputCount; i <= maxOccupiedIdx; i++) {
    -        outputRecordValues(i, batchOutputCount);
    +        try { outputRecordValues(i, batchOutputCount); }
    +        catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
    --- End diff --
    
    `IllegalStateException`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117860457
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    --- End diff --
    
    Mmm .. don't remember; I'll remove this catch (i.e. throw a DrillRuntimeException in case IllegalState shows.). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813787
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    --- End diff --
    
    Need to close the `outputStream[i]` before deleting the file?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812251
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    --- End diff --
    
    `BaseAllocator.nextPowerOfTwo()`. I've seen other implementations as well. Maybe pick one and put it in a utilities class somewhere so we don't have to reinvent it multiple times?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119973491
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    --- End diff --
    
    Changed to UserException.resourceError, as the open call failed (e.g., disk failure).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813733
  
    --- 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:
    --- End diff --
    
    This is what ALL operators must do if we add this new status code. They all must check for the code and throw an assertion of it is received.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117863850
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -546,44 +1254,204 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
          holder.value = vv0.getAccessor().get(incomingRowIdx) ;
          }
          */
    +    /*
    +    if ( handlingSpills && ( incomingRowIdx == 0 ) ) {
    +      // for debugging -- show the first row from a spilled batch
    +      Object tmp0 = (incoming).getValueAccessorById(NullableVarCharVector.class, 0).getValueVector();
    +      Object tmp1 = (incoming).getValueAccessorById(NullableVarCharVector.class, 1).getValueVector();
    +      Object tmp2 = (incoming).getValueAccessorById(NullableBigIntVector.class, 2).getValueVector();
    +
    +      if (tmp0 != null && tmp1 != null && tmp2 != null) {
    +        NullableVarCharVector vv0 = ((NullableVarCharVector) tmp0);
    +        NullableVarCharVector vv1 = ((NullableVarCharVector) tmp1);
    +        NullableBigIntVector  vv2 = ((NullableBigIntVector) tmp2);
    +        logger.debug("The first row = {} , {} , {}", vv0.getAccessor().get(incomingRowIdx), vv1.getAccessor().get(incomingRowIdx), vv2.getAccessor().get(incomingRowIdx));
    +      }
    +    }
    +    */
    +    // The hash code is computed once, then its lower bits are used to determine the
    +    // partition to use, and the higher bits determine the location in the hash table.
    +    int hashCode;
    +    try {
    +      htables[0].updateBatches();
    +      hashCode = htables[0].getHashCode(incomingRowIdx);
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
     
    -    htable.put(incomingRowIdx, htIdxHolder, 1 /* retry count */);
    +    // right shift hash code for secondary (or tertiary...) spilling
    +    for (int i = 0; i < cycleNum; i++) { hashCode >>>= bitsInMask; }
     
    +    int currentPartition = hashCode & partitionMask ;
    +    hashCode >>>= bitsInMask;
    +    HashTable.PutStatus putStatus = null;
    +    long allocatedBefore = allocator.getAllocatedMemory();
    +
    +    // Insert the key columns into the hash table
    +    try {
    +      putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode);
    +    } catch (OutOfMemoryException exc) {
    +      throw new OutOfMemoryException(getOOMErrorMsg(), exc); // may happen when can not spill
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
         int currentIdx = htIdxHolder.value;
     
    -    // get the batch index and index within the batch
    -    if (currentIdx >= batchHolders.size() * HashTable.BATCH_SIZE) {
    -      addBatchHolder();
    +    long addedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +    if ( addedMem > 0 ) {
    +      logger.trace("MEMORY CHECK HT: allocated {}  added {} partition {}",allocatedBefore,addedMem,currentPartition);
         }
    -    BatchHolder bh = batchHolders.get((currentIdx >>> 16) & HashTable.BATCH_MASK);
    +
    +    // Check if put() added a new batch (for the keys) inside the hash table, hence a matching batch
    +    // (for the aggregate columns) needs to be created
    +    if ( putStatus == HashTable.PutStatus.NEW_BATCH_ADDED ) {
    +      try {
    +        long allocatedBeforeAggCol = allocator.getAllocatedMemory();
    +
    +        addBatchHolder(currentPartition);
    +
    +        if ( plannedBatches > 0 ) { plannedBatches--; } // just allocated a planned batch
    +        long totalAddedMem = allocator.getAllocatedMemory() - allocatedBefore;
    --- End diff --
    
    totalAddedMem is for both the group-by keys and the aggr columns. 
    allocatedBefore is the initial size of the allocation, then the keys are added to the hash table, (then allocatedBeforeAggCol keeps the size), then a batch holder for the agg-columns is added, and the total is computed. 
    The total is important, like if the incoming batch becomes bigger, and we try to adjust the estimate for the batch size.
    The allocatedBeforeAggCol only gives some tracing refinement to tell which batch grew (the one in the hash table or the one for the agg columns).
     


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812448
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    --- End diff --
    
    The ClassTransformationException (CTE) occurs only during code generation. But, this is a generated class (that is, the byte code method of generating code merged this code with generated code). Is the generated class generating more generated classes?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813857
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    --- End diff --
    
    This code makes me wonder... How many output streams are open at any one time? Drill is a highly concurrent system: we could have 1000s of fragments. If each has, say, a hash agg with 16 partitions, do we run the risk of 16,000 open file handles? Or, is the file handle opened only when needed for reading or writing?
    
    In general, as we add more spilling, we may need a global file handle cache that controls the number of open files. The same issue arises in the sort merge phase: all spill files maintain an open file handle; we might exceed some limit.
    
    For this PR, consider how long the file handle is open. Perhaps we need to file a JIRA about managing the total number of open files.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118103777
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -86,6 +92,19 @@
     
       BooleanValidator EXTERNAL_SORT_DISABLE_MANAGED_OPTION = new BooleanValidator("exec.sort.disable_managed", false);
     
    +  // Hash Aggregate Options
    +
    +  String HASHAGG_NUM_PARTITIONS_KEY = "drill.exec.hashagg.num_partitions";
    +  LongValidator HASHAGG_NUM_PARTITIONS = new RangeLongValidator("exec.hashagg.num_partitions", 1, 128, 32); // 1 means - no spilling
    +  String HASHAGG_MAX_MEMORY_KEY = "drill.exec.hashagg.mem_limit";
    +  LongValidator HASHAGG_MAX_MEMORY = new RangeLongValidator("exec.hashagg.mem_limit", 0, Integer.MAX_VALUE, 0);
    +  // min batches is used for tuning (each partition needs so many batches when planning the number of partitions,
    +  // or reserve this number when calculating whether the remaining available memory is too small and requires a spill.)
    +  // Low value may OOM (e.g., when incoming rows become wider), higher values use fewer partitions but are safer
    +  String HASHAGG_MIN_BATCHES_PER_PARTITION_KEY = "drill.exec.hashagg.min_batches_per_partition";
    +  LongValidator HASHAGG_MIN_BATCHES_PER_PARTITION = new RangeLongValidator("exec.hashagg.min_batches_per_partition", 2, 5, 3);
    +  String HASHAGG_SPILL_DIRS = "drill.exec.hashagg.spill.directories";
    --- End diff --
    
    All three are both config and session options.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119260986
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    --- End diff --
    
    One reason for this is minimizing the code change -- the new code resembles the old code, only using arrays instead of scalars.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813901
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    --- End diff --
    
    `UserException.dataWriteError`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118592786
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java ---
    @@ -133,6 +133,9 @@
          the need to turn off join optimization may go away.
        */
       public static final BooleanValidator JOIN_OPTIMIZATION = new BooleanValidator("planner.enable_join_optimization", true);
    +  // for testing purpose
    --- End diff --
    
    @VisibleForTesting annotates methods; but this is a session option. 
    Also (hidden) is the possibility that this option may be used in production in case some query yields a single phase hashagg but still has too much data to handle.    


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119481421
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    --- End diff --
    
    Done: 
    
    `      } catch (ClassTransformationException e) {
            throw UserException.unsupportedError(e)
                .message("Code generation error - likely an error in the code.")
                .build(logger);
          } catch (IOException e) {
            throw UserException.resourceError(e)
                .message("IO Error while creating a hash table.")
                .build(logger);
          } catch (SchemaChangeException sce) {
            throw new IllegalStateException("Unexpected Schema Change while creating a hash table",sce);
          }`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122321138
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -154,7 +152,7 @@
       private int cycleNum = 0; // primary, secondary, tertiary, etc.
       private int originalPartition = -1; // the partition a secondary reads from
     
    -  private class SpilledPartition { public int spilledBatches; public String spillFile /* Path filePath */; int cycleNum; int origPartn; int prevOrigPartn; }
    +  private class SpilledPartition { public int spilledBatches; public String spillFile; int cycleNum; int origPartn; int prevOrigPartn; }
    --- End diff --
    
    Done, thanks.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/822


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813964
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
    --- End diff --
    
    Is the outcome used as a state? Otherwise, why not just
    ```
    return IterOutcome.NONE;
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813868
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    --- End diff --
    
    Method on the partition state class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812316
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    --- End diff --
    
    `AbstractBase.MAX_ALLOCATION` to avoid having to keep this magic number in sync with that constant.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122858048
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java ---
    @@ -149,14 +149,24 @@ public IterOutcome innerNext() {
           if ( aggOut == HashAggregator.AggIterOutcome.AGG_OK ) { return IterOutcome.OK; }
           // if RESTART - continue below with doWork() - read some spilled partition, just like reading incoming
           incoming = aggregator.getNewIncoming(); // Restart - incoming was just changed
    -      if ( wasKilled ) { // if kill() was called before, then finish up
    -        aggregator.cleanup();
    -        incoming.kill(false);
    -        return IterOutcome.NONE;
    -      }
         }
     
    -    AggOutcome out = aggregator.doWork();
    +    if ( wasKilled ) { // if kill() was called before, then finish up
    --- End diff --
    
    Spaces, here and below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118811771
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -18,82 +18,161 @@
     package org.apache.drill.exec.physical.impl.aggregate;
     
     import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
     import java.util.ArrayList;
     import java.util.Iterator;
     import java.util.List;
    +import java.util.concurrent.TimeUnit;
     
     import javax.inject.Named;
     
    +import com.google.common.base.Stopwatch;
    +
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
     import org.apache.drill.common.expression.ErrorCollector;
     import org.apache.drill.common.expression.ErrorCollectorImpl;
     import org.apache.drill.common.expression.ExpressionPosition;
     import org.apache.drill.common.expression.FieldReference;
     import org.apache.drill.common.expression.LogicalExpression;
    +
    +import org.apache.drill.common.expression.SchemaPath;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.cache.VectorAccessibleSerializable;
     import org.apache.drill.exec.compile.sig.RuntimeOverridden;
     import org.apache.drill.exec.exception.ClassTransformationException;
    +import org.apache.drill.exec.exception.OutOfMemoryException;
     import org.apache.drill.exec.exception.SchemaChangeException;
     import org.apache.drill.exec.expr.TypeHelper;
    +
     import org.apache.drill.exec.memory.BufferAllocator;
     import org.apache.drill.exec.ops.FragmentContext;
     import org.apache.drill.exec.ops.MetricDef;
    +import org.apache.drill.exec.ops.OperatorContext;
     import org.apache.drill.exec.ops.OperatorStats;
     import org.apache.drill.exec.physical.config.HashAggregate;
     import org.apache.drill.exec.physical.impl.common.ChainedHashTable;
     import org.apache.drill.exec.physical.impl.common.HashTable;
     import org.apache.drill.exec.physical.impl.common.HashTableConfig;
     import org.apache.drill.exec.physical.impl.common.HashTableStats;
     import org.apache.drill.exec.physical.impl.common.IndexPointer;
    +
    +import org.apache.drill.exec.physical.impl.spill.RecordBatchSizer;
    +
    +import org.apache.drill.exec.physical.impl.spill.SpillSet;
    +import org.apache.drill.exec.planner.physical.AggPrelBase;
    +
    +import org.apache.drill.exec.proto.UserBitShared;
    +import org.apache.drill.exec.record.CloseableRecordBatch;
    +
     import org.apache.drill.exec.record.MaterializedField;
    +
     import org.apache.drill.exec.record.RecordBatch;
    -import org.apache.drill.exec.record.RecordBatch.IterOutcome;
    -import org.apache.drill.exec.record.TypedFieldId;
    +import org.apache.drill.exec.record.BatchSchema;
    +
     import org.apache.drill.exec.record.VectorContainer;
    +
    +import org.apache.drill.exec.record.TypedFieldId;
    +
    +import org.apache.drill.exec.record.RecordBatch.IterOutcome;
     import org.apache.drill.exec.record.VectorWrapper;
    +import org.apache.drill.exec.record.WritableBatch;
    +import org.apache.drill.exec.record.selection.SelectionVector2;
    +import org.apache.drill.exec.record.selection.SelectionVector4;
    +
     import org.apache.drill.exec.vector.AllocationHelper;
    +
     import org.apache.drill.exec.vector.FixedWidthVector;
     import org.apache.drill.exec.vector.ObjectVector;
     import org.apache.drill.exec.vector.ValueVector;
    +
     import org.apache.drill.exec.vector.VariableWidthVector;
     
    +import org.apache.hadoop.fs.Path;
    +
    +import static org.apache.drill.exec.record.RecordBatch.MAX_BATCH_SIZE;
    +
    --- End diff --
    
    It is impressive how you were able to slide spilling into the existing code structure. Performance and modularity are never required, of course, but it may be worth at least considering them.
    
    Putting so much code in a template has a large drawback. Our current byte-code based code generation performs at its worst when templates are large. This template is the base for the generated code. In traditional Java, the size of a subclass is independent of the size of the superclass. So, if we used "plain old" Java, the size of this template would have no performance impact.
    
    But, with byte-code manipulation, each generated class contains a complete copy of the byte code for the template class. With a huge template, we make a huge copy every time. We pay a cost in terms of the time it takes to make the copy, then analyze the resulting byte codes. Also, we fill up the code cache with many copies of the same code.
    
    Three solutions.
    
    1. Ignore the problem. (Which is probably the right choice until performance becomes a concern.)
    2. Refactor the code so that the bulk of logic is in a non-template class, with only a thin layer of code in the template.
    3. Use "plain old" Java compilation for this class to avoid the many large copies described above.
    
    The other problem is that this class has so many state variables that full testing and understanding will be hard. There is value in smaller chunks to reduce the cost of testing and maintenance.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117862000
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -546,44 +1254,204 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
          holder.value = vv0.getAccessor().get(incomingRowIdx) ;
          }
          */
    +    /*
    +    if ( handlingSpills && ( incomingRowIdx == 0 ) ) {
    --- End diff --
    
    This debugging code is written for a specific schema (VarChar,VarChar,Bigint); placing it inside EXTRA_DEBUG_1 may create the impression that it is a generic debug code. Leaving it as a comment/sample allows for easy rewriting to print out data (of any other schema).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117801284
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java ---
    @@ -544,86 +572,69 @@ private static int roundUpToPowerOf2(int number) {
         return rounded;
       }
     
    -  @Override
    -  public void put(int incomingRowIdx, IndexPointer htIdxHolder, int retryCount) {
    -    put(incomingRowIdx, htIdxHolder);
    +  public int getHashCode(int incomingRowIdx) throws SchemaChangeException {
    +    return getHashBuild(incomingRowIdx);
       }
     
    -  private PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder) {
    +  /** put() uses the hash code (from gethashCode() above) to insert the key(s) from the incoming
    +   * row into the hash table. The code selects the bucket in the startIndices, then the keys are
    +   * placed into the chained list - by storing the key values into a batch, and updating its
    +   * "links" member. Last it modifies the index holder to the batch offset so that the caller
    +   * can store the remaining parts of the row into a matching batch (outside the hash table).
    +   * Returning
    +   *
    +   * @param incomingRowIdx - position of the incoming row
    +   * @param htIdxHolder - to return batch + batch-offset (for caller to manage a matching batch)
    +   * @param hashCode - computed over the key(s) by calling getHashCode()
    +   * @return Status - the key(s) was ADDED or was already PRESENT
    +   */
    +  @Override
    +  public PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode) throws SchemaChangeException {
     
    -    int hash = getHashBuild(incomingRowIdx);
    -    int i = getBucketIndex(hash, numBuckets());
    -    int startIdx = startIndices.getAccessor().get(i);
    +    int bucketIndex = getBucketIndex(hashCode, numBuckets());
    +    int startIdx = startIndices.getAccessor().get(bucketIndex);
         int currentIdx;
    -    int currentIdxWithinBatch;
    -    BatchHolder bh;
         BatchHolder lastEntryBatch = null;
         int lastEntryIdxWithinBatch = EMPTY_SLOT;
     
    +    // if startIdx is non-empty, follow the hash chain links until we find a matching
    +    // key or reach the end of the chain (and remember the last link there)
    +    for ( currentIdxHolder.value = startIdx;
    +          currentIdxHolder.value != EMPTY_SLOT;
    +          /* isKeyMatch() below also advances the currentIdxHolder to the next link */) {
     
    -    if (startIdx == EMPTY_SLOT) {
    -      // this is the first entry in this bucket; find the first available slot in the
    -      // container of keys and values
    -      currentIdx = freeIndex++;
    -      addBatchIfNeeded(currentIdx);
    +      // remember the current link, which would be the last when the next link is empty
    +      lastEntryBatch = batchHolders.get((currentIdxHolder.value >>> 16) & HashTable.BATCH_MASK);
    +      lastEntryIdxWithinBatch = currentIdxHolder.value & BATCH_MASK;
     
    -      if (EXTRA_DEBUG) {
    -        logger.debug("Empty bucket index = {}. incomingRowIdx = {}; inserting new entry at currentIdx = {}.", i,
    -            incomingRowIdx, currentIdx);
    +      if (lastEntryBatch.isKeyMatch(incomingRowIdx, currentIdxHolder, false)) {
    +        htIdxHolder.value = currentIdxHolder.value;
    +        return PutStatus.KEY_PRESENT;
           }
    -
    -      insertEntry(incomingRowIdx, currentIdx, hash, lastEntryBatch, lastEntryIdxWithinBatch);
    -      // update the start index array
    -      startIndices.getMutator().setSafe(getBucketIndex(hash, numBuckets()), currentIdx);
    -      htIdxHolder.value = currentIdx;
    -      return PutStatus.KEY_ADDED;
         }
     
    -    currentIdx = startIdx;
    -    boolean found = false;
    -
    -    bh = batchHolders.get((currentIdx >>> 16) & BATCH_MASK);
    -    currentIdxHolder.value = currentIdx;
    -
    -    // if startIdx is non-empty, follow the hash chain links until we find a matching
    -    // key or reach the end of the chain
    -    while (true) {
    -      currentIdxWithinBatch = currentIdxHolder.value & BATCH_MASK;
    +    // no match was found, so insert a new entry
    +    currentIdx = freeIndex++;
    +    boolean addedBatch = addBatchIfNeeded(currentIdx);
     
    -      if (bh.isKeyMatch(incomingRowIdx, currentIdxHolder, false)) {
    -        htIdxHolder.value = currentIdxHolder.value;
    -        found = true;
    -        break;
    -      } else if (currentIdxHolder.value == EMPTY_SLOT) {
    -        lastEntryBatch = bh;
    -        lastEntryIdxWithinBatch = currentIdxWithinBatch;
    -        break;
    -      } else {
    -        bh = batchHolders.get((currentIdxHolder.value >>> 16) & HashTable.BATCH_MASK);
    -        lastEntryBatch = bh;
    -      }
    +    if (EXTRA_DEBUG) {
    +      logger.debug("No match was found for incomingRowIdx = {}; inserting new entry at currentIdx = {}.", incomingRowIdx, currentIdx);
         }
     
    -    if (!found) {
    -      // no match was found, so insert a new entry
    -      currentIdx = freeIndex++;
    -      addBatchIfNeeded(currentIdx);
    +    insertEntry(incomingRowIdx, currentIdx, hashCode, lastEntryBatch, lastEntryIdxWithinBatch);
     
    -      if (EXTRA_DEBUG) {
    -        logger.debug("No match was found for incomingRowIdx = {}; inserting new entry at currentIdx = {}.", incomingRowIdx, currentIdx);
    -      }
    -
    -      insertEntry(incomingRowIdx, currentIdx, hash, lastEntryBatch, lastEntryIdxWithinBatch);
    -      htIdxHolder.value = currentIdx;
    -      return PutStatus.KEY_ADDED;
    +    // if there was no hash chain at this bucket, need to update the start index array
    +    if (startIdx == EMPTY_SLOT) {
    +      startIndices.getMutator().setSafe(getBucketIndex(hashCode, numBuckets()), currentIdx);
         }
    -
    -    return found ? PutStatus.KEY_PRESENT : PutStatus.KEY_ADDED;
    +    htIdxHolder.value = currentIdx;
    +    return  addedBatch ? PutStatus.NEW_BATCH_ADDED :
    +        ( freeIndex + 1 > batchHolders.size() * BATCH_SIZE ) ?
    --- End diff --
    
    The former: Last key in a batch. This information is needed at the Hash Aggr to initiate a check for memory pressure (could calculate this information, but looks cleaner/simpler to get that as a special status from the hash table). Hopefully we'd never have to deal with batches of size 1 (row), as it would conflict - both "last row" and "new batch" at the same insertion. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812194
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -230,15 +452,35 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           throw new IllegalArgumentException("Wrong number of workspace variables.");
         }
     
    -//    this.context = context;
    +    this.context = context;
         this.stats = stats;
    -    this.allocator = allocator;
    +    this.allocator = oContext.getAllocator();
    +    this.oContext = oContext;
         this.incoming = incoming;
    -//    this.schema = incoming.getSchema();
         this.outgoing = outgoing;
         this.outContainer = outContainer;
    +    this.operatorId = hashAggrConfig.getOperatorId();
    +
    +    is2ndPhase = hashAggrConfig.getAggPhase() == AggPrelBase.OperatorPhase.PHASE_2of2;
    +    isTwoPhase = hashAggrConfig.getAggPhase() != AggPrelBase.OperatorPhase.PHASE_1of1;
    +    canSpill = isTwoPhase; // single phase can not spill
    --- End diff --
    
    Here we have three related booleans, or 2^8 cases. Consider using an enum to identify the (likely much smaller) number of actual cases. Maybe `ONE_PASS, FIRST_PHASE, SECOND_PHASE`?
    
    Then if the code does lots of "if this phase do that" kind of logic, it may be handy to have a single base class with common logic, then three (or whatever) base classes that define the phase-specific logic. Much easier to test and understand.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122310219
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -130,6 +127,7 @@
       private int currentIndex = 0;
       private IterOutcome outcome;
       private int numGroupedRecords = 0;
    +  private int currentBatchRecordCount = 0; // Performance: Avoid repeated calls to getRecordCount()
    --- End diff --
    
    Not sure that the very small savings in time is worth the complexity of keeping a cached copy in sync. If needed for an inner loop, can it be a local variable instead?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118814041
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
         }
     
    -    // get the number of records in the batch holder that are pending output
    -    int numPendingOutput = batchHolders.get(outBatchIndex).getNumPendingOutput();
    +    // Initialization (covers the case of early output)
    +    ArrayList<BatchHolder> currPartition = batchHolders[earlyPartition];
    +    int currOutBatchIndex = outBatchIndex[earlyPartition];
    +    int partitionToReturn = earlyPartition;
    +
    +    if ( ! earlyOutput ) {
    +      // Update the next partition to return (if needed)
    +      // skip fully returned (or spilled) partitions
    +      while (nextPartitionToReturn < numPartitions) {
    +        //
    +        // If this partition was spilled - spill the rest of it and skip it
    +        //
    +        if ( isSpilled(nextPartitionToReturn) ) {
    +          spillAPartition(nextPartitionToReturn); // spill the rest
    +          SpilledPartition sp = new SpilledPartition();
    +          sp.spillFile = spillFiles[nextPartitionToReturn];
    +          sp.spilledBatches = spilledBatchesCount[nextPartitionToReturn];
    +          sp.cycleNum = cycleNum; // remember the current cycle
    +          sp.origPartn = nextPartitionToReturn; // for debugging / filename
    +          sp.prevOrigPartn = originalPartition; // for debugging / filename
    +          spilledPartitionsList.add(sp);
    +          try {
    +            reinitPartition(nextPartitionToReturn); // free the memory
    +          } catch (Exception e) {throw new RuntimeException(e);}
    +          try {
    +            long posn = spillSet.getPosition(outputStream[nextPartitionToReturn]);
    +            spillSet.tallyWriteBytes(posn); // for the IO stats
    +            outputStream[nextPartitionToReturn].close();
    +          } catch (IOException e) { throw new RuntimeException(e); }
    +          outputStream[nextPartitionToReturn] = null;
    +        }
    +        else {
    +          currPartition = batchHolders[nextPartitionToReturn];
    +          currOutBatchIndex = outBatchIndex[nextPartitionToReturn];
    +          // If curr batch (partition X index) is not empty - proceed to return it
    +          if (currOutBatchIndex < currPartition.size() && 0 != currPartition.get(currOutBatchIndex).getNumPendingOutput()) {
    +            break;
    +          }
    +        }
    +        nextPartitionToReturn++; // else check next partition
    +      }
    +
    +      // if passed the last partition
    +      if (nextPartitionToReturn >= numPartitions) {
    +        // The following "if" is probably never used; due to a similar check at the end of this method
    +        if ( spilledPartitionsList.isEmpty() ) { // and no spilled partitions
    +          allFlushed = true;
    +          this.outcome = IterOutcome.NONE;
    +          if ( is2ndPhase ) {
    +            stats.setLongStat(Metric.SPILL_MB, // update stats - total MB spilled
    +                (int) Math.round(spillSet.getWriteBytes() / 1024.0D / 1024.0));
    +          }
    +          return outcome;  // then return NONE
    +        }
    +        // Else - there are still spilled partitions to process - pick one and handle just like a new incoming
    +        buildComplete = false; // go back and call doWork() again
    +        handlingSpills = true; // beginning to work on the spill files
    +        // pick a spilled partition; set a new incoming ...
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        SpilledRecordbatch newIncoming = newSpilledRecordBatch(sp.spillFile, sp.spilledBatches, context, schema, oContext);
    +        originalPartition = sp.origPartn; // used for the filename
    +        logger.trace("Reading back spilled original partition {} as an incoming",originalPartition);
    +        // prevOriginalPartition = sp.prevOrigPartn;
    +        // Initialize .... new incoming, new set of partitions
    +        try { initializeSetup(newIncoming); } catch (Exception e) { throw new RuntimeException(e); }
    +        // update the cycle num if needed
    +        // The current cycle num should always be one larger than in the spilled partition
    +        if ( cycleNum == sp.cycleNum ) {
    +          cycleNum = 1 + sp.cycleNum;
    +          stats.setLongStat(Metric.SPILL_CYCLE, cycleNum); // update stats
    +          // report memory stressful situations
    +          if ( cycleNum == 2 ) { logger.info("SECONDARY SPILLING "); }
    +          if ( cycleNum == 3 ) { logger.info("TERTIARY SPILLING "); }
    +        }
    +        if ( EXTRA_DEBUG_SPILL ) {
    +          logger.debug("Start reading spilled partition {} (prev {}) from cycle {} (with {} batches). More {} spilled partitions left.",
    +              sp.origPartn, sp.prevOrigPartn, sp.cycleNum, sp.spilledBatches, spilledPartitionsList.size());
    +        }
    +        return IterOutcome.RESTART;
    +      }
    +
    +      partitionToReturn = nextPartitionToReturn ;
     
    -    if (numPendingOutput == 0) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
         }
     
    +    // get the number of records in the batch holder that are pending output
    +    int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +    // The following accounting is for logging, metrics, etc.
    +    rowsInPartition += numPendingOutput ;
    +    if ( ! handlingSpills ) { rowsNotSpilled += numPendingOutput; }
    +    else { rowsSpilledReturned += numPendingOutput; }
    +    if ( earlyOutput ) { rowsReturnedEarly += numPendingOutput; }
    +
         allocateOutgoing(numPendingOutput);
     
    -    batchHolders.get(outBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +    currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
         int numOutputRecords = outNumRecordsHolder.value;
     
         if (EXTRA_DEBUG_1) {
           logger.debug("After output values: outStartIdx = {}, outNumRecords = {}", outStartIdxHolder.value, outNumRecordsHolder.value);
         }
    -    this.htable.outputKeys(outBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +    this.htables[partitionToReturn].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
     
         // set the value count for outgoing batch value vectors
         for (VectorWrapper<?> v : outgoing) {
           v.getValueVector().getMutator().setValueCount(numOutputRecords);
         }
     
    -//    outputCount += numOutputRecords;
    -
         this.outcome = IterOutcome.OK;
     
    -    logger.debug("HashAggregate: Output current batch index {} with {} records.", outBatchIndex, numOutputRecords);
    +    // logger.debug("HashAggregate: Output {} current batch index {} with {} records for partition {}.", earlyOutput ? "(early)" : "",
    +    //    outBatchIndex, numOutputRecords, partitionToReturn);
    +    if ( EXTRA_DEBUG_SPILL && is2ndPhase ) {
    +      logger.debug("So far returned {} + SpilledReturned {}  total {} (spilled {})",rowsNotSpilled,rowsSpilledReturned,
    +        rowsNotSpilled+rowsSpilledReturned,
    +        rowsSpilled);
    +    }
     
         lastBatchOutputCount = numOutputRecords;
    -    outBatchIndex++;
    -    if (outBatchIndex == batchHolders.size()) {
    -      allFlushed = true;
    +    outBatchIndex[partitionToReturn]++;
    +    // if just flushed the last batch in the partition
    +    if (outBatchIndex[partitionToReturn] == currPartition.size()) {
    +
    +      if ( EXTRA_DEBUG_SPILL ) {
    +        logger.debug("HashAggregate: {} Flushed partition {} with {} batches total {} rows",
    +            earlyOutput ? "(Early)" : "",
    +            partitionToReturn, outBatchIndex[partitionToReturn], rowsInPartition);
    +      }
    +      rowsInPartition = 0; // reset to count for the next partition
    +
    +      try {
    +        // deallocate memory used by this partition, and re-initialize
    +        reinitPartition(partitionToReturn);
    +      } catch (SchemaChangeException sce) {
    +        throw new DrillRuntimeException("Hash Aggregation can not handle schema changes.");
    +      } catch (Exception e) { /* just ignore */ }
    --- End diff --
    
    Even, say, an OOM?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117105244
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java ---
    @@ -49,12 +49,19 @@ public int getOperatorType() {
         return CoreOperatorType.EXTERNAL_SORT_VALUE;
       }
     
    -  // Set here, rather than the base class, because this is the only
    -  // operator, at present, that makes use of the maximum allocation.
    -  // Remove this, in favor of the base class version, when Drill
    -  // sets the memory allocation for all operators.
    -
    +  /**
    --- End diff --
    
    Can just remove the Javadoc so it is inherited from the base method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122858748
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java ---
    @@ -149,14 +149,24 @@ public IterOutcome innerNext() {
           if ( aggOut == HashAggregator.AggIterOutcome.AGG_OK ) { return IterOutcome.OK; }
           // if RESTART - continue below with doWork() - read some spilled partition, just like reading incoming
           incoming = aggregator.getNewIncoming(); // Restart - incoming was just changed
    -      if ( wasKilled ) { // if kill() was called before, then finish up
    -        aggregator.cleanup();
    -        incoming.kill(false);
    -        return IterOutcome.NONE;
    -      }
         }
     
    -    AggOutcome out = aggregator.doWork();
    +    if ( wasKilled ) { // if kill() was called before, then finish up
    +      aggregator.cleanup();
    +      incoming.kill(false);
    +      return IterOutcome.NONE;
    +    }
    +
    +    // Read and aggregate records
    +    // ( may need to run again if the spilled partition that was read
    +    //   generated new partitions that were all spilled )
    +    AggOutcome out = AggOutcome.CALL_WORK_AGAIN;
    +    while ( out == AggOutcome.CALL_WORK_AGAIN) {
    +      //
    +      //  Read incoming batches and process their records
    +      //
    +      out = aggregator.doWork();
    +    }
    --- End diff --
    
    Scratch that, I see you need the value of "out". So:
    ```
      AggOutcome out;
      do {
          //
          //  Read incoming batches and process their records
          //
          out = aggregator.doWork();
      } while (out == AggOutcome.CALL_WORK_AGAIN) {
    ```
    
    Or Even:
    ```
      //  Read incoming batches and process their records
      AggOutcome out;
      while ((out = aggregator.doWork()) == AggOutcome.CALL_WORK_AGAIN) {
        // Nothing to do
      }
    ```



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119241352
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java ---
    @@ -136,15 +136,21 @@ public IterOutcome innerNext() {
           return IterOutcome.NONE;
         }
     
    -    if (aggregator.buildComplete() && !aggregator.allFlushed()) {
    -      // aggregation is complete and not all records have been output yet
    -      return aggregator.outputCurrentBatch();
    +    // if aggregation is complete and not all records have been output yet
    +    if (aggregator.buildComplete() ||
    +        // or: 1st phase need to return (not fully grouped) partial output due to memory pressure
    +        aggregator.earlyOutput()) {
    +      // then output the next batch downstream
    +      IterOutcome out = aggregator.outputCurrentBatch();
    --- End diff --
    
    Done !!  Looks so simple in retrospective ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117104008
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java ---
    @@ -99,17 +100,31 @@ public void setCost(double cost) {
         this.cost = cost;
       }
     
    -  // Not available. Presumably because Drill does not currently use
    -  // this value, though it does appear in some test physical plans.
    -//  public void setMaxAllocation(long alloc) {
    -//    maxAllocation = alloc;
    -//  }
    -
       @Override
       public long getMaxAllocation() {
         return maxAllocation;
       }
     
    +  /**
    +   * Any operator that supports spilling should override this method
    +   * @param maxAllocation The max memory allocation to be set
    +   */
    +  @Override
    +  public void setMaxAllocation(long maxAllocation) {
    --- End diff --
    
    There was a problem with this method, which is why it was commented out. Trying to remember... Something about the internal, temporary serializations in planning causing values to be overwritten. Jackson will see this method and use it for deserializing the value. May have to fiddle around a bit to remember the issue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117107593
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -205,10 +225,10 @@ drill.exec: {
             // Deprecated for managed xsort; used only by legacy xsort
             threshold: 40000,
             // File system to use. Local file system by default.
    -        fs: "file:///"
    +        fs: ${drill.exec.spill.fs},
    --- End diff --
    
    Suggestion: explain what we are doing:
    
    In Drill versions prior to 1.11, only sort spilled. Starting in 1.11, other operators spill.
    Spilling is now configured in the spill group above. For backward compatibility, existing configuration files will still configure spilling here. Please move spill configuration to the new group above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117102340
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -86,6 +92,19 @@
     
       BooleanValidator EXTERNAL_SORT_DISABLE_MANAGED_OPTION = new BooleanValidator("exec.sort.disable_managed", false);
     
    +  // Hash Aggregate Options
    +
    +  String HASHAGG_NUM_PARTITIONS_KEY = "drill.exec.hashagg.num_partitions";
    --- End diff --
    
    For config options, the path is usually "drill.exec.something". But, for system/session options, there is no need for the top "drill." namespace, so they usually are of the form "exec.something" or even shorter. Check out some of the other names and you'll see the pattern.
    
    Or, am I confused? Normally, the option name appears as "key", then a validator, using that name, appears below. But, in the validator, we have a different name (one that does, in fact, follow the usual option rules.) So, we we have both a config and a system/session option?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118811633
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java ---
    @@ -136,15 +136,21 @@ public IterOutcome innerNext() {
           return IterOutcome.NONE;
         }
     
    -    if (aggregator.buildComplete() && !aggregator.allFlushed()) {
    -      // aggregation is complete and not all records have been output yet
    -      return aggregator.outputCurrentBatch();
    +    // if aggregation is complete and not all records have been output yet
    +    if (aggregator.buildComplete() ||
    +        // or: 1st phase need to return (not fully grouped) partial output due to memory pressure
    +        aggregator.earlyOutput()) {
    +      // then output the next batch downstream
    +      IterOutcome out = aggregator.outputCurrentBatch();
    --- End diff --
    
    Since `HashAggregator` is not an operator executor (AKA record batch), it does not have to follow the iterator protocol and use the `IterOutcome` enum. Instead, you can define your own. You won't need the `OK_NEW_SCHEMA`, `OUT_OF_MEMORY`, `FAIL` or `NOT_YET` values. All you seem to need is `OK`, `NONE` and `RESTART`.
    
    This approach will avoid the need to change the `IterOutcome` enum and export your states to all of the Drill iterator protocol.
    
    Did something similar in Sort for the iterator class that returns either in-memory or merged spilled batches.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119260493
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    --- End diff --
    
    Added:
    `logger.warn("Spilling was disabled - not enough memory available for internal partitioning");`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812151
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -230,15 +452,35 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           throw new IllegalArgumentException("Wrong number of workspace variables.");
         }
     
    -//    this.context = context;
    +    this.context = context;
         this.stats = stats;
    -    this.allocator = allocator;
    +    this.allocator = oContext.getAllocator();
    +    this.oContext = oContext;
         this.incoming = incoming;
    -//    this.schema = incoming.getSchema();
         this.outgoing = outgoing;
         this.outContainer = outContainer;
    +    this.operatorId = hashAggrConfig.getOperatorId();
    +
    +    is2ndPhase = hashAggrConfig.getAggPhase() == AggPrelBase.OperatorPhase.PHASE_2of2;
    +    isTwoPhase = hashAggrConfig.getAggPhase() != AggPrelBase.OperatorPhase.PHASE_1of1;
    +    canSpill = isTwoPhase; // single phase can not spill
    --- End diff --
    
    All this state is copied for each generated class. But, nothing here is specific to the types of the vectors in the batches. As noted above, all this stuff should be factored out into a non-template class with the template holding only that code which is type-specific.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119518393
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    +    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
    +    logger.trace("Incoming sizer: {}",sizer);
    +    // An empty batch only has the schema, can not tell actual length of varchars
    +    // else use the actual varchars length, each capped at 50 (to match the space allocation)
    +    estRowWidth = sizer.rowCount() == 0 ? sizer.stdRowWidth() : sizer.netRowWidthCap50();
    --- End diff --
    
    Like the previous comment above - this is dead code, a leftover. This code would only be called when the batch contains rows. Maybe should change this (and the one above) to an assert. 
     


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119977034
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
         }
     
    -    // get the number of records in the batch holder that are pending output
    -    int numPendingOutput = batchHolders.get(outBatchIndex).getNumPendingOutput();
    +    // Initialization (covers the case of early output)
    +    ArrayList<BatchHolder> currPartition = batchHolders[earlyPartition];
    +    int currOutBatchIndex = outBatchIndex[earlyPartition];
    +    int partitionToReturn = earlyPartition;
    +
    +    if ( ! earlyOutput ) {
    +      // Update the next partition to return (if needed)
    +      // skip fully returned (or spilled) partitions
    +      while (nextPartitionToReturn < numPartitions) {
    +        //
    +        // If this partition was spilled - spill the rest of it and skip it
    +        //
    +        if ( isSpilled(nextPartitionToReturn) ) {
    +          spillAPartition(nextPartitionToReturn); // spill the rest
    +          SpilledPartition sp = new SpilledPartition();
    +          sp.spillFile = spillFiles[nextPartitionToReturn];
    +          sp.spilledBatches = spilledBatchesCount[nextPartitionToReturn];
    +          sp.cycleNum = cycleNum; // remember the current cycle
    +          sp.origPartn = nextPartitionToReturn; // for debugging / filename
    +          sp.prevOrigPartn = originalPartition; // for debugging / filename
    +          spilledPartitionsList.add(sp);
    +          try {
    +            reinitPartition(nextPartitionToReturn); // free the memory
    +          } catch (Exception e) {throw new RuntimeException(e);}
    --- End diff --
    
    Actually now seems that reinitPartition() need not throw any exception !!
    Cleaned up all the catches, etc. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119749313
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -323,38 +717,32 @@ public AggOutcome doWork() {
                     if (EXTRA_DEBUG_1) {
                       logger.debug("Received new schema.  Batch has {} records.", incoming.getRecordCount());
                     }
    -//                newSchema = true;
                     this.cleanup();
                     // TODO: new schema case needs to be handled appropriately
                     return AggOutcome.UPDATE_AGGREGATOR;
     
                   case OK:
                     resetIndex();
    -                if (incoming.getRecordCount() == 0) {
    -                  continue;
    -                } else {
    -                  checkGroupAndAggrValues(currentIndex);
    -                  incIndex();
    -
    -                  if (EXTRA_DEBUG_1) {
    -                    logger.debug("Continuing outside loop");
    -                  }
    -                  continue outside;
    +
    +                if (EXTRA_DEBUG_1) {
    +                  logger.debug("Continuing outside loop");
                     }
    +                continue outside;
     
                   case NONE:
    -                // outcome = out;
    +                underlyingIndex = 0; // in case need to handle a spilled partition
    +                try { currentIndex = getVectorIndex(underlyingIndex); }
    +                catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
    --- End diff --
    
    Done -- all SCE catchers either call UnsupportedOperationException, or (when clearly not possible then) IllegalStateException 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117865877
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -546,44 +1254,204 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
          holder.value = vv0.getAccessor().get(incomingRowIdx) ;
          }
          */
    +    /*
    +    if ( handlingSpills && ( incomingRowIdx == 0 ) ) {
    +      // for debugging -- show the first row from a spilled batch
    +      Object tmp0 = (incoming).getValueAccessorById(NullableVarCharVector.class, 0).getValueVector();
    +      Object tmp1 = (incoming).getValueAccessorById(NullableVarCharVector.class, 1).getValueVector();
    +      Object tmp2 = (incoming).getValueAccessorById(NullableBigIntVector.class, 2).getValueVector();
    +
    +      if (tmp0 != null && tmp1 != null && tmp2 != null) {
    +        NullableVarCharVector vv0 = ((NullableVarCharVector) tmp0);
    +        NullableVarCharVector vv1 = ((NullableVarCharVector) tmp1);
    +        NullableBigIntVector  vv2 = ((NullableBigIntVector) tmp2);
    +        logger.debug("The first row = {} , {} , {}", vv0.getAccessor().get(incomingRowIdx), vv1.getAccessor().get(incomingRowIdx), vv2.getAccessor().get(incomingRowIdx));
    +      }
    +    }
    +    */
    +    // The hash code is computed once, then its lower bits are used to determine the
    +    // partition to use, and the higher bits determine the location in the hash table.
    +    int hashCode;
    +    try {
    +      htables[0].updateBatches();
    +      hashCode = htables[0].getHashCode(incomingRowIdx);
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
     
    -    htable.put(incomingRowIdx, htIdxHolder, 1 /* retry count */);
    +    // right shift hash code for secondary (or tertiary...) spilling
    +    for (int i = 0; i < cycleNum; i++) { hashCode >>>= bitsInMask; }
     
    +    int currentPartition = hashCode & partitionMask ;
    +    hashCode >>>= bitsInMask;
    +    HashTable.PutStatus putStatus = null;
    +    long allocatedBefore = allocator.getAllocatedMemory();
    +
    +    // Insert the key columns into the hash table
    +    try {
    +      putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode);
    +    } catch (OutOfMemoryException exc) {
    +      throw new OutOfMemoryException(getOOMErrorMsg(), exc); // may happen when can not spill
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
         int currentIdx = htIdxHolder.value;
     
    -    // get the batch index and index within the batch
    -    if (currentIdx >= batchHolders.size() * HashTable.BATCH_SIZE) {
    -      addBatchHolder();
    +    long addedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +    if ( addedMem > 0 ) {
    +      logger.trace("MEMORY CHECK HT: allocated {}  added {} partition {}",allocatedBefore,addedMem,currentPartition);
         }
    -    BatchHolder bh = batchHolders.get((currentIdx >>> 16) & HashTable.BATCH_MASK);
    +
    +    // Check if put() added a new batch (for the keys) inside the hash table, hence a matching batch
    +    // (for the aggregate columns) needs to be created
    +    if ( putStatus == HashTable.PutStatus.NEW_BATCH_ADDED ) {
    +      try {
    +        long allocatedBeforeAggCol = allocator.getAllocatedMemory();
    +
    +        addBatchHolder(currentPartition);
    +
    +        if ( plannedBatches > 0 ) { plannedBatches--; } // just allocated a planned batch
    +        long totalAddedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +        logger.trace("MEMORY CHECK AGG: added {}  total (with HT) added {}",allocator.getAllocatedMemory()-allocatedBeforeAggCol,totalAddedMem);
    +        // resize the batch estimate if needed (e.g., varchars may take more memory than estimated)
    +        if ( totalAddedMem > estMaxBatchSize ) {
    +          logger.trace("Adjusting Batch size estimate from {} to {}",estMaxBatchSize,totalAddedMem);
    +          estMaxBatchSize = totalAddedMem;
    +        }
    +      } catch (OutOfMemoryException exc) {
    +        throw new OutOfMemoryException(getOOMErrorMsg(), exc); // may happen when can not spill
    +      }
    +    }
    +    BatchHolder bh = batchHolders[currentPartition].get((currentIdx >>> 16) & HashTable.BATCH_MASK);
         int idxWithinBatch = currentIdx & HashTable.BATCH_MASK;
     
    +    if (bh.updateAggrValues(incomingRowIdx, idxWithinBatch)) {
    +      numGroupedRecords++;
    +    }
    +
    +    // ===================================================================================
    +    // If the last batch just became full - that is the time to check the memory limits !!
    +    // If exceeded, then need to spill (if 2nd phase) or output early (1st)
    +    // (Skip this if cannot spill; in such case an OOM may be encountered later)
    +    // ===================================================================================
    +    if ( putStatus == HashTable.PutStatus.KEY_ADDED_LAST && canSpill ) {
    +
    +      plannedBatches++; // planning to allocate one more batch
    +
    +      // calculate the (max) new memory needed now
    +      long hashTableDoublingSizeNeeded = 0; // in case the hash table(s) would resize
    +      for ( HashTable ht : htables ) {
    --- End diff --
    
    The check for memory pressure takes place whenever any new batch becomes full (because soon after a new batch would be needed by that partition). However the current incoming batch may still hold many unprocessed rows, destined to all the partitions, and these rows could trigger resizing at each and every partition (in the worst case).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122323606
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -512,122 +509,122 @@ private void updateEstMaxBatchSize(RecordBatch incoming) {
         }
       }
     
    +  /**
    +   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
    +   *  Once complete, get the incoming NEXT batch and process it as well, etc.
    +   *  For 1st phase, may return when an early output needs to be performed.
    +   *
    +   * @return Agg outcome status
    +   */
       @Override
       public AggOutcome doWork() {
    -    try {
    -      // Note: Keeping the outer and inner try blocks here to maintain some similarity with
    -      // StreamingAggregate which does somethings conditionally in the outer try block.
    -      // 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) {
    +    while (true) {
     
    -        // This would be called only once - after actual data arrives on incoming
    -        if ( schema == null && incoming.getRecordCount() > 0 ) {
    -          this.schema = incoming.getSchema();
    -          // Calculate the number of partitions based on actual incoming data
    -          delayedSetup();
    -        }
    +      // This would be called only once - first time actual data arrives on incoming
    +      if ( schema == null && incoming.getRecordCount() > 0 ) {
    +        this.schema = incoming.getSchema();
    +        currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
    +        // Calculate the number of partitions based on actual incoming data
    +        delayedSetup();
    +      }
     
    -        // loop through existing records, aggregating the values as necessary.
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Starting outer loop of doWork()...");
    +      //
    +      //  loop through existing records in this batch, aggregating the values as necessary.
    +      //
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Starting outer loop of doWork()...");
    +      }
    +      for (; underlyingIndex < currentBatchRecordCount; incIndex()) {
    +        if (EXTRA_DEBUG_2) {
    +          logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
             }
    -        for (; underlyingIndex < incoming.getRecordCount(); incIndex()) {
    -          if (EXTRA_DEBUG_2) {
    -            logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
    -          }
    -          checkGroupAndAggrValues(currentIndex);
    -          // If adding a group discovered a memory pressure during 1st phase, then start
    -          // outputing some partition to free memory.
    -          if ( earlyOutput ) {
    -            outputCurrentBatch();
    -            incIndex(); // next time continue with the next incoming row
    -            return AggOutcome.RETURN_OUTCOME;
    -          }
    +        checkGroupAndAggrValues(currentIndex);
    +        // If adding a group discovered a memory pressure during 1st phase, then start
    +        // outputing some partition downstream in order to free memory.
    +        if ( earlyOutput ) {
    +          outputCurrentBatch();
    +          incIndex(); // next time continue with the next incoming row
    +          return AggOutcome.RETURN_OUTCOME;
             }
    +      }
    +
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Processed {} records", underlyingIndex);
    +      }
     
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Processed {} records", underlyingIndex);
    +      // Cleanup the previous batch since we are done processing it.
    +      for (VectorWrapper<?> v : incoming) {
    +        v.getValueVector().clear();
    +      }
    +      //
    +      // Get the NEXT input batch, initially from the upstream, later (if there was a spill)
    +      // from one of the spill files (The spill case is handled differently here to avoid
    +      // collecting stats on the spilled records)
    +      //
    +      if ( handlingSpills ) {
    +        outcome = context.shouldContinue() ? incoming.next() : IterOutcome.STOP;
    +      } else {
    +        long beforeAlloc = allocator.getAllocatedMemory();
    +
    +        // Get the next RecordBatch from the incoming (i.e. upstream operator)
    +        outcome = outgoing.next(0, incoming);
    +
    +        // If incoming batch is bigger than our estimate - adjust the estimate to match
    +        long afterAlloc = allocator.getAllocatedMemory();
    +        long incomingBatchSize = afterAlloc - beforeAlloc;
    +        if ( estMaxBatchSize < incomingBatchSize) {
    +          logger.trace("Found a bigger incoming batch: {} , prior estimate was: {}", incomingBatchSize, estMaxBatchSize);
    +          estMaxBatchSize = incomingBatchSize;
             }
    +      }
     
    -        try {
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Received IterOutcome of {}", outcome);
    +      }
     
    -          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();
    +      // Handle various results from getting the next batch
    +      switch (outcome) {
    +        case OUT_OF_MEMORY:
    +        case NOT_YET:
    --- End diff --
    
    Again - this is the original Hash Agg code (just re-indented, as the while(true) and try blocks were removed).  The "outcome" comes from getting the next incoming batch, hence OUT_OF_MEMORY may (?) occur if a grossly abnormal incoming batch shows up unexpectedly. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122312907
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -512,122 +509,122 @@ private void updateEstMaxBatchSize(RecordBatch incoming) {
         }
       }
     
    +  /**
    +   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
    +   *  Once complete, get the incoming NEXT batch and process it as well, etc.
    +   *  For 1st phase, may return when an early output needs to be performed.
    +   *
    +   * @return Agg outcome status
    +   */
       @Override
       public AggOutcome doWork() {
    -    try {
    -      // Note: Keeping the outer and inner try blocks here to maintain some similarity with
    -      // StreamingAggregate which does somethings conditionally in the outer try block.
    -      // 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) {
    +    while (true) {
     
    -        // This would be called only once - after actual data arrives on incoming
    -        if ( schema == null && incoming.getRecordCount() > 0 ) {
    -          this.schema = incoming.getSchema();
    -          // Calculate the number of partitions based on actual incoming data
    -          delayedSetup();
    -        }
    +      // This would be called only once - first time actual data arrives on incoming
    +      if ( schema == null && incoming.getRecordCount() > 0 ) {
    +        this.schema = incoming.getSchema();
    +        currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
    +        // Calculate the number of partitions based on actual incoming data
    +        delayedSetup();
    +      }
     
    -        // loop through existing records, aggregating the values as necessary.
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Starting outer loop of doWork()...");
    +      //
    +      //  loop through existing records in this batch, aggregating the values as necessary.
    +      //
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Starting outer loop of doWork()...");
    +      }
    +      for (; underlyingIndex < currentBatchRecordCount; incIndex()) {
    +        if (EXTRA_DEBUG_2) {
    +          logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
             }
    -        for (; underlyingIndex < incoming.getRecordCount(); incIndex()) {
    -          if (EXTRA_DEBUG_2) {
    -            logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
    -          }
    -          checkGroupAndAggrValues(currentIndex);
    -          // If adding a group discovered a memory pressure during 1st phase, then start
    -          // outputing some partition to free memory.
    -          if ( earlyOutput ) {
    -            outputCurrentBatch();
    -            incIndex(); // next time continue with the next incoming row
    -            return AggOutcome.RETURN_OUTCOME;
    -          }
    +        checkGroupAndAggrValues(currentIndex);
    +        // If adding a group discovered a memory pressure during 1st phase, then start
    +        // outputing some partition downstream in order to free memory.
    +        if ( earlyOutput ) {
    +          outputCurrentBatch();
    +          incIndex(); // next time continue with the next incoming row
    +          return AggOutcome.RETURN_OUTCOME;
             }
    +      }
    +
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Processed {} records", underlyingIndex);
    +      }
     
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Processed {} records", underlyingIndex);
    +      // Cleanup the previous batch since we are done processing it.
    +      for (VectorWrapper<?> v : incoming) {
    +        v.getValueVector().clear();
    +      }
    +      //
    +      // Get the NEXT input batch, initially from the upstream, later (if there was a spill)
    +      // from one of the spill files (The spill case is handled differently here to avoid
    +      // collecting stats on the spilled records)
    +      //
    +      if ( handlingSpills ) {
    +        outcome = context.shouldContinue() ? incoming.next() : IterOutcome.STOP;
    +      } else {
    +        long beforeAlloc = allocator.getAllocatedMemory();
    +
    +        // Get the next RecordBatch from the incoming (i.e. upstream operator)
    +        outcome = outgoing.next(0, incoming);
    +
    +        // If incoming batch is bigger than our estimate - adjust the estimate to match
    +        long afterAlloc = allocator.getAllocatedMemory();
    +        long incomingBatchSize = afterAlloc - beforeAlloc;
    +        if ( estMaxBatchSize < incomingBatchSize) {
    +          logger.trace("Found a bigger incoming batch: {} , prior estimate was: {}", incomingBatchSize, estMaxBatchSize);
    +          estMaxBatchSize = incomingBatchSize;
             }
    +      }
     
    -        try {
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Received IterOutcome of {}", outcome);
    +      }
     
    -          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();
    +      // Handle various results from getting the next batch
    +      switch (outcome) {
    +        case OUT_OF_MEMORY:
    +        case NOT_YET:
    --- End diff --
    
    The `OUT_OF_MEMORY` is really an error condition. `NOT_YET` is not yet supported. is the `RETURN_OUTCOME` the proper return value? Or, should this throw a user exception?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117106034
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java ---
    @@ -133,6 +133,9 @@
          the need to turn off join optimization may go away.
        */
       public static final BooleanValidator JOIN_OPTIMIZATION = new BooleanValidator("planner.enable_join_optimization", true);
    +  // for testing purpose
    --- End diff --
    
    You can use the handy Guava \@VisibleForTesting annotation here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117078902
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    --- End diff --
    
    You could add the reason why it was disabled in this case since it happened without user intervention. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119500755
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    --- End diff --
    
    All "throw ClassTransformationException" clauses were removed (how come Intellij did not mark them as redundant ?)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119748684
  
    --- 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:
    --- End diff --
    
    RESTART was eliminated ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813677
  
    --- 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 ) {
    +          this.schema = incoming.getSchema();
    +          // Calculate the number of partitions based on actual incoming data
    +          delayedSetup();
    +        }
    +
    --- End diff --
    
    Comment on original code below: no need for the EXTRA_DEBUG_1 and EXTRA_DEBUG_2: just use trace level logging and avoid the extra complexity.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122312536
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -512,122 +509,122 @@ private void updateEstMaxBatchSize(RecordBatch incoming) {
         }
       }
     
    +  /**
    +   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
    +   *  Once complete, get the incoming NEXT batch and process it as well, etc.
    +   *  For 1st phase, may return when an early output needs to be performed.
    +   *
    +   * @return Agg outcome status
    +   */
       @Override
       public AggOutcome doWork() {
    -    try {
    -      // Note: Keeping the outer and inner try blocks here to maintain some similarity with
    -      // StreamingAggregate which does somethings conditionally in the outer try block.
    -      // 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) {
    +    while (true) {
     
    -        // This would be called only once - after actual data arrives on incoming
    -        if ( schema == null && incoming.getRecordCount() > 0 ) {
    -          this.schema = incoming.getSchema();
    -          // Calculate the number of partitions based on actual incoming data
    -          delayedSetup();
    -        }
    +      // This would be called only once - first time actual data arrives on incoming
    +      if ( schema == null && incoming.getRecordCount() > 0 ) {
    +        this.schema = incoming.getSchema();
    +        currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
    +        // Calculate the number of partitions based on actual incoming data
    +        delayedSetup();
    +      }
     
    -        // loop through existing records, aggregating the values as necessary.
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Starting outer loop of doWork()...");
    +      //
    +      //  loop through existing records in this batch, aggregating the values as necessary.
    +      //
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Starting outer loop of doWork()...");
    +      }
    +      for (; underlyingIndex < currentBatchRecordCount; incIndex()) {
    +        if (EXTRA_DEBUG_2) {
    +          logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
             }
    -        for (; underlyingIndex < incoming.getRecordCount(); incIndex()) {
    -          if (EXTRA_DEBUG_2) {
    -            logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
    -          }
    -          checkGroupAndAggrValues(currentIndex);
    -          // If adding a group discovered a memory pressure during 1st phase, then start
    -          // outputing some partition to free memory.
    -          if ( earlyOutput ) {
    -            outputCurrentBatch();
    -            incIndex(); // next time continue with the next incoming row
    -            return AggOutcome.RETURN_OUTCOME;
    -          }
    +        checkGroupAndAggrValues(currentIndex);
    +        // If adding a group discovered a memory pressure during 1st phase, then start
    +        // outputing some partition downstream in order to free memory.
    +        if ( earlyOutput ) {
    +          outputCurrentBatch();
    +          incIndex(); // next time continue with the next incoming row
    +          return AggOutcome.RETURN_OUTCOME;
             }
    +      }
    +
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Processed {} records", underlyingIndex);
    +      }
     
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Processed {} records", underlyingIndex);
    +      // Cleanup the previous batch since we are done processing it.
    +      for (VectorWrapper<?> v : incoming) {
    --- End diff --
    
    Two comments/questions here.
    
    First, can we be sure that there is always a Selection Vector Remover between the hash agg and anything that can emit a batch that uses an SV4 (such as sort)? Otherwise, the `getValueVector()` method won't work.
    
    Second, is it certain that no other code references the vectors from this container?
    
    If both those invariants are met, then, yes, it is the job of this operator to release buffers created by the upstream.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122324273
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -512,122 +509,122 @@ private void updateEstMaxBatchSize(RecordBatch incoming) {
         }
       }
     
    +  /**
    +   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
    +   *  Once complete, get the incoming NEXT batch and process it as well, etc.
    +   *  For 1st phase, may return when an early output needs to be performed.
    +   *
    +   * @return Agg outcome status
    +   */
       @Override
       public AggOutcome doWork() {
    -    try {
    -      // Note: Keeping the outer and inner try blocks here to maintain some similarity with
    -      // StreamingAggregate which does somethings conditionally in the outer try block.
    -      // 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) {
    +    while (true) {
     
    -        // This would be called only once - after actual data arrives on incoming
    -        if ( schema == null && incoming.getRecordCount() > 0 ) {
    -          this.schema = incoming.getSchema();
    -          // Calculate the number of partitions based on actual incoming data
    -          delayedSetup();
    -        }
    +      // This would be called only once - first time actual data arrives on incoming
    +      if ( schema == null && incoming.getRecordCount() > 0 ) {
    +        this.schema = incoming.getSchema();
    +        currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
    +        // Calculate the number of partitions based on actual incoming data
    +        delayedSetup();
    +      }
     
    -        // loop through existing records, aggregating the values as necessary.
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Starting outer loop of doWork()...");
    +      //
    +      //  loop through existing records in this batch, aggregating the values as necessary.
    +      //
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Starting outer loop of doWork()...");
    +      }
    +      for (; underlyingIndex < currentBatchRecordCount; incIndex()) {
    +        if (EXTRA_DEBUG_2) {
    +          logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
             }
    -        for (; underlyingIndex < incoming.getRecordCount(); incIndex()) {
    -          if (EXTRA_DEBUG_2) {
    -            logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
    -          }
    -          checkGroupAndAggrValues(currentIndex);
    -          // If adding a group discovered a memory pressure during 1st phase, then start
    -          // outputing some partition to free memory.
    -          if ( earlyOutput ) {
    -            outputCurrentBatch();
    -            incIndex(); // next time continue with the next incoming row
    -            return AggOutcome.RETURN_OUTCOME;
    -          }
    +        checkGroupAndAggrValues(currentIndex);
    +        // If adding a group discovered a memory pressure during 1st phase, then start
    +        // outputing some partition downstream in order to free memory.
    +        if ( earlyOutput ) {
    +          outputCurrentBatch();
    +          incIndex(); // next time continue with the next incoming row
    +          return AggOutcome.RETURN_OUTCOME;
             }
    +      }
    +
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Processed {} records", underlyingIndex);
    +      }
     
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Processed {} records", underlyingIndex);
    +      // Cleanup the previous batch since we are done processing it.
    +      for (VectorWrapper<?> v : incoming) {
    +        v.getValueVector().clear();
    +      }
    +      //
    +      // Get the NEXT input batch, initially from the upstream, later (if there was a spill)
    +      // from one of the spill files (The spill case is handled differently here to avoid
    +      // collecting stats on the spilled records)
    +      //
    +      if ( handlingSpills ) {
    +        outcome = context.shouldContinue() ? incoming.next() : IterOutcome.STOP;
    +      } else {
    +        long beforeAlloc = allocator.getAllocatedMemory();
    +
    +        // Get the next RecordBatch from the incoming (i.e. upstream operator)
    +        outcome = outgoing.next(0, incoming);
    +
    +        // If incoming batch is bigger than our estimate - adjust the estimate to match
    +        long afterAlloc = allocator.getAllocatedMemory();
    +        long incomingBatchSize = afterAlloc - beforeAlloc;
    +        if ( estMaxBatchSize < incomingBatchSize) {
    +          logger.trace("Found a bigger incoming batch: {} , prior estimate was: {}", incomingBatchSize, estMaxBatchSize);
    +          estMaxBatchSize = incomingBatchSize;
             }
    +      }
     
    -        try {
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Received IterOutcome of {}", outcome);
    +      }
     
    -          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();
    +      // Handle various results from getting the next batch
    +      switch (outcome) {
    +        case OUT_OF_MEMORY:
    +        case NOT_YET:
    +          return AggOutcome.RETURN_OUTCOME;
     
    -            // Get the next RecordBatch from the incoming
    -            IterOutcome out = outgoing.next(0, incoming);
    +        case OK_NEW_SCHEMA:
    --- End diff --
    
    This case leads to immediate failure; even the first batch does not get here. The error message returned is "Hash aggregate does not support schema change" . (Again -- all is original code). 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119258769
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -230,15 +452,35 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           throw new IllegalArgumentException("Wrong number of workspace variables.");
         }
     
    -//    this.context = context;
    +    this.context = context;
         this.stats = stats;
    -    this.allocator = allocator;
    +    this.allocator = oContext.getAllocator();
    +    this.oContext = oContext;
         this.incoming = incoming;
    -//    this.schema = incoming.getSchema();
         this.outgoing = outgoing;
         this.outContainer = outContainer;
    +    this.operatorId = hashAggrConfig.getOperatorId();
    +
    +    is2ndPhase = hashAggrConfig.getAggPhase() == AggPrelBase.OperatorPhase.PHASE_2of2;
    +    isTwoPhase = hashAggrConfig.getAggPhase() != AggPrelBase.OperatorPhase.PHASE_1of1;
    +    canSpill = isTwoPhase; // single phase can not spill
    --- End diff --
    
    Three booleans is 2^3 ( == 8), and this space is sparse as there are dependencies (e.g. Single phase can not spill, and has no 1st/2nd). 
      Those booleans are used only in a couple of places in the code to make a logical difference, hence it'll be an overkill to subclass based on these changes. They are used in many other places for logging messages ....


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117869278
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -64,6 +64,12 @@
       String SPOOLING_BUFFER_MEMORY = "drill.exec.buffer.spooling.size";
       String BATCH_PURGE_THRESHOLD = "drill.exec.sort.purge.threshold";
     
    +  // Spill Options common to all spilling operators
    --- End diff --
    
    OK, boot-time. Wish Drill had a clear way to distinguish boot-time from session options.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118811524
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java ---
    @@ -99,17 +100,31 @@ public void setCost(double cost) {
         this.cost = cost;
       }
     
    -  // Not available. Presumably because Drill does not currently use
    -  // this value, though it does appear in some test physical plans.
    -//  public void setMaxAllocation(long alloc) {
    -//    maxAllocation = alloc;
    -//  }
    -
       @Override
       public long getMaxAllocation() {
         return maxAllocation;
       }
     
    +  /**
    +   * Any operator that supports spilling should override this method
    +   * @param maxAllocation The max memory allocation to be set
    +   */
    +  @Override
    +  public void setMaxAllocation(long maxAllocation) {
    --- End diff --
    
    If we just want to fool Jackson we can use `@JsonIgnore`. Would be better to track down the root cause -- whatever it was.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812261
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    --- End diff --
    
    Not sure this is even legal. A record batch must have a schema, even if the schema is an empty set of columns.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118153400
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java ---
    @@ -49,12 +49,19 @@ public int getOperatorType() {
         return CoreOperatorType.EXTERNAL_SORT_VALUE;
       }
     
    -  // Set here, rather than the base class, because this is the only
    -  // operator, at present, that makes use of the maximum allocation.
    -  // Remove this, in favor of the base class version, when Drill
    -  // sets the memory allocation for all operators.
    -
    +  /**
    --- End diff --
    
    But still need a local comment to be clear about this code .... so does not matter much. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119518069
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    --- End diff --
    
    This if() is never taken. This method is only invoked once, called by delayedSetup() which in turn is only called once, inside this if():
    
    ```
            // This would be called only once - after actual data arrives on incoming
            if ( schema == null && incoming.getRecordCount() > 0 ) {
              this.schema = incoming.getSchema();
              // Calculate the number of partitions based on actual incoming data
              delayedSetup();
            }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117104450
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java ---
    @@ -99,17 +100,31 @@ public void setCost(double cost) {
         this.cost = cost;
       }
     
    -  // Not available. Presumably because Drill does not currently use
    -  // this value, though it does appear in some test physical plans.
    -//  public void setMaxAllocation(long alloc) {
    -//    maxAllocation = alloc;
    -//  }
    -
       @Override
       public long getMaxAllocation() {
         return maxAllocation;
       }
     
    +  /**
    +   * Any operator that supports spilling should override this method
    +   * @param maxAllocation The max memory allocation to be set
    +   */
    +  @Override
    +  public void setMaxAllocation(long maxAllocation) {
    +    this.maxAllocation = maxAllocation;
    +    /*throw new DrillRuntimeException("Unsupported method: setMaxAllocation()");*/
    +  }
    +
    +  /**
    +   * Any operator that supports spilling should override this method (and return true)
    +   * @return false
    +   */
    +  @Override
    +  public boolean getBufferedOperator() { return false; }
    --- End diff --
    
    get --> is
    I believe Jackson supports the common "is" idiom for booleans. See [this post|http://stackoverflow.com/questions/32270422/jackson-renames-primitive-boolean-field-by-removing-is].


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117108685
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.drill.exec.physical.impl.agg;
    +
    +import ch.qos.logback.classic.Level;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.physical.impl.aggregate.HashAggTemplate;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.proto.UserBitShared;
    +import org.apache.drill.test.ClientFixture;
    +import org.apache.drill.test.ClusterFixture;
    +import org.apache.drill.test.FixtureBuilder;
    +import org.apache.drill.test.LogFixture;
    +import org.apache.drill.test.ProfileParser;
    +import org.apache.drill.test.QueryBuilder;
    +import org.junit.Test;
    +
    +import java.util.List;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +/**
    + *  Test spilling for the Hash Aggr operator (using the mock reader)
    + */
    +public class TestHashAggrSpill extends BaseTestQuery {
    +
    +    private void runAndDump(ClientFixture client, String sql, long expectedRows, long spillCycle, long spilledPartitions) throws Exception {
    +        String plan = client.queryBuilder().sql(sql).explainJson();
    +
    +        QueryBuilder.QuerySummary summary = client.queryBuilder().sql(sql).run();
    +        if ( expectedRows > 0 ) {
    +            assertEquals(expectedRows, summary.recordCount());
    +        }
    +        // System.out.println(String.format("======== \n Results: %,d records, %d batches, %,d ms\n ========", summary.recordCount(), summary.batchCount(), summary.runTimeMs() ) );
    +
    +        //System.out.println("Query ID: " + summary.queryIdString());
    +        ProfileParser profile = client.parseProfile(summary.queryIdString());
    +        //profile.print();
    +        List<ProfileParser.OperatorProfile> ops = profile.getOpsOfType(UserBitShared.CoreOperatorType.HASH_AGGREGATE_VALUE);
    +
    +        assertTrue( ! ops.isEmpty() );
    +        // check for the first op only
    +        ProfileParser.OperatorProfile hag0 = ops.get(0);
    +        long opCycle = hag0.getMetric(HashAggTemplate.Metric.SPILL_CYCLE.ordinal());
    +        assertEquals(spillCycle, opCycle);
    +        long op_spilled_partitions = hag0.getMetric(HashAggTemplate.Metric.SPILLED_PARTITIONS.ordinal());
    +        assertEquals(spilledPartitions, op_spilled_partitions);
    +        /* assertEquals(3, ops.size());
    +        for ( int i = 0; i < ops.size(); i++ ) {
    +            ProfileParser.OperatorProfile hag = ops.get(i);
    +            long cycle = hag.getMetric(HashAggTemplate.Metric.SPILL_CYCLE.ordinal());
    +            long num_partitions = hag.getMetric(HashAggTemplate.Metric.NUM_PARTITIONS.ordinal());
    +            long spilled_partitions = hag.getMetric(HashAggTemplate.Metric.SPILLED_PARTITIONS.ordinal());
    +            long mb_spilled = hag.getMetric(HashAggTemplate.Metric.SPILL_MB.ordinal());
    +            System.out.println(String.format("(%d) Spill cycle: %d, num partitions: %d, spilled partitions: %d, MB spilled: %d", i,cycle, num_partitions, spilled_partitions,
    +                mb_spilled));
    +        } */
    +    }
    +
    +    /**
    +     * Test "normal" spilling: Only 2 partitions (out of 4) would require spilling
    +     * ("normal spill" means spill-cycle = 1 )
    +     *
    +     * @throws Exception
    +     */
    +    @Test
    +    public void testHashAggrSpill() throws Exception {
    +        LogFixture.LogFixtureBuilder logBuilder = LogFixture.builder()
    +            .toConsole()
    +            .logger("org.apache.drill.exec.physical.impl.aggregate", Level.WARN)
    +            ;
    +
    +        FixtureBuilder builder = ClusterFixture.builder()
    +            .configProperty(ExecConstants.HASHAGG_MAX_MEMORY_KEY,62_000_000)
    +            .configProperty(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY,16)
    +            .configProperty(ExecConstants.HASHAGG_MIN_BATCHES_PER_PARTITION_KEY,3)
    +            .sessionOption(PlannerSettings.FORCE_2PHASE_AGGR_KEY,true)
    +            // .sessionOption(PlannerSettings.EXCHANGE.getOptionName(), true)
    +            .maxParallelization(2)
    +            .saveProfiles()
    +            //.keepLocalFiles()
    +            ;
    +        try (LogFixture logs = logBuilder.build();
    +             ClusterFixture cluster = builder.build();
    +             ClientFixture client = cluster.clientFixture()) {
    +            String sql = "SELECT empid_s17, dept_i, branch_i, AVG(salary_i) FROM `mock`.`employee_1200K` GROUP BY empid_s17, dept_i, branch_i";
    +            runAndDump(client, sql, 1_200_000, 1, 1);
    +        }
    +    }
    +
    +    /**
    +     *  Test "secondary" spilling -- Some of the spilled partitions cause more spilling as they are read back
    +     *  (Hence spill-cycle = 2 )
    +     *
    +     * @throws Exception
    +     */
    +    @Test
    --- End diff --
    
    Question: are two tests sufficient to test all the code paths in the modified code? Might we need test for specific cases? Will provide detailed suggestions as I review the actual spill code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118811998
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    +    private VectorContainer container = null;
    --- End diff --
    
    No need to initialize to null; Java does that for you for member variables (AKA fields.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122324618
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -512,122 +509,122 @@ private void updateEstMaxBatchSize(RecordBatch incoming) {
         }
       }
     
    +  /**
    +   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
    +   *  Once complete, get the incoming NEXT batch and process it as well, etc.
    +   *  For 1st phase, may return when an early output needs to be performed.
    +   *
    +   * @return Agg outcome status
    +   */
       @Override
       public AggOutcome doWork() {
    -    try {
    -      // Note: Keeping the outer and inner try blocks here to maintain some similarity with
    -      // StreamingAggregate which does somethings conditionally in the outer try block.
    -      // 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) {
    +    while (true) {
     
    -        // This would be called only once - after actual data arrives on incoming
    -        if ( schema == null && incoming.getRecordCount() > 0 ) {
    -          this.schema = incoming.getSchema();
    -          // Calculate the number of partitions based on actual incoming data
    -          delayedSetup();
    -        }
    +      // This would be called only once - first time actual data arrives on incoming
    +      if ( schema == null && incoming.getRecordCount() > 0 ) {
    +        this.schema = incoming.getSchema();
    +        currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
    +        // Calculate the number of partitions based on actual incoming data
    +        delayedSetup();
    +      }
     
    -        // loop through existing records, aggregating the values as necessary.
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Starting outer loop of doWork()...");
    +      //
    +      //  loop through existing records in this batch, aggregating the values as necessary.
    +      //
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Starting outer loop of doWork()...");
    +      }
    +      for (; underlyingIndex < currentBatchRecordCount; incIndex()) {
    +        if (EXTRA_DEBUG_2) {
    +          logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
             }
    -        for (; underlyingIndex < incoming.getRecordCount(); incIndex()) {
    -          if (EXTRA_DEBUG_2) {
    -            logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
    -          }
    -          checkGroupAndAggrValues(currentIndex);
    -          // If adding a group discovered a memory pressure during 1st phase, then start
    -          // outputing some partition to free memory.
    -          if ( earlyOutput ) {
    -            outputCurrentBatch();
    -            incIndex(); // next time continue with the next incoming row
    -            return AggOutcome.RETURN_OUTCOME;
    -          }
    +        checkGroupAndAggrValues(currentIndex);
    +        // If adding a group discovered a memory pressure during 1st phase, then start
    +        // outputing some partition downstream in order to free memory.
    +        if ( earlyOutput ) {
    +          outputCurrentBatch();
    +          incIndex(); // next time continue with the next incoming row
    +          return AggOutcome.RETURN_OUTCOME;
             }
    +      }
    +
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Processed {} records", underlyingIndex);
    +      }
     
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Processed {} records", underlyingIndex);
    +      // Cleanup the previous batch since we are done processing it.
    +      for (VectorWrapper<?> v : incoming) {
    +        v.getValueVector().clear();
    +      }
    +      //
    +      // Get the NEXT input batch, initially from the upstream, later (if there was a spill)
    +      // from one of the spill files (The spill case is handled differently here to avoid
    +      // collecting stats on the spilled records)
    +      //
    +      if ( handlingSpills ) {
    +        outcome = context.shouldContinue() ? incoming.next() : IterOutcome.STOP;
    +      } else {
    +        long beforeAlloc = allocator.getAllocatedMemory();
    +
    +        // Get the next RecordBatch from the incoming (i.e. upstream operator)
    +        outcome = outgoing.next(0, incoming);
    +
    +        // If incoming batch is bigger than our estimate - adjust the estimate to match
    +        long afterAlloc = allocator.getAllocatedMemory();
    +        long incomingBatchSize = afterAlloc - beforeAlloc;
    +        if ( estMaxBatchSize < incomingBatchSize) {
    +          logger.trace("Found a bigger incoming batch: {} , prior estimate was: {}", incomingBatchSize, estMaxBatchSize);
    +          estMaxBatchSize = incomingBatchSize;
             }
    +      }
     
    -        try {
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Received IterOutcome of {}", outcome);
    +      }
     
    -          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();
    +      // Handle various results from getting the next batch
    +      switch (outcome) {
    +        case OUT_OF_MEMORY:
    +        case NOT_YET:
    --- End diff --
    
    The problem with `OUT_OF_MEMORY` is that it basically does not work. But, if this is original code, then we can address that problem another time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117861343
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    --- End diff --
    
    When spilling a partition, all its batches (except sometimes the last one) are full (i.e. 64K); hence no need to combine. (These are not the incoming batches, but new batches constructed inside the Hash Aggr). 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119259090
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813769
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -323,38 +717,32 @@ public AggOutcome doWork() {
                     if (EXTRA_DEBUG_1) {
                       logger.debug("Received new schema.  Batch has {} records.", incoming.getRecordCount());
                     }
    -//                newSchema = true;
                     this.cleanup();
                     // TODO: new schema case needs to be handled appropriately
                     return AggOutcome.UPDATE_AGGREGATOR;
     
                   case OK:
                     resetIndex();
    -                if (incoming.getRecordCount() == 0) {
    -                  continue;
    -                } else {
    -                  checkGroupAndAggrValues(currentIndex);
    -                  incIndex();
    -
    -                  if (EXTRA_DEBUG_1) {
    -                    logger.debug("Continuing outside loop");
    -                  }
    -                  continue outside;
    +
    +                if (EXTRA_DEBUG_1) {
    +                  logger.debug("Continuing outside loop");
                     }
    +                continue outside;
     
                   case NONE:
    -                // outcome = out;
    +                underlyingIndex = 0; // in case need to handle a spilled partition
    +                try { currentIndex = getVectorIndex(underlyingIndex); }
    +                catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
    --- End diff --
    
    `IllegalStateException`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118804514
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -64,6 +64,12 @@
       String SPOOLING_BUFFER_MEMORY = "drill.exec.buffer.spooling.size";
       String BATCH_PURGE_THRESHOLD = "drill.exec.sort.purge.threshold";
     
    +  // Spill Options common to all spilling operators
    --- End diff --
    
    Looks like a solution is coming. See DRILL-5547.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813870
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
    --- End diff --
    
    `IllegalStateException`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122324232
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -130,6 +127,7 @@
       private int currentIndex = 0;
       private IterOutcome outcome;
       private int numGroupedRecords = 0;
    +  private int currentBatchRecordCount = 0; // Performance: Avoid repeated calls to getRecordCount()
    --- End diff --
    
    Thanks for the explanation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117078208
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    --- End diff --
    
    Suppose numPendingOutput is small (e.g 100 rows) but there are several such batches,  we would end up calling allocateOutgoing() multiple times for small amounts...shouldn't we try to combine these into smaller number of outgoing batches ? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122322345
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -512,122 +509,122 @@ private void updateEstMaxBatchSize(RecordBatch incoming) {
         }
       }
     
    +  /**
    +   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
    +   *  Once complete, get the incoming NEXT batch and process it as well, etc.
    +   *  For 1st phase, may return when an early output needs to be performed.
    +   *
    +   * @return Agg outcome status
    +   */
       @Override
       public AggOutcome doWork() {
    -    try {
    -      // Note: Keeping the outer and inner try blocks here to maintain some similarity with
    -      // StreamingAggregate which does somethings conditionally in the outer try block.
    -      // 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) {
    +    while (true) {
     
    -        // This would be called only once - after actual data arrives on incoming
    -        if ( schema == null && incoming.getRecordCount() > 0 ) {
    -          this.schema = incoming.getSchema();
    -          // Calculate the number of partitions based on actual incoming data
    -          delayedSetup();
    -        }
    +      // This would be called only once - first time actual data arrives on incoming
    +      if ( schema == null && incoming.getRecordCount() > 0 ) {
    +        this.schema = incoming.getSchema();
    +        currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
    +        // Calculate the number of partitions based on actual incoming data
    +        delayedSetup();
    +      }
     
    -        // loop through existing records, aggregating the values as necessary.
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Starting outer loop of doWork()...");
    +      //
    +      //  loop through existing records in this batch, aggregating the values as necessary.
    +      //
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Starting outer loop of doWork()...");
    +      }
    +      for (; underlyingIndex < currentBatchRecordCount; incIndex()) {
    +        if (EXTRA_DEBUG_2) {
    +          logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
             }
    -        for (; underlyingIndex < incoming.getRecordCount(); incIndex()) {
    -          if (EXTRA_DEBUG_2) {
    -            logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
    -          }
    -          checkGroupAndAggrValues(currentIndex);
    -          // If adding a group discovered a memory pressure during 1st phase, then start
    -          // outputing some partition to free memory.
    -          if ( earlyOutput ) {
    -            outputCurrentBatch();
    -            incIndex(); // next time continue with the next incoming row
    -            return AggOutcome.RETURN_OUTCOME;
    -          }
    +        checkGroupAndAggrValues(currentIndex);
    +        // If adding a group discovered a memory pressure during 1st phase, then start
    +        // outputing some partition downstream in order to free memory.
    +        if ( earlyOutput ) {
    +          outputCurrentBatch();
    +          incIndex(); // next time continue with the next incoming row
    +          return AggOutcome.RETURN_OUTCOME;
             }
    +      }
    +
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Processed {} records", underlyingIndex);
    +      }
     
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Processed {} records", underlyingIndex);
    +      // Cleanup the previous batch since we are done processing it.
    +      for (VectorWrapper<?> v : incoming) {
    --- End diff --
    
    Mmmm... this is the original Hash Agg code. Hence probably the two invariants are met.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118545693
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -179,6 +179,26 @@ drill.exec: {
         // Use plain Java compilation where available
         prefer_plain_java: false
       },
    +  spill: {
    --- End diff --
    
    Added "spill" and "hashagg" sections in the override example file, with some comments:
    
      spill: {
         # These options are common to all spilling operators.
         # They can be overriden, per operator (but this is just for
         # backward compatibility, and may be deprecated in the future)
         directories : [ "/tmp/drill/spill" ],
         fs : "file:///"
      }
      hashagg: {
        # The partitions divide the work inside the hashagg, to ease
        # handling spilling. This initial figure is tuned down when
        # memory is limited.
        #  Setting this option to 1 disables spilling !
        num_partitions: 32,
        spill: {
            # The 2 options below override the common ones
            # they should be deprecated in the future
            directories : [ "/tmp/drill/spill" ],
            fs : "file:///"
        }
      },



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813748
  
    --- 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 --
    
    Shouldn't this throw an `IllegalStateException`? Something is terribly wrong if we get this where it is not expected. Simply logging zillions of messages to the log is not likely to call attention to the problem.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117108013
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -179,6 +179,26 @@ drill.exec: {
         // Use plain Java compilation where available
         prefer_plain_java: false
       },
    +  spill: {
    --- End diff --
    
    Please modify distribution/resources/drill-override-example.conf to explain how to set up spilling. Add a comment (see below) about deprecating the sort-specific config, but that it will still work for a few more releases.
    
    This file (drill-module.conf) is visible only to developers. drill-override.conf is the file users modify. drill-override-example.conf is the example "starter" file that new users start with.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118814075
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
         }
     
    -    // get the number of records in the batch holder that are pending output
    -    int numPendingOutput = batchHolders.get(outBatchIndex).getNumPendingOutput();
    +    // Initialization (covers the case of early output)
    +    ArrayList<BatchHolder> currPartition = batchHolders[earlyPartition];
    +    int currOutBatchIndex = outBatchIndex[earlyPartition];
    +    int partitionToReturn = earlyPartition;
    +
    +    if ( ! earlyOutput ) {
    +      // Update the next partition to return (if needed)
    +      // skip fully returned (or spilled) partitions
    +      while (nextPartitionToReturn < numPartitions) {
    +        //
    +        // If this partition was spilled - spill the rest of it and skip it
    +        //
    +        if ( isSpilled(nextPartitionToReturn) ) {
    +          spillAPartition(nextPartitionToReturn); // spill the rest
    +          SpilledPartition sp = new SpilledPartition();
    +          sp.spillFile = spillFiles[nextPartitionToReturn];
    +          sp.spilledBatches = spilledBatchesCount[nextPartitionToReturn];
    +          sp.cycleNum = cycleNum; // remember the current cycle
    +          sp.origPartn = nextPartitionToReturn; // for debugging / filename
    +          sp.prevOrigPartn = originalPartition; // for debugging / filename
    +          spilledPartitionsList.add(sp);
    +          try {
    +            reinitPartition(nextPartitionToReturn); // free the memory
    +          } catch (Exception e) {throw new RuntimeException(e);}
    +          try {
    +            long posn = spillSet.getPosition(outputStream[nextPartitionToReturn]);
    +            spillSet.tallyWriteBytes(posn); // for the IO stats
    +            outputStream[nextPartitionToReturn].close();
    +          } catch (IOException e) { throw new RuntimeException(e); }
    +          outputStream[nextPartitionToReturn] = null;
    +        }
    +        else {
    +          currPartition = batchHolders[nextPartitionToReturn];
    +          currOutBatchIndex = outBatchIndex[nextPartitionToReturn];
    +          // If curr batch (partition X index) is not empty - proceed to return it
    +          if (currOutBatchIndex < currPartition.size() && 0 != currPartition.get(currOutBatchIndex).getNumPendingOutput()) {
    +            break;
    +          }
    +        }
    +        nextPartitionToReturn++; // else check next partition
    +      }
    +
    +      // if passed the last partition
    +      if (nextPartitionToReturn >= numPartitions) {
    +        // The following "if" is probably never used; due to a similar check at the end of this method
    +        if ( spilledPartitionsList.isEmpty() ) { // and no spilled partitions
    +          allFlushed = true;
    +          this.outcome = IterOutcome.NONE;
    +          if ( is2ndPhase ) {
    +            stats.setLongStat(Metric.SPILL_MB, // update stats - total MB spilled
    +                (int) Math.round(spillSet.getWriteBytes() / 1024.0D / 1024.0));
    +          }
    +          return outcome;  // then return NONE
    +        }
    +        // Else - there are still spilled partitions to process - pick one and handle just like a new incoming
    +        buildComplete = false; // go back and call doWork() again
    +        handlingSpills = true; // beginning to work on the spill files
    +        // pick a spilled partition; set a new incoming ...
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        SpilledRecordbatch newIncoming = newSpilledRecordBatch(sp.spillFile, sp.spilledBatches, context, schema, oContext);
    +        originalPartition = sp.origPartn; // used for the filename
    +        logger.trace("Reading back spilled original partition {} as an incoming",originalPartition);
    +        // prevOriginalPartition = sp.prevOrigPartn;
    +        // Initialize .... new incoming, new set of partitions
    +        try { initializeSetup(newIncoming); } catch (Exception e) { throw new RuntimeException(e); }
    +        // update the cycle num if needed
    +        // The current cycle num should always be one larger than in the spilled partition
    +        if ( cycleNum == sp.cycleNum ) {
    +          cycleNum = 1 + sp.cycleNum;
    +          stats.setLongStat(Metric.SPILL_CYCLE, cycleNum); // update stats
    +          // report memory stressful situations
    +          if ( cycleNum == 2 ) { logger.info("SECONDARY SPILLING "); }
    +          if ( cycleNum == 3 ) { logger.info("TERTIARY SPILLING "); }
    +        }
    +        if ( EXTRA_DEBUG_SPILL ) {
    +          logger.debug("Start reading spilled partition {} (prev {}) from cycle {} (with {} batches). More {} spilled partitions left.",
    +              sp.origPartn, sp.prevOrigPartn, sp.cycleNum, sp.spilledBatches, spilledPartitionsList.size());
    +        }
    +        return IterOutcome.RESTART;
    +      }
    +
    +      partitionToReturn = nextPartitionToReturn ;
     
    -    if (numPendingOutput == 0) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
         }
     
    +    // get the number of records in the batch holder that are pending output
    +    int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +    // The following accounting is for logging, metrics, etc.
    +    rowsInPartition += numPendingOutput ;
    +    if ( ! handlingSpills ) { rowsNotSpilled += numPendingOutput; }
    +    else { rowsSpilledReturned += numPendingOutput; }
    +    if ( earlyOutput ) { rowsReturnedEarly += numPendingOutput; }
    +
         allocateOutgoing(numPendingOutput);
     
    -    batchHolders.get(outBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +    currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
         int numOutputRecords = outNumRecordsHolder.value;
     
         if (EXTRA_DEBUG_1) {
           logger.debug("After output values: outStartIdx = {}, outNumRecords = {}", outStartIdxHolder.value, outNumRecordsHolder.value);
         }
    -    this.htable.outputKeys(outBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +    this.htables[partitionToReturn].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
     
         // set the value count for outgoing batch value vectors
         for (VectorWrapper<?> v : outgoing) {
           v.getValueVector().getMutator().setValueCount(numOutputRecords);
         }
     
    -//    outputCount += numOutputRecords;
    -
         this.outcome = IterOutcome.OK;
     
    -    logger.debug("HashAggregate: Output current batch index {} with {} records.", outBatchIndex, numOutputRecords);
    +    // logger.debug("HashAggregate: Output {} current batch index {} with {} records for partition {}.", earlyOutput ? "(early)" : "",
    +    //    outBatchIndex, numOutputRecords, partitionToReturn);
    +    if ( EXTRA_DEBUG_SPILL && is2ndPhase ) {
    +      logger.debug("So far returned {} + SpilledReturned {}  total {} (spilled {})",rowsNotSpilled,rowsSpilledReturned,
    +        rowsNotSpilled+rowsSpilledReturned,
    +        rowsSpilled);
    +    }
     
         lastBatchOutputCount = numOutputRecords;
    -    outBatchIndex++;
    -    if (outBatchIndex == batchHolders.size()) {
    -      allFlushed = true;
    +    outBatchIndex[partitionToReturn]++;
    +    // if just flushed the last batch in the partition
    +    if (outBatchIndex[partitionToReturn] == currPartition.size()) {
    +
    +      if ( EXTRA_DEBUG_SPILL ) {
    +        logger.debug("HashAggregate: {} Flushed partition {} with {} batches total {} rows",
    +            earlyOutput ? "(Early)" : "",
    +            partitionToReturn, outBatchIndex[partitionToReturn], rowsInPartition);
    +      }
    +      rowsInPartition = 0; // reset to count for the next partition
    +
    +      try {
    +        // deallocate memory used by this partition, and re-initialize
    +        reinitPartition(partitionToReturn);
    +      } catch (SchemaChangeException sce) {
    +        throw new DrillRuntimeException("Hash Aggregation can not handle schema changes.");
    +      } catch (Exception e) { /* just ignore */ }
    +
    +      if ( earlyOutput ) {
    --- End diff --
    
    General comment. This code is very long and very complex. It has a high "cyclomatic complexity." It is unlikely that we can generate system-level tests that hit all the if conditions. And, if some new developer had to modify this code, how would they know if they broke anything?
    
    This stuff probably needs to be divided up into smaller chunks that can be unit tested (and understood.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812014
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    +    private VectorContainer container = null;
    --- End diff --
    
    Can this be final? A spilled record batch only ever has one container: the one that it will spill?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by rchallapalli <gi...@git.apache.org>.
Github user rchallapalli commented on the issue:

    https://github.com/apache/drill/pull/822
  
    And based on my initial testing, I observed that the hash-agg is only using half of the memory allocated. Since the planner by default uses a 2-phase agg, the memory computation logic divides the allocated memory between the 2 hash-agg operators in the plan. This is grossly in-efficient. And every test written would need to be modified once this issue gets resolved. Hence I would push for a fix sooner than later


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119975022
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    --- End diff --
    
    The field "schema" is also used as a flag to note (when null) "Have not yet received any non-empty incoming batch". Null is its initial value, which is set to a non-null after the first non-empty incoming batch arrives. 
      In this code (there was an actual test case that failed here), no real data was ever received, so when time came to output results, a NONE output was produced. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812681
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    +    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
    +    logger.trace("Incoming sizer: {}",sizer);
    +    // An empty batch only has the schema, can not tell actual length of varchars
    +    // else use the actual varchars length, each capped at 50 (to match the space allocation)
    +    estRowWidth = sizer.rowCount() == 0 ? sizer.stdRowWidth() : sizer.netRowWidthCap50();
    +    estMaxBatchSize = estRowWidth * MAX_BATCH_SIZE;
    +
    +    // Get approx max (varchar) column width to get better memory allocation
    +    maxColumnWidth = Math.max(sizer.maxSize(), VARIABLE_MIN_WIDTH_VALUE_SIZE);
    +    maxColumnWidth = Math.min(maxColumnWidth, VARIABLE_MAX_WIDTH_VALUE_SIZE);
    +
    +    logger.trace("{} phase. Estimated row width: {}  batch size: {}  memory limit: {}  max column width: {}",
    +        isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",estRowWidth,estMaxBatchSize,memoryLimit,maxColumnWidth);
    --- End diff --
    
    Here, if there was a mode enum (see above), this could just be mode, estRowWidth, ... and let the string value of the enum be printed.
    
    BTW, no harm in putting a space after the comma...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117059034
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -546,44 +1254,204 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
          holder.value = vv0.getAccessor().get(incomingRowIdx) ;
          }
          */
    +    /*
    +    if ( handlingSpills && ( incomingRowIdx == 0 ) ) {
    +      // for debugging -- show the first row from a spilled batch
    +      Object tmp0 = (incoming).getValueAccessorById(NullableVarCharVector.class, 0).getValueVector();
    +      Object tmp1 = (incoming).getValueAccessorById(NullableVarCharVector.class, 1).getValueVector();
    +      Object tmp2 = (incoming).getValueAccessorById(NullableBigIntVector.class, 2).getValueVector();
    +
    +      if (tmp0 != null && tmp1 != null && tmp2 != null) {
    +        NullableVarCharVector vv0 = ((NullableVarCharVector) tmp0);
    +        NullableVarCharVector vv1 = ((NullableVarCharVector) tmp1);
    +        NullableBigIntVector  vv2 = ((NullableBigIntVector) tmp2);
    +        logger.debug("The first row = {} , {} , {}", vv0.getAccessor().get(incomingRowIdx), vv1.getAccessor().get(incomingRowIdx), vv2.getAccessor().get(incomingRowIdx));
    +      }
    +    }
    +    */
    +    // The hash code is computed once, then its lower bits are used to determine the
    +    // partition to use, and the higher bits determine the location in the hash table.
    +    int hashCode;
    +    try {
    +      htables[0].updateBatches();
    +      hashCode = htables[0].getHashCode(incomingRowIdx);
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
     
    -    htable.put(incomingRowIdx, htIdxHolder, 1 /* retry count */);
    +    // right shift hash code for secondary (or tertiary...) spilling
    +    for (int i = 0; i < cycleNum; i++) { hashCode >>>= bitsInMask; }
     
    +    int currentPartition = hashCode & partitionMask ;
    +    hashCode >>>= bitsInMask;
    +    HashTable.PutStatus putStatus = null;
    +    long allocatedBefore = allocator.getAllocatedMemory();
    +
    +    // Insert the key columns into the hash table
    +    try {
    +      putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode);
    +    } catch (OutOfMemoryException exc) {
    +      throw new OutOfMemoryException(getOOMErrorMsg(), exc); // may happen when can not spill
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
         int currentIdx = htIdxHolder.value;
     
    -    // get the batch index and index within the batch
    -    if (currentIdx >= batchHolders.size() * HashTable.BATCH_SIZE) {
    -      addBatchHolder();
    +    long addedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +    if ( addedMem > 0 ) {
    +      logger.trace("MEMORY CHECK HT: allocated {}  added {} partition {}",allocatedBefore,addedMem,currentPartition);
         }
    -    BatchHolder bh = batchHolders.get((currentIdx >>> 16) & HashTable.BATCH_MASK);
    +
    +    // Check if put() added a new batch (for the keys) inside the hash table, hence a matching batch
    +    // (for the aggregate columns) needs to be created
    +    if ( putStatus == HashTable.PutStatus.NEW_BATCH_ADDED ) {
    +      try {
    +        long allocatedBeforeAggCol = allocator.getAllocatedMemory();
    +
    +        addBatchHolder(currentPartition);
    +
    +        if ( plannedBatches > 0 ) { plannedBatches--; } // just allocated a planned batch
    +        long totalAddedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +        logger.trace("MEMORY CHECK AGG: added {}  total (with HT) added {}",allocator.getAllocatedMemory()-allocatedBeforeAggCol,totalAddedMem);
    +        // resize the batch estimate if needed (e.g., varchars may take more memory than estimated)
    +        if ( totalAddedMem > estMaxBatchSize ) {
    +          logger.trace("Adjusting Batch size estimate from {} to {}",estMaxBatchSize,totalAddedMem);
    +          estMaxBatchSize = totalAddedMem;
    +        }
    +      } catch (OutOfMemoryException exc) {
    --- End diff --
    
    Which call in this try{} block is throwing an OOM ? The addBatchHolder() does not and the other function calls are essentially computing the allocated memory.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812514
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    +    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
    +    logger.trace("Incoming sizer: {}",sizer);
    +    // An empty batch only has the schema, can not tell actual length of varchars
    +    // else use the actual varchars length, each capped at 50 (to match the space allocation)
    +    estRowWidth = sizer.rowCount() == 0 ? sizer.stdRowWidth() : sizer.netRowWidthCap50();
    --- End diff --
    
    This says that if the first batch has zero rows (the typical case), use the fake row width numbers. All the spilling calcs depend on those numbers. Is the net result GIGO?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119975302
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
    --- End diff --
    
    This code was changed anyway, as part of eliminating RESTART.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117106477
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java ---
    @@ -188,6 +188,18 @@
         NOT_YET,
     
         /**
    +     * Restart reading incoming batches.
    +     * <p>
    +     *     Prior calls to {@link #next()} returned data after all incoming batches were processed.
    +     *     But with this status, need to restart reading incoming RecordBatches.
    +     * </p>
    +     * <p>
    +     *     Currently only used internally by the Hash Aggregate.
    +     * </p>
    +     */
    --- End diff --
    
    Not entirely sure we want to extend the core Volcano iterator protocol to help with a single operator. In general, ALL operators should be prepared to handle this value since it is part of the public protocol.
    
    Can we find a way to avoid this? I suspect some code restructuring, along with an operator-specific enum, would do the trick.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117107129
  
    --- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
    @@ -179,6 +179,26 @@ drill.exec: {
         // Use plain Java compilation where available
         prefer_plain_java: false
       },
    +  spill: {
    --- End diff --
    
    Nicely done! Good use of inheritance and references.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117074811
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -546,44 +1254,204 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
          holder.value = vv0.getAccessor().get(incomingRowIdx) ;
          }
          */
    +    /*
    +    if ( handlingSpills && ( incomingRowIdx == 0 ) ) {
    +      // for debugging -- show the first row from a spilled batch
    +      Object tmp0 = (incoming).getValueAccessorById(NullableVarCharVector.class, 0).getValueVector();
    +      Object tmp1 = (incoming).getValueAccessorById(NullableVarCharVector.class, 1).getValueVector();
    +      Object tmp2 = (incoming).getValueAccessorById(NullableBigIntVector.class, 2).getValueVector();
    +
    +      if (tmp0 != null && tmp1 != null && tmp2 != null) {
    +        NullableVarCharVector vv0 = ((NullableVarCharVector) tmp0);
    +        NullableVarCharVector vv1 = ((NullableVarCharVector) tmp1);
    +        NullableBigIntVector  vv2 = ((NullableBigIntVector) tmp2);
    +        logger.debug("The first row = {} , {} , {}", vv0.getAccessor().get(incomingRowIdx), vv1.getAccessor().get(incomingRowIdx), vv2.getAccessor().get(incomingRowIdx));
    +      }
    +    }
    +    */
    +    // The hash code is computed once, then its lower bits are used to determine the
    +    // partition to use, and the higher bits determine the location in the hash table.
    +    int hashCode;
    +    try {
    +      htables[0].updateBatches();
    +      hashCode = htables[0].getHashCode(incomingRowIdx);
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
     
    -    htable.put(incomingRowIdx, htIdxHolder, 1 /* retry count */);
    +    // right shift hash code for secondary (or tertiary...) spilling
    +    for (int i = 0; i < cycleNum; i++) { hashCode >>>= bitsInMask; }
     
    +    int currentPartition = hashCode & partitionMask ;
    +    hashCode >>>= bitsInMask;
    +    HashTable.PutStatus putStatus = null;
    +    long allocatedBefore = allocator.getAllocatedMemory();
    +
    +    // Insert the key columns into the hash table
    +    try {
    +      putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode);
    +    } catch (OutOfMemoryException exc) {
    +      throw new OutOfMemoryException(getOOMErrorMsg(), exc); // may happen when can not spill
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
         int currentIdx = htIdxHolder.value;
     
    -    // get the batch index and index within the batch
    -    if (currentIdx >= batchHolders.size() * HashTable.BATCH_SIZE) {
    -      addBatchHolder();
    +    long addedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +    if ( addedMem > 0 ) {
    +      logger.trace("MEMORY CHECK HT: allocated {}  added {} partition {}",allocatedBefore,addedMem,currentPartition);
         }
    -    BatchHolder bh = batchHolders.get((currentIdx >>> 16) & HashTable.BATCH_MASK);
    +
    +    // Check if put() added a new batch (for the keys) inside the hash table, hence a matching batch
    +    // (for the aggregate columns) needs to be created
    +    if ( putStatus == HashTable.PutStatus.NEW_BATCH_ADDED ) {
    +      try {
    +        long allocatedBeforeAggCol = allocator.getAllocatedMemory();
    +
    +        addBatchHolder(currentPartition);
    +
    +        if ( plannedBatches > 0 ) { plannedBatches--; } // just allocated a planned batch
    +        long totalAddedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +        logger.trace("MEMORY CHECK AGG: added {}  total (with HT) added {}",allocator.getAllocatedMemory()-allocatedBeforeAggCol,totalAddedMem);
    +        // resize the batch estimate if needed (e.g., varchars may take more memory than estimated)
    +        if ( totalAddedMem > estMaxBatchSize ) {
    +          logger.trace("Adjusting Batch size estimate from {} to {}",estMaxBatchSize,totalAddedMem);
    +          estMaxBatchSize = totalAddedMem;
    +        }
    +      } catch (OutOfMemoryException exc) {
    +        throw new OutOfMemoryException(getOOMErrorMsg(), exc); // may happen when can not spill
    +      }
    +    }
    +    BatchHolder bh = batchHolders[currentPartition].get((currentIdx >>> 16) & HashTable.BATCH_MASK);
         int idxWithinBatch = currentIdx & HashTable.BATCH_MASK;
     
    +    if (bh.updateAggrValues(incomingRowIdx, idxWithinBatch)) {
    +      numGroupedRecords++;
    +    }
    +
    +    // ===================================================================================
    +    // If the last batch just became full - that is the time to check the memory limits !!
    +    // If exceeded, then need to spill (if 2nd phase) or output early (1st)
    +    // (Skip this if cannot spill; in such case an OOM may be encountered later)
    +    // ===================================================================================
    +    if ( putStatus == HashTable.PutStatus.KEY_ADDED_LAST && canSpill ) {
    +
    +      plannedBatches++; // planning to allocate one more batch
    +
    +      // calculate the (max) new memory needed now
    +      long hashTableDoublingSizeNeeded = 0; // in case the hash table(s) would resize
    +      for ( HashTable ht : htables ) {
    --- End diff --
    
    It is not immediately obvious why all partitions need to be asked for extra memory needed for resizing.  If a new row  causes doubling of a hash table, only 1 partition would be affected.  But if you are considering all rows from an incoming batch then yes it could in the worst case affect all partitions. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117049411
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java ---
    @@ -497,6 +510,21 @@ public int numResizing() {
         return numResizing;
       }
     
    +  /**
    +   *
    +   * @return Size of extra memory needed if the HT (i.e. startIndices) is doubled
    +   */
    +  @Override
    +  public int extraMemoryNeededForResize() {
    --- End diff --
    
    Should this return a long ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812498
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    --- End diff --
    
    Not sure this is right... According to the Drill iterator protocol, the first batch should be empty and provide only schema. As operators better honor that rule, your sample size in the first batch will shrink -- eventually to zero. Does this need logic to skip the initial batch if the row count is zero? To at least "sniff" subsequent batches and report if their row width is significantly larger than the original estimate? Such a log message will go a long way to tracking down OOM errors in production systems when this behavior occurs in practice.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118103592
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -86,6 +92,19 @@
     
       BooleanValidator EXTERNAL_SORT_DISABLE_MANAGED_OPTION = new BooleanValidator("exec.sort.disable_managed", false);
     
    +  // Hash Aggregate Options
    +
    +  String HASHAGG_NUM_PARTITIONS_KEY = "drill.exec.hashagg.num_partitions";
    --- End diff --
    
    Changed all three to have both the config option ("drill.exec.hashagg....") and its matching _KEY (same, sans the prefix "drill.") which is only used to create the validator.
    Hence all three can be set either in the config file, or for the session: 
    Number of partitions, max memory, and minimum batches per partitions.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118152042
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java ---
    @@ -99,17 +100,31 @@ public void setCost(double cost) {
         this.cost = cost;
       }
     
    -  // Not available. Presumably because Drill does not currently use
    -  // this value, though it does appear in some test physical plans.
    -//  public void setMaxAllocation(long alloc) {
    -//    maxAllocation = alloc;
    -//  }
    -
       @Override
       public long getMaxAllocation() {
         return maxAllocation;
       }
     
    +  /**
    +   * Any operator that supports spilling should override this method
    +   * @param maxAllocation The max memory allocation to be set
    +   */
    +  @Override
    +  public void setMaxAllocation(long maxAllocation) {
    +    this.maxAllocation = maxAllocation;
    +    /*throw new DrillRuntimeException("Unsupported method: setMaxAllocation()");*/
    +  }
    +
    +  /**
    +   * Any operator that supports spilling should override this method (and return true)
    +   * @return false
    +   */
    +  @Override
    +  public boolean getBufferedOperator() { return false; }
    --- End diff --
    
    Changes made ...



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117059683
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -546,44 +1254,204 @@ private void checkGroupAndAggrValues(int incomingRowIdx) {
          holder.value = vv0.getAccessor().get(incomingRowIdx) ;
          }
          */
    +    /*
    +    if ( handlingSpills && ( incomingRowIdx == 0 ) ) {
    +      // for debugging -- show the first row from a spilled batch
    +      Object tmp0 = (incoming).getValueAccessorById(NullableVarCharVector.class, 0).getValueVector();
    +      Object tmp1 = (incoming).getValueAccessorById(NullableVarCharVector.class, 1).getValueVector();
    +      Object tmp2 = (incoming).getValueAccessorById(NullableBigIntVector.class, 2).getValueVector();
    +
    +      if (tmp0 != null && tmp1 != null && tmp2 != null) {
    +        NullableVarCharVector vv0 = ((NullableVarCharVector) tmp0);
    +        NullableVarCharVector vv1 = ((NullableVarCharVector) tmp1);
    +        NullableBigIntVector  vv2 = ((NullableBigIntVector) tmp2);
    +        logger.debug("The first row = {} , {} , {}", vv0.getAccessor().get(incomingRowIdx), vv1.getAccessor().get(incomingRowIdx), vv2.getAccessor().get(incomingRowIdx));
    +      }
    +    }
    +    */
    +    // The hash code is computed once, then its lower bits are used to determine the
    +    // partition to use, and the higher bits determine the location in the hash table.
    +    int hashCode;
    +    try {
    +      htables[0].updateBatches();
    +      hashCode = htables[0].getHashCode(incomingRowIdx);
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
     
    -    htable.put(incomingRowIdx, htIdxHolder, 1 /* retry count */);
    +    // right shift hash code for secondary (or tertiary...) spilling
    +    for (int i = 0; i < cycleNum; i++) { hashCode >>>= bitsInMask; }
     
    +    int currentPartition = hashCode & partitionMask ;
    +    hashCode >>>= bitsInMask;
    +    HashTable.PutStatus putStatus = null;
    +    long allocatedBefore = allocator.getAllocatedMemory();
    +
    +    // Insert the key columns into the hash table
    +    try {
    +      putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode);
    +    } catch (OutOfMemoryException exc) {
    +      throw new OutOfMemoryException(getOOMErrorMsg(), exc); // may happen when can not spill
    +    } catch (SchemaChangeException e) {
    +      throw new IllegalStateException("Unexpected schema change", e);
    +    }
         int currentIdx = htIdxHolder.value;
     
    -    // get the batch index and index within the batch
    -    if (currentIdx >= batchHolders.size() * HashTable.BATCH_SIZE) {
    -      addBatchHolder();
    +    long addedMem = allocator.getAllocatedMemory() - allocatedBefore;
    +    if ( addedMem > 0 ) {
    +      logger.trace("MEMORY CHECK HT: allocated {}  added {} partition {}",allocatedBefore,addedMem,currentPartition);
         }
    -    BatchHolder bh = batchHolders.get((currentIdx >>> 16) & HashTable.BATCH_MASK);
    +
    +    // Check if put() added a new batch (for the keys) inside the hash table, hence a matching batch
    +    // (for the aggregate columns) needs to be created
    +    if ( putStatus == HashTable.PutStatus.NEW_BATCH_ADDED ) {
    +      try {
    +        long allocatedBeforeAggCol = allocator.getAllocatedMemory();
    +
    +        addBatchHolder(currentPartition);
    +
    +        if ( plannedBatches > 0 ) { plannedBatches--; } // just allocated a planned batch
    +        long totalAddedMem = allocator.getAllocatedMemory() - allocatedBefore;
    --- End diff --
    
    I presume totalAddedMem is the memory added for the group-by keys.  Would it be better to rename it to something like totalAddedMemGroupBy and allocatedBefore to allocatedBeforeGroupBy such that it is explicit ?   Also, why not use allocatedBeforeAggCol  here  instead of calling allocator.getAllocatedMemory() again ? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118811796
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -166,20 +252,23 @@ public BatchHolder() {
         }
     
         private boolean updateAggrValues(int incomingRowIdx, int idxWithinBatch) {
    -      updateAggrValuesInternal(incomingRowIdx, idxWithinBatch);
    +      try { updateAggrValuesInternal(incomingRowIdx, idxWithinBatch); }
    +      catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc); }
    --- End diff --
    
    Probably `IllegalStateException` to clearly state that this is a "this should never occur" kind of error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813700
  
    --- 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();
    +
    --- End diff --
    
    This function seems overly long: very hard to follow. Perhaps break out each nested loop into a function. Since the functions are called once per batch, the cost of a function call is lost in the noise of operating the iterator hierarchy. The gain, however, is lower mental cost to understand what's happening.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119736678
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    +    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
    +    logger.trace("Incoming sizer: {}",sizer);
    +    // An empty batch only has the schema, can not tell actual length of varchars
    +    // else use the actual varchars length, each capped at 50 (to match the space allocation)
    +    estRowWidth = sizer.rowCount() == 0 ? sizer.stdRowWidth() : sizer.netRowWidthCap50();
    +    estMaxBatchSize = estRowWidth * MAX_BATCH_SIZE;
    +
    +    // Get approx max (varchar) column width to get better memory allocation
    +    maxColumnWidth = Math.max(sizer.maxSize(), VARIABLE_MIN_WIDTH_VALUE_SIZE);
    +    maxColumnWidth = Math.min(maxColumnWidth, VARIABLE_MAX_WIDTH_VALUE_SIZE);
    +
    +    logger.trace("{} phase. Estimated row width: {}  batch size: {}  memory limit: {}  max column width: {}",
    +        isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",estRowWidth,estMaxBatchSize,memoryLimit,maxColumnWidth);
    +
    +    if ( estMaxBatchSize > memoryLimit ) {
    +      logger.warn("HashAggregate: Estimated max batch size {} is larger than the memory limit {}",estMaxBatchSize,memoryLimit);
    --- End diff --
    
    True; this check alerts only of the more extreme cases, where the memory available initially can not hold even a single batch (easier to explain in a warning message).
      However note that following the execution of this code, there is another check (in delayedSetup() ) which does add the overhead (as a hard coded 8MB) and gives another warning alert (when down to 1 partition; this was just added in a prior review comment; see above).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117102243
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -64,6 +64,12 @@
       String SPOOLING_BUFFER_MEMORY = "drill.exec.buffer.spooling.size";
       String BATCH_PURGE_THRESHOLD = "drill.exec.sort.purge.threshold";
     
    +  // Spill Options common to all spilling operators
    --- End diff --
    
    Spill boot-time config options...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
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 ....


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813928
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    --- End diff --
    
    This code more-or-less duplicates the code in the `SpilledRun` class mentioned above. Maybe use a single class that we can unit test once rather than having multiple copies of the code?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812097
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    +    private VectorContainer container = null;
    +    private InputStream spillStream;
    +    private int spilledBatches;
    +    private FragmentContext context;
    +    private BatchSchema schema;
    +    private OperatorContext oContext;
    +    // Path spillStreamPath;
    +    private String spillFile;
    +    VectorAccessibleSerializable vas;
    +
    +    public SpilledRecordbatch(String spillFile,/* Path spillStreamPath,*/ int spilledBatches, FragmentContext context, BatchSchema schema, OperatorContext oContext) {
    +      this.context = context;
    +      this.schema = schema;
    +      this.spilledBatches = spilledBatches;
    +      this.oContext = oContext;
    +      //this.spillStreamPath = spillStreamPath;
    +      this.spillFile = spillFile;
    +      vas = new VectorAccessibleSerializable(allocator);
    +      container = vas.get();
    +
    +      try {
    +        this.spillStream = spillSet.openForInput(spillFile);
    +      } catch (IOException e) { throw new RuntimeException(e);}
    +
    +      next(); // initialize the container
    +    }
    +
    +    @Override
    +    public SelectionVector2 getSelectionVector2() {
    +      throw new UnsupportedOperationException();
    +    }
    +
    +    @Override
    +    public SelectionVector4 getSelectionVector4() {
    +      throw new UnsupportedOperationException();
    +    }
    +
    +    @Override
    +    public TypedFieldId getValueVectorId(SchemaPath path) {
    +      return container.getValueVectorId(path);
    +    }
    +
    +    @Override
    +    public VectorWrapper<?> getValueAccessorById(Class<?> clazz, int... ids) {
    +      return container.getValueAccessorById(clazz, ids);
    +    }
    +
    +    @Override
    +    public Iterator<VectorWrapper<?>> iterator() {
    +      return container.iterator();
    +    }
    +
    +    @Override
    +    public FragmentContext getContext() { return context; }
    +
    +    @Override
    +    public BatchSchema getSchema() { return schema; }
    +
    +    @Override
    +    public WritableBatch getWritableBatch() {
    +      return WritableBatch.get(this);
    +    }
    +
    +    @Override
    +    public VectorContainer getOutgoingContainer() { return container; }
    +
    +    @Override
    +    public int getRecordCount() { return container.getRecordCount(); }
    +
    +    @Override
    +    public void kill(boolean sendUpstream) {
    +      this.close(); // delete the current spill file
    +    }
    +
    +    /**
    +     * Read the next batch from the spill file
    +     *
    +     * @return IterOutcome
    +     */
    +    @Override
    +    public IterOutcome next() {
    --- End diff --
    
    Ah! It is clear what is happening now. This RecordBatch is a kind of operator: one that reads from disk into the attached container. Quite clever, actually. But probably overly complex and hard to maintain.
    
    Maybe what you want is to reuse the `SpilledRun` class from the external sort. Maybe we can generalize that class a bit. It already has the logic to associate a spilled run with a file, iterate over rows and so on. (Though, the iterator behavior should be cleaned up...) `SpilledRun` does that without the full baggage of the `RecordBatch` protocol.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118811955
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -204,24 +293,157 @@ private int getNumPendingOutput() {
     
         @RuntimeOverridden
         public void setupInterior(@Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing,
    -        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) {
    +        @Named("aggrValuesContainer") VectorContainer aggrValuesContainer) throws SchemaChangeException {
         }
     
         @RuntimeOverridden
    -    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {
    +    public void updateAggrValuesInternal(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) throws SchemaChangeException{
         }
     
         @RuntimeOverridden
    -    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {
    +    public void outputRecordValues(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) throws SchemaChangeException{
         }
       }
     
    +  /**
    +   * An internal class to replace "incoming" - instead scanning a spilled partition file
    +   */
    +  public class SpilledRecordbatch implements CloseableRecordBatch {
    --- End diff --
    
    This is an inner class: it has an implicit pointer back to the template. As a result, this inner class is copied for each code generation; each hash operation gets not only its own instance, but also its own copy of this class. Is this intended or needed?
    
    If not, then start by making this class static. Fix up any implicit references to the outer class. Then, move the class into a separate file so that the byte code manipulation method of CG does not make a copy of this class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117052757
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java ---
    @@ -544,86 +572,69 @@ private static int roundUpToPowerOf2(int number) {
         return rounded;
       }
     
    -  @Override
    -  public void put(int incomingRowIdx, IndexPointer htIdxHolder, int retryCount) {
    -    put(incomingRowIdx, htIdxHolder);
    +  public int getHashCode(int incomingRowIdx) throws SchemaChangeException {
    +    return getHashBuild(incomingRowIdx);
       }
     
    -  private PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder) {
    +  /** put() uses the hash code (from gethashCode() above) to insert the key(s) from the incoming
    +   * row into the hash table. The code selects the bucket in the startIndices, then the keys are
    +   * placed into the chained list - by storing the key values into a batch, and updating its
    +   * "links" member. Last it modifies the index holder to the batch offset so that the caller
    +   * can store the remaining parts of the row into a matching batch (outside the hash table).
    +   * Returning
    +   *
    +   * @param incomingRowIdx - position of the incoming row
    +   * @param htIdxHolder - to return batch + batch-offset (for caller to manage a matching batch)
    +   * @param hashCode - computed over the key(s) by calling getHashCode()
    +   * @return Status - the key(s) was ADDED or was already PRESENT
    +   */
    +  @Override
    +  public PutStatus put(int incomingRowIdx, IndexPointer htIdxHolder, int hashCode) throws SchemaChangeException {
     
    -    int hash = getHashBuild(incomingRowIdx);
    -    int i = getBucketIndex(hash, numBuckets());
    -    int startIdx = startIndices.getAccessor().get(i);
    +    int bucketIndex = getBucketIndex(hashCode, numBuckets());
    +    int startIdx = startIndices.getAccessor().get(bucketIndex);
         int currentIdx;
    -    int currentIdxWithinBatch;
    -    BatchHolder bh;
         BatchHolder lastEntryBatch = null;
         int lastEntryIdxWithinBatch = EMPTY_SLOT;
     
    +    // if startIdx is non-empty, follow the hash chain links until we find a matching
    +    // key or reach the end of the chain (and remember the last link there)
    +    for ( currentIdxHolder.value = startIdx;
    +          currentIdxHolder.value != EMPTY_SLOT;
    +          /* isKeyMatch() below also advances the currentIdxHolder to the next link */) {
     
    -    if (startIdx == EMPTY_SLOT) {
    -      // this is the first entry in this bucket; find the first available slot in the
    -      // container of keys and values
    -      currentIdx = freeIndex++;
    -      addBatchIfNeeded(currentIdx);
    +      // remember the current link, which would be the last when the next link is empty
    +      lastEntryBatch = batchHolders.get((currentIdxHolder.value >>> 16) & HashTable.BATCH_MASK);
    +      lastEntryIdxWithinBatch = currentIdxHolder.value & BATCH_MASK;
     
    -      if (EXTRA_DEBUG) {
    -        logger.debug("Empty bucket index = {}. incomingRowIdx = {}; inserting new entry at currentIdx = {}.", i,
    -            incomingRowIdx, currentIdx);
    +      if (lastEntryBatch.isKeyMatch(incomingRowIdx, currentIdxHolder, false)) {
    +        htIdxHolder.value = currentIdxHolder.value;
    +        return PutStatus.KEY_PRESENT;
           }
    -
    -      insertEntry(incomingRowIdx, currentIdx, hash, lastEntryBatch, lastEntryIdxWithinBatch);
    -      // update the start index array
    -      startIndices.getMutator().setSafe(getBucketIndex(hash, numBuckets()), currentIdx);
    -      htIdxHolder.value = currentIdx;
    -      return PutStatus.KEY_ADDED;
         }
     
    -    currentIdx = startIdx;
    -    boolean found = false;
    -
    -    bh = batchHolders.get((currentIdx >>> 16) & BATCH_MASK);
    -    currentIdxHolder.value = currentIdx;
    -
    -    // if startIdx is non-empty, follow the hash chain links until we find a matching
    -    // key or reach the end of the chain
    -    while (true) {
    -      currentIdxWithinBatch = currentIdxHolder.value & BATCH_MASK;
    +    // no match was found, so insert a new entry
    +    currentIdx = freeIndex++;
    +    boolean addedBatch = addBatchIfNeeded(currentIdx);
     
    -      if (bh.isKeyMatch(incomingRowIdx, currentIdxHolder, false)) {
    -        htIdxHolder.value = currentIdxHolder.value;
    -        found = true;
    -        break;
    -      } else if (currentIdxHolder.value == EMPTY_SLOT) {
    -        lastEntryBatch = bh;
    -        lastEntryIdxWithinBatch = currentIdxWithinBatch;
    -        break;
    -      } else {
    -        bh = batchHolders.get((currentIdxHolder.value >>> 16) & HashTable.BATCH_MASK);
    -        lastEntryBatch = bh;
    -      }
    +    if (EXTRA_DEBUG) {
    +      logger.debug("No match was found for incomingRowIdx = {}; inserting new entry at currentIdx = {}.", incomingRowIdx, currentIdx);
         }
     
    -    if (!found) {
    -      // no match was found, so insert a new entry
    -      currentIdx = freeIndex++;
    -      addBatchIfNeeded(currentIdx);
    +    insertEntry(incomingRowIdx, currentIdx, hashCode, lastEntryBatch, lastEntryIdxWithinBatch);
     
    -      if (EXTRA_DEBUG) {
    -        logger.debug("No match was found for incomingRowIdx = {}; inserting new entry at currentIdx = {}.", incomingRowIdx, currentIdx);
    -      }
    -
    -      insertEntry(incomingRowIdx, currentIdx, hash, lastEntryBatch, lastEntryIdxWithinBatch);
    -      htIdxHolder.value = currentIdx;
    -      return PutStatus.KEY_ADDED;
    +    // if there was no hash chain at this bucket, need to update the start index array
    +    if (startIdx == EMPTY_SLOT) {
    +      startIndices.getMutator().setSafe(getBucketIndex(hashCode, numBuckets()), currentIdx);
         }
    -
    -    return found ? PutStatus.KEY_PRESENT : PutStatus.KEY_ADDED;
    +    htIdxHolder.value = currentIdx;
    +    return  addedBatch ? PutStatus.NEW_BATCH_ADDED :
    +        ( freeIndex + 1 > batchHolders.size() * BATCH_SIZE ) ?
    --- End diff --
    
    Does KEY_ADDED_LAST indicate the last key in a single batch or last key across all batches added so far ? It seems it is the latter but would it be sufficient to check whether the last batch is filled up because the previous batches would have already been filled up ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119519134
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    +    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
    +    logger.trace("Incoming sizer: {}",sizer);
    +    // An empty batch only has the schema, can not tell actual length of varchars
    +    // else use the actual varchars length, each capped at 50 (to match the space allocation)
    +    estRowWidth = sizer.rowCount() == 0 ? sizer.stdRowWidth() : sizer.netRowWidthCap50();
    +    estMaxBatchSize = estRowWidth * MAX_BATCH_SIZE;
    +
    +    // Get approx max (varchar) column width to get better memory allocation
    +    maxColumnWidth = Math.max(sizer.maxSize(), VARIABLE_MIN_WIDTH_VALUE_SIZE);
    +    maxColumnWidth = Math.min(maxColumnWidth, VARIABLE_MAX_WIDTH_VALUE_SIZE);
    --- End diff --
    
    8 is the minimum, and yes, the max is 50. This figure is only used for allocating "max * 64K" as the size for the value vector. If the actual (average) size is greater than 50, then the value vector (actually the DrillBuf ?) would be resized as needed.  
      This is the same as in the prior code. The only difference is that for sizes < 50, we don't waste space allocating the 50 (times 64K). This can be a big memory saving when there are many key columns, all of which are small varchars. 
    
     


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118813780
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    --- End diff --
    
    Again, would be good to do this in a partition state class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118148905
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java ---
    @@ -99,17 +100,31 @@ public void setCost(double cost) {
         this.cost = cost;
       }
     
    -  // Not available. Presumably because Drill does not currently use
    -  // this value, though it does appear in some test physical plans.
    -//  public void setMaxAllocation(long alloc) {
    -//    maxAllocation = alloc;
    -//  }
    -
       @Override
       public long getMaxAllocation() {
         return maxAllocation;
       }
     
    +  /**
    +   * Any operator that supports spilling should override this method
    +   * @param maxAllocation The max memory allocation to be set
    +   */
    +  @Override
    +  public void setMaxAllocation(long maxAllocation) {
    --- End diff --
    
    That's what happens when you give a software "intelligence" to make decisions for you ..... shall we pick another method name to fool Jackson ? 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122863264
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java ---
    @@ -149,14 +149,24 @@ public IterOutcome innerNext() {
           if ( aggOut == HashAggregator.AggIterOutcome.AGG_OK ) { return IterOutcome.OK; }
           // if RESTART - continue below with doWork() - read some spilled partition, just like reading incoming
           incoming = aggregator.getNewIncoming(); // Restart - incoming was just changed
    -      if ( wasKilled ) { // if kill() was called before, then finish up
    -        aggregator.cleanup();
    -        incoming.kill(false);
    -        return IterOutcome.NONE;
    -      }
         }
     
    -    AggOutcome out = aggregator.doWork();
    +    if ( wasKilled ) { // if kill() was called before, then finish up
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r117106129
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java ---
    @@ -34,7 +34,7 @@
      * <p>
      *   A key thing to know is that the Iterator provided by a record batch must
      *   align with the rank positions of the field IDs provided using
    - *   {@link getValueVectorId}.
    + *   {@link #getValueVectorId}.
    --- End diff --
    
    Good catch!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122863244
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java ---
    @@ -149,14 +149,24 @@ public IterOutcome innerNext() {
           if ( aggOut == HashAggregator.AggIterOutcome.AGG_OK ) { return IterOutcome.OK; }
           // if RESTART - continue below with doWork() - read some spilled partition, just like reading incoming
           incoming = aggregator.getNewIncoming(); // Restart - incoming was just changed
    -      if ( wasKilled ) { // if kill() was called before, then finish up
    -        aggregator.cleanup();
    -        incoming.kill(false);
    -        return IterOutcome.NONE;
    -      }
         }
     
    -    AggOutcome out = aggregator.doWork();
    +    if ( wasKilled ) { // if kill() was called before, then finish up
    +      aggregator.cleanup();
    +      incoming.kill(false);
    +      return IterOutcome.NONE;
    +    }
    +
    +    // Read and aggregate records
    +    // ( may need to run again if the spilled partition that was read
    +    //   generated new partitions that were all spilled )
    +    AggOutcome out = AggOutcome.CALL_WORK_AGAIN;
    +    while ( out == AggOutcome.CALL_WORK_AGAIN) {
    +      //
    +      //  Read incoming batches and process their records
    +      //
    +      out = aggregator.doWork();
    +    }
    --- End diff --
    
    Done ( do while ...)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r122313161
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -512,122 +509,122 @@ private void updateEstMaxBatchSize(RecordBatch incoming) {
         }
       }
     
    +  /**
    +   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
    +   *  Once complete, get the incoming NEXT batch and process it as well, etc.
    +   *  For 1st phase, may return when an early output needs to be performed.
    +   *
    +   * @return Agg outcome status
    +   */
       @Override
       public AggOutcome doWork() {
    -    try {
    -      // Note: Keeping the outer and inner try blocks here to maintain some similarity with
    -      // StreamingAggregate which does somethings conditionally in the outer try block.
    -      // 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) {
    +    while (true) {
     
    -        // This would be called only once - after actual data arrives on incoming
    -        if ( schema == null && incoming.getRecordCount() > 0 ) {
    -          this.schema = incoming.getSchema();
    -          // Calculate the number of partitions based on actual incoming data
    -          delayedSetup();
    -        }
    +      // This would be called only once - first time actual data arrives on incoming
    +      if ( schema == null && incoming.getRecordCount() > 0 ) {
    +        this.schema = incoming.getSchema();
    +        currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
    +        // Calculate the number of partitions based on actual incoming data
    +        delayedSetup();
    +      }
     
    -        // loop through existing records, aggregating the values as necessary.
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Starting outer loop of doWork()...");
    +      //
    +      //  loop through existing records in this batch, aggregating the values as necessary.
    +      //
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Starting outer loop of doWork()...");
    +      }
    +      for (; underlyingIndex < currentBatchRecordCount; incIndex()) {
    +        if (EXTRA_DEBUG_2) {
    +          logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
             }
    -        for (; underlyingIndex < incoming.getRecordCount(); incIndex()) {
    -          if (EXTRA_DEBUG_2) {
    -            logger.debug("Doing loop with values underlying {}, current {}", underlyingIndex, currentIndex);
    -          }
    -          checkGroupAndAggrValues(currentIndex);
    -          // If adding a group discovered a memory pressure during 1st phase, then start
    -          // outputing some partition to free memory.
    -          if ( earlyOutput ) {
    -            outputCurrentBatch();
    -            incIndex(); // next time continue with the next incoming row
    -            return AggOutcome.RETURN_OUTCOME;
    -          }
    +        checkGroupAndAggrValues(currentIndex);
    +        // If adding a group discovered a memory pressure during 1st phase, then start
    +        // outputing some partition downstream in order to free memory.
    +        if ( earlyOutput ) {
    +          outputCurrentBatch();
    +          incIndex(); // next time continue with the next incoming row
    +          return AggOutcome.RETURN_OUTCOME;
             }
    +      }
    +
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Processed {} records", underlyingIndex);
    +      }
     
    -        if (EXTRA_DEBUG_1) {
    -          logger.debug("Processed {} records", underlyingIndex);
    +      // Cleanup the previous batch since we are done processing it.
    +      for (VectorWrapper<?> v : incoming) {
    +        v.getValueVector().clear();
    +      }
    +      //
    +      // Get the NEXT input batch, initially from the upstream, later (if there was a spill)
    +      // from one of the spill files (The spill case is handled differently here to avoid
    +      // collecting stats on the spilled records)
    +      //
    +      if ( handlingSpills ) {
    +        outcome = context.shouldContinue() ? incoming.next() : IterOutcome.STOP;
    +      } else {
    +        long beforeAlloc = allocator.getAllocatedMemory();
    +
    +        // Get the next RecordBatch from the incoming (i.e. upstream operator)
    +        outcome = outgoing.next(0, incoming);
    +
    +        // If incoming batch is bigger than our estimate - adjust the estimate to match
    +        long afterAlloc = allocator.getAllocatedMemory();
    +        long incomingBatchSize = afterAlloc - beforeAlloc;
    +        if ( estMaxBatchSize < incomingBatchSize) {
    +          logger.trace("Found a bigger incoming batch: {} , prior estimate was: {}", incomingBatchSize, estMaxBatchSize);
    +          estMaxBatchSize = incomingBatchSize;
             }
    +      }
     
    -        try {
    +      if (EXTRA_DEBUG_1) {
    +        logger.debug("Received IterOutcome of {}", outcome);
    +      }
     
    -          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();
    +      // Handle various results from getting the next batch
    +      switch (outcome) {
    +        case OUT_OF_MEMORY:
    +        case NOT_YET:
    +          return AggOutcome.RETURN_OUTCOME;
     
    -            // Get the next RecordBatch from the incoming
    -            IterOutcome out = outgoing.next(0, incoming);
    +        case OK_NEW_SCHEMA:
    --- End diff --
    
    Does this operator handle a new schema from upstream, except on the first batch? Can this occur when reading from a spill file? (The spill file was for a prior schema, say, that has since shifted to a new schema, so reading the spill file shifts back to the old schema?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812656
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    +      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
    +    }
    +  }
    +  /**
    +   * get new incoming: (when reading spilled files like an "incoming")
    +   * @return The (newly replaced) incoming
    +   */
    +  @Override
    +  public RecordBatch getNewIncoming() { return incoming; }
    +
    +  private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, ClassTransformationException, IOException {
    +    baseHashTable.updateIncoming(newIncoming); // after a spill - a new incoming
    +    this.incoming = newIncoming;
    +    nextPartitionToReturn = 0;
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      htables[i].reinit(newIncoming);
    +      if ( batchHolders[i] != null) {
    +        for (BatchHolder bh : batchHolders[i]) {
    +          bh.clear();
    +        }
    +        batchHolders[i].clear();
    +        batchHolders[i] = new ArrayList<BatchHolder>();
    +      }
    +      outBatchIndex[i] = 0;
    +      outputStream[i] = null;
    +      spilledBatchesCount[i] = 0;
    +      // spilledPaths[i] = null;
    +      spillFiles[i] = null;
    +    }
    +  }
    +
    +  /**
    +   *  Update the estimated max batch size to be used in the Hash Aggr Op.
    +   *  using the record batch size to get the row width.
    +   * @param incoming
    +   */
    +  private void updateEstMaxBatchSize(RecordBatch incoming) {
    +    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
    +    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
    +    logger.trace("Incoming sizer: {}",sizer);
    +    // An empty batch only has the schema, can not tell actual length of varchars
    +    // else use the actual varchars length, each capped at 50 (to match the space allocation)
    +    estRowWidth = sizer.rowCount() == 0 ? sizer.stdRowWidth() : sizer.netRowWidthCap50();
    +    estMaxBatchSize = estRowWidth * MAX_BATCH_SIZE;
    +
    +    // Get approx max (varchar) column width to get better memory allocation
    +    maxColumnWidth = Math.max(sizer.maxSize(), VARIABLE_MIN_WIDTH_VALUE_SIZE);
    +    maxColumnWidth = Math.min(maxColumnWidth, VARIABLE_MAX_WIDTH_VALUE_SIZE);
    +
    +    logger.trace("{} phase. Estimated row width: {}  batch size: {}  memory limit: {}  max column width: {}",
    +        isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",estRowWidth,estMaxBatchSize,memoryLimit,maxColumnWidth);
    +
    +    if ( estMaxBatchSize > memoryLimit ) {
    +      logger.warn("HashAggregate: Estimated max batch size {} is larger than the memory limit {}",estMaxBatchSize,memoryLimit);
    --- End diff --
    
    Good check, but is it a bit optimistic? Shouldn't the check also include necessary overhead beyond the one batch?
    ```
    (necessary overhead + estMaxBatchSize) > memoryLimit
    ```
    Can progress be made with only one batch? What is the minimum? Seems to be tricky with all the special cases that ended up being needed. Maybe a check for the in-memory version, the x partition spill versions?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119748615
  
    --- 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();
    +
    --- End diff --
    
    The doWork() function's basic outline is the same as before; this code change just peppered few lines of code in few places. I was trying to minimize the change ....



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118814054
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -400,114 +782,411 @@ public IterOutcome getOutcome() {
     
       @Override
       public int getOutputCount() {
    -    // return outputCount;
         return lastBatchOutputCount;
       }
     
       @Override
       public void cleanup() {
    -    if (htable != null) {
    -      htable.clear();
    -      htable = null;
    -    }
    +      if ( schema == null ) { return; } // not set up; nothing to clean
    +      for ( int i = 0; i < numPartitions; i++) {
    +          if (htables[i] != null) {
    +              htables[i].clear();
    +              htables[i] = null;
    +          }
    +          if ( batchHolders[i] != null) {
    +              for (BatchHolder bh : batchHolders[i]) {
    +                    bh.clear();
    +              }
    +              batchHolders[i].clear();
    +              batchHolders[i] = null;
    +          }
    +
    +          // delete any (still active) output spill file
    +          if ( outputStream[i] != null && spillFiles[i] != null) {
    +            try {
    +              spillSet.delete(spillFiles[i]);
    +            } catch(IOException e) {
    +              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
    +            }
    +          }
    +      }
    +      // delete any spill file left in unread spilled partitions
    +      while ( ! spilledPartitionsList.isEmpty() ) {
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        try {
    +          spillSet.delete(sp.spillFile);
    +        } catch(IOException e) {
    +          logger.warn("Cleanup: Failed to delete spill file {}",sp.spillFile);
    +        }
    +      }
    +      spillSet.close(); // delete the spill directory(ies)
         htIdxHolder = null;
         materializedValueFields = null;
         outStartIdxHolder = null;
         outNumRecordsHolder = null;
    +  }
     
    -    if (batchHolders != null) {
    -      for (BatchHolder bh : batchHolders) {
    +  // First free the memory used by the given (spilled) partition (i.e., hash table plus batches)
    +  // then reallocate them in pristine state to allow the partition to continue receiving rows
    +  private void reinitPartition(int part) throws SchemaChangeException, ClassTransformationException, IOException {
    +    assert htables[part] != null;
    +    htables[part].reset();
    +    if ( batchHolders[part] != null) {
    +      for (BatchHolder bh : batchHolders[part]) {
             bh.clear();
           }
    -      batchHolders.clear();
    -      batchHolders = null;
    +      batchHolders[part].clear();
         }
    +    batchHolders[part] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
       }
     
    -//  private final AggOutcome setOkAndReturn() {
    -//    this.outcome = IterOutcome.OK;
    -//    for (VectorWrapper<?> v : outgoing) {
    -//      v.getValueVector().getMutator().setValueCount(outputCount);
    -//    }
    -//    return AggOutcome.RETURN_OUTCOME;
    -//  }
     
       private final void incIndex() {
         underlyingIndex++;
         if (underlyingIndex >= incoming.getRecordCount()) {
           currentIndex = Integer.MAX_VALUE;
           return;
         }
    -    currentIndex = getVectorIndex(underlyingIndex);
    +    try { currentIndex = getVectorIndex(underlyingIndex); }
    +    catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc);}
       }
     
       private final void resetIndex() {
         underlyingIndex = -1;
         incIndex();
       }
     
    -  private void addBatchHolder() {
    +  private boolean isSpilled(int part) {
    +    return outputStream[part] != null;
    +  }
    +  /**
    +   * Which partition to choose for flushing out (i.e. spill or return) ?
    +   * - The current partition (to which a new bach holder is added) has a priority,
    +   *   because its last batch holder is full.
    +   * - Also the largest prior spilled partition has some priority, as it is already spilled;
    +   *   but spilling too few rows (e.g. a single batch) gets us nothing.
    +   * - So the largest non-spilled partition has some priority, to get more memory freed.
    +   * Need to weigh the above three options.
    +   *
    +   *  @param currPart - The partition that hit the memory limit (gets a priority)
    +   *  @return The partition (number) chosen to be spilled
    +   */
    +  private int chooseAPartitionToFlush(int currPart) {
    +    if ( ! is2ndPhase ) { return currPart; } // 1st phase: just use the current partition
    +    int currPartSize = batchHolders[currPart].size();
    +    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
    +    // first find the largest spilled partition
    +    int maxSizeSpilled = -1;
    +    int indexMaxSpilled = -1;
    +    for (int isp = 0; isp < numPartitions; isp++ ) {
    +      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
    +        maxSizeSpilled = batchHolders[isp].size();
    +        indexMaxSpilled = isp;
    +      }
    +    }
    +    // Give the current (if already spilled) some priority
    +    if ( isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
    +      maxSizeSpilled = currPartSize ;
    +      indexMaxSpilled = currPart;
    +    }
    +    // now find the largest non-spilled partition
    +    int maxSize = -1;
    +    int indexMax = -1;
    +    // Use the largest spilled (if found) as a base line, with a factor of 4
    +    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
    +      indexMax = indexMaxSpilled;
    +      maxSize = 4 * maxSizeSpilled ;
    +    }
    +    for ( int insp = 0; insp < numPartitions; insp++) {
    +      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
    +        indexMax = insp;
    +        maxSize = batchHolders[insp].size();
    +      }
    +    }
    +    // again - priority to the current partition
    +    if ( ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
    +      return currPart;
    +    }
    +    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
    +      return -1; // try skipping this spill
    +    }
    +    return indexMax;
    +  }
    +
    +  /**
    +   * Iterate through the batches of the given partition, writing them to a file
    +   *
    +   * @param part The partition (number) to spill
    +   */
    +  private void spillAPartition(int part) {
    +
    +    ArrayList<BatchHolder> currPartition = batchHolders[part];
    +    rowsInPartition = 0;
    +    if ( EXTRA_DEBUG_SPILL ) {
    +      logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, cycleNum, currPartition.size());
    +    }
    +
    +    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
    +
    +    // If this is the first spill for this partition, create an output stream
    +    if ( ! isSpilled(part) ) {
    +
    +      spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
    +
    +      try {
    +        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to open spill file: " + spillFiles[part]);
    +      }
    +    }
    +
    +    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
    +
    +      // get the number of records in the batch holder that are pending output
    +      int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +      rowsInPartition += numPendingOutput;  // for logging
    +      rowsSpilled += numPendingOutput;
    +
    +      allocateOutgoing(numPendingOutput);
    +
    +      currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +      int numOutputRecords = outNumRecordsHolder.value;
    +
    +      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +      // set the value count for outgoing batch value vectors
    +      /* int i = 0; */
    +      for (VectorWrapper<?> v : outgoing) {
    +        v.getValueVector().getMutator().setValueCount(numOutputRecords);
    +        /*
    +        // print out the first row to be spilled ( varchar, varchar, bigint )
    +        try {
    +          if (i++ < 2) {
    +            NullableVarCharVector vv = ((NullableVarCharVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          } else {
    +            NullableBigIntVector vv = ((NullableBigIntVector) v.getValueVector());
    +            logger.info("FIRST ROW = {}", vv.getAccessor().get(0));
    +          }
    +        } catch (Exception e) { logger.info("While printing the first row - Got an exception = {}",e); }
    +        */
    +      }
    +
    +      outContainer.setRecordCount(numPendingOutput);
    +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
    +      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
    +      Stopwatch watch = Stopwatch.createStarted();
    +      try {
    +        outputBatch.writeToStream(outputStream[part]);
    +      } catch (IOException e) {
    +        throw new DrillRuntimeException("Hash Aggregation failed to write to output stream: " + outputStream[part].toString());
    +      }
    +      outContainer.zeroVectors();
    +      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
    +    }
    +
    +    spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
    +
    +    logger.trace("HASH AGG: Spilled {} rows from {} batches of partition {}", rowsInPartition, currPartition.size(), part);
    +  }
    +
    +  private void addBatchHolder(int part) {
    +
         BatchHolder bh = newBatchHolder();
    -    batchHolders.add(bh);
    +    batchHolders[part].add(bh);
     
         if (EXTRA_DEBUG_1) {
    -      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders.size());
    +      logger.debug("HashAggregate: Added new batch; num batches = {}.", batchHolders[part].size());
         }
     
         bh.setup();
       }
     
    -  // Overridden in the generated class when created as plain Java code.
    -
    +  // These methods are overridden in the generated class when created as plain Java code.
       protected BatchHolder newBatchHolder() {
         return new BatchHolder();
       }
    +  protected SpilledRecordbatch newSpilledRecordBatch(String arg1, int arg2, FragmentContext arg4, BatchSchema arg5, OperatorContext arg6) {
    +    return new SpilledRecordbatch(arg1, arg2, arg4, arg5, arg6);
    +  }
     
    +  /**
    +   * Output the next batch from partition "nextPartitionToReturn"
    +   *
    +   * @return iteration outcome (e.g., OK, NONE ...)
    +   */
       @Override
       public IterOutcome outputCurrentBatch() {
    -    if (outBatchIndex >= batchHolders.size()) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
    +
    +    // when incoming was an empty batch, just finish up
    +    if ( schema == null ) {
    +      logger.trace("Incoming was empty; output is an empty batch.");
    +      this.outcome = IterOutcome.NONE; // no records were read
    +      allFlushed = true;
    +      return this.outcome;
         }
     
    -    // get the number of records in the batch holder that are pending output
    -    int numPendingOutput = batchHolders.get(outBatchIndex).getNumPendingOutput();
    +    // Initialization (covers the case of early output)
    +    ArrayList<BatchHolder> currPartition = batchHolders[earlyPartition];
    +    int currOutBatchIndex = outBatchIndex[earlyPartition];
    +    int partitionToReturn = earlyPartition;
    +
    +    if ( ! earlyOutput ) {
    +      // Update the next partition to return (if needed)
    +      // skip fully returned (or spilled) partitions
    +      while (nextPartitionToReturn < numPartitions) {
    +        //
    +        // If this partition was spilled - spill the rest of it and skip it
    +        //
    +        if ( isSpilled(nextPartitionToReturn) ) {
    +          spillAPartition(nextPartitionToReturn); // spill the rest
    +          SpilledPartition sp = new SpilledPartition();
    +          sp.spillFile = spillFiles[nextPartitionToReturn];
    +          sp.spilledBatches = spilledBatchesCount[nextPartitionToReturn];
    +          sp.cycleNum = cycleNum; // remember the current cycle
    +          sp.origPartn = nextPartitionToReturn; // for debugging / filename
    +          sp.prevOrigPartn = originalPartition; // for debugging / filename
    +          spilledPartitionsList.add(sp);
    +          try {
    +            reinitPartition(nextPartitionToReturn); // free the memory
    +          } catch (Exception e) {throw new RuntimeException(e);}
    +          try {
    +            long posn = spillSet.getPosition(outputStream[nextPartitionToReturn]);
    +            spillSet.tallyWriteBytes(posn); // for the IO stats
    +            outputStream[nextPartitionToReturn].close();
    +          } catch (IOException e) { throw new RuntimeException(e); }
    +          outputStream[nextPartitionToReturn] = null;
    +        }
    +        else {
    +          currPartition = batchHolders[nextPartitionToReturn];
    +          currOutBatchIndex = outBatchIndex[nextPartitionToReturn];
    +          // If curr batch (partition X index) is not empty - proceed to return it
    +          if (currOutBatchIndex < currPartition.size() && 0 != currPartition.get(currOutBatchIndex).getNumPendingOutput()) {
    +            break;
    +          }
    +        }
    +        nextPartitionToReturn++; // else check next partition
    +      }
    +
    +      // if passed the last partition
    +      if (nextPartitionToReturn >= numPartitions) {
    +        // The following "if" is probably never used; due to a similar check at the end of this method
    +        if ( spilledPartitionsList.isEmpty() ) { // and no spilled partitions
    +          allFlushed = true;
    +          this.outcome = IterOutcome.NONE;
    +          if ( is2ndPhase ) {
    +            stats.setLongStat(Metric.SPILL_MB, // update stats - total MB spilled
    +                (int) Math.round(spillSet.getWriteBytes() / 1024.0D / 1024.0));
    +          }
    +          return outcome;  // then return NONE
    +        }
    +        // Else - there are still spilled partitions to process - pick one and handle just like a new incoming
    +        buildComplete = false; // go back and call doWork() again
    +        handlingSpills = true; // beginning to work on the spill files
    +        // pick a spilled partition; set a new incoming ...
    +        SpilledPartition sp = spilledPartitionsList.remove(0);
    +        SpilledRecordbatch newIncoming = newSpilledRecordBatch(sp.spillFile, sp.spilledBatches, context, schema, oContext);
    +        originalPartition = sp.origPartn; // used for the filename
    +        logger.trace("Reading back spilled original partition {} as an incoming",originalPartition);
    +        // prevOriginalPartition = sp.prevOrigPartn;
    +        // Initialize .... new incoming, new set of partitions
    +        try { initializeSetup(newIncoming); } catch (Exception e) { throw new RuntimeException(e); }
    +        // update the cycle num if needed
    +        // The current cycle num should always be one larger than in the spilled partition
    +        if ( cycleNum == sp.cycleNum ) {
    +          cycleNum = 1 + sp.cycleNum;
    +          stats.setLongStat(Metric.SPILL_CYCLE, cycleNum); // update stats
    +          // report memory stressful situations
    +          if ( cycleNum == 2 ) { logger.info("SECONDARY SPILLING "); }
    +          if ( cycleNum == 3 ) { logger.info("TERTIARY SPILLING "); }
    +        }
    +        if ( EXTRA_DEBUG_SPILL ) {
    +          logger.debug("Start reading spilled partition {} (prev {}) from cycle {} (with {} batches). More {} spilled partitions left.",
    +              sp.origPartn, sp.prevOrigPartn, sp.cycleNum, sp.spilledBatches, spilledPartitionsList.size());
    +        }
    +        return IterOutcome.RESTART;
    +      }
    +
    +      partitionToReturn = nextPartitionToReturn ;
     
    -    if (numPendingOutput == 0) {
    -      this.outcome = IterOutcome.NONE;
    -      return outcome;
         }
     
    +    // get the number of records in the batch holder that are pending output
    +    int numPendingOutput = currPartition.get(currOutBatchIndex).getNumPendingOutput();
    +
    +    // The following accounting is for logging, metrics, etc.
    +    rowsInPartition += numPendingOutput ;
    +    if ( ! handlingSpills ) { rowsNotSpilled += numPendingOutput; }
    +    else { rowsSpilledReturned += numPendingOutput; }
    +    if ( earlyOutput ) { rowsReturnedEarly += numPendingOutput; }
    +
         allocateOutgoing(numPendingOutput);
     
    -    batchHolders.get(outBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
    +    currPartition.get(currOutBatchIndex).outputValues(outStartIdxHolder, outNumRecordsHolder);
         int numOutputRecords = outNumRecordsHolder.value;
     
         if (EXTRA_DEBUG_1) {
           logger.debug("After output values: outStartIdx = {}, outNumRecords = {}", outStartIdxHolder.value, outNumRecordsHolder.value);
         }
    -    this.htable.outputKeys(outBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
    +
    +    this.htables[partitionToReturn].outputKeys(currOutBatchIndex, this.outContainer, outStartIdxHolder.value, outNumRecordsHolder.value);
     
         // set the value count for outgoing batch value vectors
         for (VectorWrapper<?> v : outgoing) {
           v.getValueVector().getMutator().setValueCount(numOutputRecords);
         }
     
    -//    outputCount += numOutputRecords;
    -
         this.outcome = IterOutcome.OK;
     
    -    logger.debug("HashAggregate: Output current batch index {} with {} records.", outBatchIndex, numOutputRecords);
    +    // logger.debug("HashAggregate: Output {} current batch index {} with {} records for partition {}.", earlyOutput ? "(early)" : "",
    +    //    outBatchIndex, numOutputRecords, partitionToReturn);
    +    if ( EXTRA_DEBUG_SPILL && is2ndPhase ) {
    +      logger.debug("So far returned {} + SpilledReturned {}  total {} (spilled {})",rowsNotSpilled,rowsSpilledReturned,
    +        rowsNotSpilled+rowsSpilledReturned,
    +        rowsSpilled);
    +    }
     
         lastBatchOutputCount = numOutputRecords;
    -    outBatchIndex++;
    -    if (outBatchIndex == batchHolders.size()) {
    -      allFlushed = true;
    +    outBatchIndex[partitionToReturn]++;
    +    // if just flushed the last batch in the partition
    +    if (outBatchIndex[partitionToReturn] == currPartition.size()) {
    +
    +      if ( EXTRA_DEBUG_SPILL ) {
    +        logger.debug("HashAggregate: {} Flushed partition {} with {} batches total {} rows",
    +            earlyOutput ? "(Early)" : "",
    +            partitionToReturn, outBatchIndex[partitionToReturn], rowsInPartition);
    +      }
    +      rowsInPartition = 0; // reset to count for the next partition
    +
    +      try {
    +        // deallocate memory used by this partition, and re-initialize
    +        reinitPartition(partitionToReturn);
    +      } catch (SchemaChangeException sce) {
    +        throw new DrillRuntimeException("Hash Aggregation can not handle schema changes.");
    --- End diff --
    
    This isn't a real schema change, it is a bogus one that is an artifact of the way the low level functions work. Throw an `IllegalStateException`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r118812409
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    +    else {
    +      // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
    +      updateEstMaxBatchSize(incoming);
    +    }
    +    long memAvail = memoryLimit - allocator.getAllocatedMemory();
    +    if ( !canSpill ) { // single phase, or spill disabled by configuation
    +      numPartitions = 1; // single phase should use only a single partition (to save memory)
    +    } else { // two phase
    +      // Adjust down the number of partitions if needed - when the memory available can not hold as
    +      // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
    +      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 8 * 1024 * 1024) > memAvail ) {
    +        numPartitions /= 2;
    +        if ( numPartitions < 2) {
    +          if ( is2ndPhase ) { canSpill = false; } // 2nd phase needs at least 2 to make progress
    +          break;
    +        }
    +      }
    +    }
    +    logger.trace("{} phase. Number of partitions chosen: {}. {} spill", isTwoPhase?(is2ndPhase?"2nd":"1st"):"Single",
    +        numPartitions, canSpill ? "Can" : "Cannot");
    +
    +    // The following initial safety check should be revisited once we can lower the number of rows in a batch
    +    // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
    +    if ( numPartitions == 1 ) {
    +      // if too little memory - behave like the old code -- no memory limit for hash aggregate
    +      allocator.setLimit(10_000_000_000L);
    +    }
    +    // Based on the number of partitions: Set the mask and bit count
    +    partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
    +    bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
    +
    +    // Create arrays (one entry per partition)
    +    htables = new HashTable[numPartitions] ;
    +    batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
    +    outBatchIndex = new int[numPartitions] ;
    +    outputStream = new OutputStream[numPartitions];
    +    spilledBatchesCount = new int[numPartitions];
    +    // spilledPaths = new Path[numPartitions];
    +    spillFiles = new String[numPartitions];
    +    spilledPartitionsList = new ArrayList<SpilledPartition>();
    +
    +    plannedBatches = numPartitions; // each partition should allocate its first batch
    +
    +    // initialize every (per partition) entry in the arrays
    +    for (int i = 0; i < numPartitions; i++ ) {
    +      try {
    +        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds, numPartitions);
    +        this.htables[i].setMaxVarcharSize(maxColumnWidth);
    +      } catch (IllegalStateException ise) {} // ignore
    +      catch (Exception e) { throw new DrillRuntimeException(e); }
    --- End diff --
    
    Why translate Exception (checked) into DRE (unchecked?)
    
    Better would be to catch the actual exception, then determine how to explain the problem to the user by throwing the proper `UserException`: out of memory, disk I/O or whatever.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119259465
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -266,17 +508,138 @@ public void setup(HashAggregate hashAggrConfig, HashTableConfig htConfig, Fragme
           }
         }
     
    -    ChainedHashTable ht =
    +    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
    +    baseHashTable =
             new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
    -    this.htable = ht.createAndSetupHashTable(groupByOutFieldIds);
    -
    +    this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
         numGroupByOutFields = groupByOutFieldIds.length;
    -    batchHolders = new ArrayList<BatchHolder>();
    -    // First BatchHolder is created when the first put request is received.
     
         doSetup(incoming);
       }
     
    +  /**
    +   *  Delayed setup are the parts from setup() that can only be set after actual data arrives in incoming
    +   *  This data is used to compute the number of partitions.
    +   */
    +  private void delayedSetup() {
    +
    +    // Set the number of partitions from the configuration (raise to a power of two, if needed)
    +    numPartitions = context.getConfig().getInt(ExecConstants.HASHAGG_NUM_PARTITIONS_KEY);
    +    if ( numPartitions == 1 ) {
    +      canSpill = false;
    +      logger.warn("Spilling was disabled");
    +    }
    +    while (Integer.bitCount(numPartitions) > 1) { // in case not a power of 2
    +      numPartitions++;
    +    }
    +    if ( schema == null ) { estMaxBatchSize = 0; } // incoming was an empty batch
    --- End diff --
    
    I think some test case (with an empty batch) failed there; anyway does not hurt. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #822: DRILL-5457: Spill implementation for Hash Aggregate

Posted by Ben-Zvi <gi...@git.apache.org>.
Github user Ben-Zvi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/822#discussion_r119245285
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java ---
    @@ -166,20 +252,23 @@ public BatchHolder() {
         }
     
         private boolean updateAggrValues(int incomingRowIdx, int idxWithinBatch) {
    -      updateAggrValuesInternal(incomingRowIdx, idxWithinBatch);
    +      try { updateAggrValuesInternal(incomingRowIdx, idxWithinBatch); }
    +      catch (SchemaChangeException sc) { throw new DrillRuntimeException(sc); }
    --- End diff --
    
    Yes, but only when %100 sure; else if there's a chance for a SchemaChange -- Better use UnsupportedOperationException ...  
      Code change done !!



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---