korowa opened a new pull request, #5490:
URL: https://github.com/apache/arrow-datafusion/pull/5490

   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and 
enhancements and this helps us generate change logs for our releases. You can 
link an issue to this PR using the GitHub syntax. For example `Closes #123` 
indicates that this PR will close issue #123.
   -->
   
   Part of #5339
   Part of #5220 
   
   # Rationale for this change
   
   <!--
    Why are you proposing this change? If this is already explained clearly in 
the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your 
changes and offer better suggestions for fixes.  
   -->
   
   Control over memory allocations made by `HashJoinExec`
   
   # What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is 
sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   Major change is `MemoryReservation.try_grow` calls added to 3 places where 
allocations for build-side data and related structures are performed:
   1. build-side record batches collection -- similar to `CrossJoinExec` memory 
reserved for each `RecordBatch` fetched from build-side, reservation size 
calculated using already fetched batch, which could lead to insignificant (no 
more then single batch size) overallocations
   2. build-side hashtable -- there is only one allocation happens during 
`RawTable::with_capacity()` and it could be massive, so it seems reasonable to 
preemptively estimate the size required for `RawTable` and reserve it in pool 
-- the calculations of the size are described in comment
   3. visited_left_side bitmap in each stream (if required by join type) -- 
like in case with hashtable, there is an attempt to estimate required size and 
reserve memory before `ArrayBuilder` creation.
   
   Another change is stream/operator-level reservation (also affects 
`CrossJoinExec`) -- storing `MemoryReservation`, and freeing it after 
`...JoinStream` (how it was done in #5339) doesn't seem to be correct for 
`CrossJoinExec` and `HashJoinExec` with `PartitionMode::CollectLeft` for a 
couple of reasons:
   1. only one stream out of multiple partitions executes `collect_left_data` 
function and reserves memory -- in case of partition processing time being 
skewed, the only stream with reservation could be dropped before Join execution 
is completed
   2. LeftData stored in `...JoinExec` rather than in `...ExecStream`, and the 
object could potentially outlive reservation
   
   These issues are fixed by `OperatorMemoryReservation` added to both 
`CrossJoinExec` and `HashJoinExec` -- for hash joins this reservation used only 
for `PartitionedMode::CollectLeft` cases, in case of parittioned joins -- 
stream-level memory reservations are used.
   
   # Are these changes tested?
   
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   5. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are 
they covered by existing tests)?
   -->
   
   Test cases for overallocation has been added for all `JoinType`s, for both 
`CollectLeft` and `Partitioned` modes.
   
   # Are there any user-facing changes?
   
   <!--
   If there are user-facing changes then we may require documentation to be 
updated before approving the PR.
   -->
   
   `HashJoinExec` should now respect runtime memory limitations.
   
   <!--
   If there are any breaking changes to public APIs, please add the `api 
change` label.
   -->


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to