[
https://issues.apache.org/jira/browse/DRILL-6310?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16516696#comment-16516696
]
ASF GitHub Bot commented on DRILL-6310:
---------------------------------------
ilooner commented on a change in pull request #1324: DRILL-6310: limit batch
size for hash aggregate
URL: https://github.com/apache/drill/pull/1324#discussion_r196299718
##########
File path:
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
##########
@@ -1317,7 +1364,7 @@ private void checkGroupAndAggrValues(int incomingRowIdx)
{
useReservedValuesMemory(); // try to preempt an OOM by using the
reserve
- addBatchHolder(currentPartition); // allocate a new (internal) values
batch
+ addBatchHolder(currentPartition, getBatchSize()); // allocate a new
(internal) values batch
Review comment:
I don't really agree with this approach. We are dynamically adjusting the
BatchHolder size based on new records we see coming in, but does this even make
sense to do?
We don't really have control over which partition incoming records get
hashed into, so records will get added to BatchHolders that were sized based on
older input data. And those BatchHolder sizes may no longer make sense for the
new data. Therefore there probably isn't any benefit to doing this.
I guess this was done because with the current code the size of an output
batch is equivalent to the size of a batch holder since an entire batch holder
is emitted at once. But instead of trying to work within that limitation I
think we should actually change to code to allow the output of a partial batch
holder. Additionally we should not allow heterogenous batch holder sizes. We
could use the statistics from the first input batch to do a reasonable sizing
for batch holders instead of the current 64k default, which would be a great
change. However, after a BatchHolder size is selected it should not be changed.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> limit batch size for hash aggregate
> -----------------------------------
>
> Key: DRILL-6310
> URL: https://issues.apache.org/jira/browse/DRILL-6310
> Project: Apache Drill
> Issue Type: Improvement
> Components: Execution - Flow
> Affects Versions: 1.13.0
> Reporter: Padma Penumarthy
> Assignee: Padma Penumarthy
> Priority: Major
> Fix For: 1.14.0
>
>
> limit batch size for hash aggregate based on memory.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)