-
Notifications
You must be signed in to change notification settings - Fork 1.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Memory limited hash join #5490
Memory limited hash join #5490
Conversation
Thank you @korowa -- I plan to review this PR carefully tomorrow morning |
Nice PR! I think it would be great if we could run some benchmarks to show that we're not regressing too much (e.g. running tpch benchmark queries with joins). Some reasons I defaulted to initializing the hashmap using the size of the left side is as following:
|
Thanks @Dandandan!
I've done a couple of iterations for
With the result (a bit floating but I can't see any noticeable regressions -- ofc I may be wrong):
Also
is not showing any difference in join build / probe times. Full log is in attachment. As for now, I tend to think that these changes do not affect performance significantly (due to the fact that reservations and memory estimations executed only once per partition/query), but I'll be happy to perform any additional checks.
All of them look pretty reasonable and I agree with this approach, I was just trying to explain why I've decided to duplicate calculations from some functions subsequently called by |
Awesome, thanks for checking. It might be worth it to repeat the benchmark run with |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I went over this change very carefully @korowa -- really nice work 🏆
I am running the benchmarks but assuming they look good I think this PR can be merged.
cc @liukun4515
/// [`MemoryReservation`] used at query operator level | ||
/// `Option` wrapper allows to initialize empty reservation in operator constructor, | ||
/// and set it to actual reservation at stream level. | ||
pub(crate) type OperatorMemoryReservation = Arc<Mutex<Option<SharedMemoryReservation>>>; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder if the double layer of Mutex is necessary (Mutex over an Option of a Mutex)
Update after reading the code I see why it is necessary.
Perhaps it would make the code clearer (as a follow on PR, perhaps) if we had a proper SharedMemoryReservation
that could be cloned and handled mutability internally. 🤔
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Unfortunately I haven't found better solution yet, but I'll check if there is better way to wrap MemoryReservation
into something allowing attempts of creation (or initialization) reservation from multiple streams.
Another solution, probably, is to allow to allow creation of MemoryReservation without providing context (in HashJoinExec::new
), and registering it in MemoryPool
later (in HashJoinExec.execute
, when we have context available).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So, if there are no objections, I'd prefer to figure out proper way to handle operator-level reservations as a separate follow-up PR, or during adding memory limitations to one of remaining joins (at least NL will have the same issue 100%)
@@ -1173,6 +1149,18 @@ impl HashJoinStream { | |||
}; | |||
build_timer.done(); | |||
|
|||
// Reserving memory for visited_left_side bitmap in case it hasn't been initialied yet | |||
// and join_type requires to store it | |||
if self.visited_left_side.is_none() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You might be able to make this less verbose by moving the initialization into the initialization of visited_left_size
let visited_left_side = self.visited_left_side.get_or_insert_with(|| {
let visited_bitmap_size = bit_util::ceil(left_data.1.num_rows(), 8);
self.reservation.lock().try_grow(visited_bitmap_size)?;
self.join_metrics.build_mem_used.add(visited_bitmap_size);
...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It was my first intention, but then I've realized that Option
doesn't have fallible versions of get_or_insert
-like methods (i.e.try_get_or_insert_with
in this case would be just right), and, the only alternative I saw, was storing visited_left_side
as a `Result<_, DataFusionError> which, I guess, would also be not really clean.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I ran with the tpch query benchmark with SF10
(10GB) using memory per @Dandandan 's suggestion
cargo run --release --bin tpch -- benchmark datafusion -m --query 17 --iterations 5 --path ~/tpch_data/parquet_data_SF10 --format parquet
Here is the result on main (merge-base with this PR, deeaa56)
Query 17 iteration 0 took 86475.5 ms and returned 1 rows
Query 17 iteration 1 took 80791.0 ms and returned 1 rows
Query 17 iteration 2 took 79514.7 ms and returned 1 rows
Query 17 iteration 3 took 81253.2 ms and returned 1 rows
Query 17 iteration 4 took 99265.5 ms and returned 1 rows
Query 17 avg time: 85459.99 ms
Here it is on this branch (if anything it looks faster 🤔 )
Query 17 iteration 0 took 80484.2 ms and returned 1 rows
Query 17 iteration 1 took 87880.9 ms and returned 1 rows
Query 17 iteration 2 took 82665.1 ms and returned 1 rows
Query 17 iteration 3 took 83641.5 ms and returned 1 rows
Query 17 iteration 4 took 83985.6 ms and returned 1 rows
Query 17 avg time: 83731.47 ms
Nice work
|
||
reservation.lock().try_grow(estimated_hastable_size)?; | ||
metrics.build_mem_used.add(estimated_hastable_size); | ||
|
||
let mut hashmap = JoinHashMap(RawTable::with_capacity(num_rows)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah I didn't realize the allocation is the same, only guarding against overallocation👍
I've also benchmarked vs main using
|
It looks like the changes don't affect performance, which is perfect 👍 |
Thanks again @korowa ! |
Benchmark runs are scheduled for baseline = ff013e2 and contender = f5d23ff. f5d23ff is a master commit associated with this PR. Results will be available as each benchmark for each run completes. |
Thank you @alamb @Dandandan ! I've also created #5531 to support passing boolean by values in one of the tests added in this PR 😞 |
Thank you for your diligence here 🏅 |
Nice, thanks @korowa ! |
Which issue does this PR close?
Part of #5339
Part of #5220
Rationale for this change
Control over memory allocations made by
HashJoinExec
What changes are included 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:CrossJoinExec
memory reserved for eachRecordBatch
fetched from build-side, reservation size calculated using already fetched batch, which could lead to insignificant (no more then single batch size) overallocationsRawTable::with_capacity()
and it could be massive, so it seems reasonable to preemptively estimate the size required forRawTable
and reserve it in pool -- the calculations of the size are described in commentArrayBuilder
creation.Another change is stream/operator-level reservation (also affects
CrossJoinExec
) -- storingMemoryReservation
, and freeing it after...JoinStream
(how it was done in #5339) doesn't seem to be correct forCrossJoinExec
andHashJoinExec
withPartitionMode::CollectLeft
for a couple of reasons: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...JoinExec
rather than in...ExecStream
, and the object could potentially outlive reservationThese issues are fixed by
OperatorMemoryReservation
added to bothCrossJoinExec
andHashJoinExec
-- for hash joins this reservation used only forPartitionedMode::CollectLeft
cases, in case of parittioned joins -- stream-level memory reservations are used.Are these changes tested?
Test cases for overallocation has been added for all
JoinType
s, for bothCollectLeft
andPartitioned
modes.Are there any user-facing changes?
HashJoinExec
should now respect runtime memory limitations.