Fix bug while materializing scan's result to frames#15987
Fix bug while materializing scan's result to frames#15987cryptoe merged 6 commits intoapache:masterfrom
Conversation
| // Math.max(allocationSize, bytes) in case "bytes" is greater than SOFT_MAXIMUM_ALLOCATION_SIZE. | ||
| final Optional<ResourceHolder<WritableMemory>> newMemory = | ||
| allocator.allocate(Math.max(nextAllocationSize, bytes)); | ||
| allocator.allocate(Math.min(allocator.available(), Math.max(nextAllocationSize, bytes))); |
There was a problem hiding this comment.
- Could you please explain this statement.
If you look at line 129, we are 100% sure that (bytes<=alllocator.avalable()) so why would be allocate a new chunk which is exactly same as allocate.available().
We should never go more than the bytes required rite?
NextAllocationSize is more of minCheck.
There was a problem hiding this comment.
I'll explain with an example here, consider the following scenario:
nextAllocationSizewill be in multiples of 2, let's say for a particular run of the dependable memory, it is 1024.- Someone wants to allocate 600 bytes, i.e.
bytes = 600 - Remaining memory in the allocator is 700, therefore
allocator.available() = 700
Since someone wants to allocate 600 bytes, and the available bytes in the allocator are 700, the allocation should succeed, and we should be able to give them 600 bytes (This is what Line 129 also checks)
However, if you look at the original code, it will do
allocator.allocate(Math.max(1024, 600) i.e. allocator.allocate(1024) which would fail.
The new code will do
allocator.allocate(Math.min(700, Math.max(1024, 600)), i.e. allocator.allocate(700), which should pass (which is the required behavior).
nextAllocationSize is more like a "2x buffer allocator limit" - every time we hit the limit of allocation, we multiply the next block to allocate by 2. So we do allocations like 1, 2, 4, 8, 16, ...., to minimize the number of allocations we need to do (and amortize the cost, as per my understanding). I think a similar principle is applied when dictionaries do dynamic hashing.
Therefore even though the caller requests bytes, we give the caller a larger block, so that the next time the caller requests bytes, we don't reallocate again. However, this fails to take into account the edge case, that even though the caller requests x bytes, and the allocator can satisfy that condition, but not the condition for nextAllocationSize, we fail, even though we should pass. Hence a cap of the allocation, and the available memory. In normal cases, allocator.available >>>> Math.max(nextAllocationSize, bytes), therefore most of the time the code should be doing what it's supposed to.
There was a problem hiding this comment.
Added a comment in the code to clarify.
There was a problem hiding this comment.
Thanks for the explanation. SGTM
| throw DruidException | ||
| .forPersona(DruidException.Persona.DEVELOPER) | ||
| .ofCategory(DruidException.Category.CAPACITY_EXCEEDED) | ||
| .build("Subquery's row size exceeds the frame size and therefore cannot write the subquery's " |
There was a problem hiding this comment.
We should mention here that a single row size exceeds the frameSize limit of xxx. We should also mention the corrective action.
If the user is hitting into this limit, they should change the input data with either smaller rows or change the query in such a way that does not read particularly fat columns.
There was a problem hiding this comment.
I will update the message with the frame size.
However, I don't think it makes sense to put the corrective action here, given that this is an error message aimed at the DEVELOPER persona. Messages aimed at developers mean that something went wrong here and we don't expect to hit these criteria. Given that this is a special case wherein the user's data shape can trigger this, I still think that we shouldn't add a corrective action because:
a) If the row > frameSize, it usually is due to a large array/string column, and not lots of individual columns.
b) There may not be a way to correct the subquery while preserving the correctness.
c) Such subqueries won't get limited properly, because the individual row size is too large, that we expect to overshoot the memory limits a lot.
In all, we usually won't expect to hit this error message. If we still want to present a corrective action, I think we should do both of the following instead:
a) Change the persona of the error message, which I am not a big fan of, given that hitting this usually means suboptimal use of memory limiting, since each row > 8MB (wherein each row will become hard to limit)
b) And change the corrective action to disable the memory based limiting, since it is a super-specific case (that we shouldn't be supporting anyways)
WDYT?
There was a problem hiding this comment.
Changed the message
|
Thanks for the review @cryptoe! Addressed the comments in the latest commit. |
Description
While converting
Sequence<ScanResultValue>toSequence<Frames>, whenmaxSubqueryBytesis enabled, we batch the results to prevent creating a single frame perScanResultValue.Batching requires peeking into the actual value, and checking if the row signature of the scan result’s value matches that of the previous value.Since we can do this indefinitely (in the worst case all of them have the same signature), we keep fetching them and accumulating them in a list (on the heap). We don’t really know how much to batch before we actually write the value as frames.
The PR modifies the batching logic to not accumulate the results in an intermediary list.
Key changed/added classes in this PR
MyFooOurBarTheirBazThis PR has: