-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
colexec: fix spilling queue #58013
colexec: fix spilling queue #58013
Conversation
35a1814
to
364361a
Compare
364361a
to
0c57b12
Compare
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.
Nice to see this 😂 For reference, this is what I did a while ago but never ended up finishing: https://github.com/asubiotto/cockroach/commit/450cecbf71b0004779054f48f8c718b5b01eb393.
Are there any router benchmarks to check out?
Reviewed 6 of 10 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @asubiotto and @yuzefovich)
pkg/col/coldata/testutils.go, line 63 at r1 (raw file):
// 'maybeHasNulls' field, so we override it manually to be 'true' for // both nulls vectors if it is 'true' for at least one of them. This is // acceptable since we still check the bitmaps precisely.
Why do we have to do this? Won't the require.Equal check fail in the previous version of this code if maybeHasNulls is different?
pkg/sql/colexec/routers.go, line 115 at r1 (raw file):
// zeroBatchAdded indicates whether zero-length batch was added to this // output (meaning that no more batches will be added). zeroBatchAdded bool
Instead of adding a boolean, I think it's preferable to add this as a state or use one of the states to indicate this.
pkg/sql/colexec/routers.go, line 188 at r1 (raw file):
// alwaysFlush, if set to true, will always flush o.mu.pendingBatch to // o.mu.data. alwaysFlush bool
Do we need to add this to the spilling queue? I think it gives us good testing coverage of what happens when we spill in the routers.
pkg/sql/colexec/routers.go, line 319 at r1 (raw file):
// writing rows to a fast output if we have to write to disk for a single // slow output. func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch) bool {
Why is the selection argument being removed? It means we have to do extra copies to set the desired selection vector on a batch. I think the way I handled this is to add an optional selection argument to the spilling queue, which will use the batch's selection vector if the argument is unset. I prefer that, but I'm also biased, so let's discuss
pkg/sql/colexec/spilling_queue.go, line 180 at r1 (raw file):
q.unlimitedAllocator.PerformOperation(q.diskQueueDeselectionScratch.ColVecs(), func() { for i := range q.typs { q.diskQueueDeselectionScratch.ColVec(i).Copy(
This looks like when we only have in-memory items, we will deselect into a scratch batch, then additionally append this batch into a tailBatch
. We should merge these two steps into one, i.e. deselect and append at the same time.
0c57b12
to
4aaf068
Compare
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.
The benchmarks are very good (this is somewhat surprising to me, tbh).
I also prototyped keeping a separate selection vector as you suggested below, and the comparison shows pretty small improvement when avoiding the copying of the selection onto the selection vector of the batch. To me this minor perf improvement doesn't seem worth the confusion of having separate selection things.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @asubiotto)
pkg/col/coldata/testutils.go, line 63 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Why do we have to do this? Won't the require.Equal check fail in the previous version of this code if maybeHasNulls is different?
Without this override, previously it would fail when it shouldn't. Consider two nulls vectors without any nulls set (i.e. all elements are valid), the bitmaps are bunch of 0xFF
, the only difference between two vectors is that maybeHasNulls
is set to different values - such vectors should be treated as equal by this check, but the check would fail. This came up because of the way we track all input tuples in TestSpillingQueue
- we create a window into a huge batch, and although there might not be any null values within the window, if there are some in the whole vector (outside of the window), maybeHasNulls
is still set to true
.
pkg/sql/colexec/routers.go, line 115 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Instead of adding a boolean, I think it's preferable to add this as a state or use one of the states to indicate this.
Done.
pkg/sql/colexec/routers.go, line 188 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Do we need to add this to the spilling queue? I think it gives us good testing coverage of what happens when we spill in the routers.
I think you might be confused on the meaning of alwaysFlush
on master - it disables the coalescing behavior in the router output. So are you wondering whether it is worth adding this knob to the spilling queue (to disable the newly-added coalescing behavior)? I'm not sure, it doesn't seem useful to me if we're always running with the coalescing behavior present in the production. Or am I missing something?
pkg/sql/colexec/routers.go, line 319 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Why is the selection argument being removed? It means we have to do extra copies to set the desired selection vector on a batch. I think the way I handled this is to add an optional selection argument to the spilling queue, which will use the batch's selection vector if the argument is unset. I prefer that, but I'm also biased, so let's discuss
Personally, I don't like having separate selection since all batches might already have the selection vector set, so it could be quite confusing - which one is it? Are all tuples in separate selection also selected according to the selection vector on the batch? Should we check that case?
I find it a lot cleaner to set the desired selection on the batch explicitly.
I agree that performance should be taken into the consideration, and I'll run a benchmark to see whether it is a noticeable hit, but my guess is that it won't be.
Update: benchmarks are in the main thread.
pkg/sql/colexec/spilling_queue.go, line 180 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
This looks like when we only have in-memory items, we will deselect into a scratch batch, then additionally append this batch into a
tailBatch
. We should merge these two steps into one, i.e. deselect and append at the same time.
Note that this code blocks applies only if we're spilling the batch to disk. In the code that deals with the in-memory batches the deselection is done during the append.
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.
Reviewed 3 of 10 files at r1, 1 of 1 files at r2.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @asubiotto and @yuzefovich)
pkg/sql/colexec/routers.go, line 188 at r1 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
I think you might be confused on the meaning of
alwaysFlush
on master - it disables the coalescing behavior in the router output. So are you wondering whether it is worth adding this knob to the spilling queue (to disable the newly-added coalescing behavior)? I'm not sure, it doesn't seem useful to me if we're always running with the coalescing behavior present in the production. Or am I missing something?
It's useful to test cases in which we actually spill to disk (even though it's writing to an in-memory file system) in unit tests without having to write a bunch of data. I still believe it's useful for router tests and might be so for other uses of the spilling queue as well now that in-memory buffering is delegated to the queue.
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.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @asubiotto)
pkg/sql/colexec/routers.go, line 188 at r1 (raw file):
It's useful to test cases in which we actually spill to disk (even though it's writing to an in-memory file system) in unit tests without having to write a bunch of data. I still believe it's useful for router tests and might be so for other uses of the spilling queue as well now that in-memory buffering is delegated to the queue.
Note that there is no coalescing behavior going on when the batch is enqueued to the disk queue, so old alwaysFlush
knob only disables the coalescing behavior when the batch is kept in the in-memory buffer. Currently, this knob is used only in TestHashRouterOneOutput
unit test, and with the knob removed we spill to disk the same number of times as with the knob present.
I have thought some more about the need for this knob, and I still don't see it. Can you come up with a concrete example when disabling the coalescing behavior of the in-memory buffer provides us new test coverage that we don't already get otherwise?
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.
Reviewed 3 of 10 files at r1, 1 of 1 files at r2.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @asubiotto and @yuzefovich)
pkg/sql/colexec/routers.go, line 188 at r1 (raw file):
Sorry, apparently didn't end up sending this comment:
so old alwaysFlush knob only disables the coalescing behavior when the batch is kept in the in-memory buffer.
But now we have coalescing behavior in the spilling queue itself, right?
It's useful to test cases in which we actually spill to disk (even though it's writing to an in-memory file system) in unit tests without having to write a bunch of data. I still believe it's useful for router tests and might be so for other uses of the spilling queue as well now that in-memory buffering is delegated to the queue.
Maybe a more concrete example of what I'm thinking will be more productive:
Assume you write tuple A to the spilling queue and then tuple B. With the spilling queue's in-memory coalescing tuple A and tuple B are never Enqueue
d to the underlying disk queue. However, with alwaysFlush
, both of them are. I guess technically the disk queue will do its own in-memory buffering, but having this alwaysFlush
testing knob allows us to test code paths that Enqueue
and Dequeue
from the disk queue.
4aaf068
to
118f2fc
Compare
Added a knob to limit the number of batches added to the in-memory buffer as we discussed. |
98209de
to
a7055f2
Compare
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.
Reviewed 6 of 6 files at r3.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @asubiotto and @yuzefovich)
pkg/sql/colexec/spilling_queue.go, line 76 at r3 (raw file):
// numEnqueues tracks the number of times enqueue() has been called with // non-zero batch. numEnqueues int
Nit: This might be a bit specific. Consider making it a testing callback (onEnqueue
) that can optionally be set.
This commit refactors `enqueue` method of the spilling queue to deep-copy the passed-in batches if they are kept in memory. Previous behavior was suboptimal because it was forcing the caller to always allocate a new batch. Additionally, the spilling queue will now perform a coalescing step by attempting to append as many tuples to the tail in-memory batch as possible. The in-memory batches are allocated with dynamically increasing capacity. This allows us to significantly simplify the code of the router outputs which were performing the coalescing step previously. Additionally, this commit fixes a couple of uses of `enqueue` method (the router outputs and the merge joiner) in which they forgot to enqueue a zero-length batch which is necessary when the disk queue is initialized. Release note: None
a7055f2
to
d92d7c8
Compare
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.
TFTR!
bors r+
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @asubiotto)
pkg/sql/colexec/routers.go, line 188 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Sorry, apparently didn't end up sending this comment:
so old alwaysFlush knob only disables the coalescing behavior when the batch is kept in the in-memory buffer.
But now we have coalescing behavior in the spilling queue itself, right?
It's useful to test cases in which we actually spill to disk (even though it's writing to an in-memory file system) in unit tests without having to write a bunch of data. I still believe it's useful for router tests and might be so for other uses of the spilling queue as well now that in-memory buffering is delegated to the queue.
Maybe a more concrete example of what I'm thinking will be more productive:
Assume you write tuple A to the spilling queue and then tuple B. With the spilling queue's in-memory coalescing tuple A and tuple B are never
Enqueue
d to the underlying disk queue. However, withalwaysFlush
, both of them are. I guess technically the disk queue will do its own in-memory buffering, but having thisalwaysFlush
testing knob allows us to test code paths thatEnqueue
andDequeue
from the disk queue.
Done.
pkg/sql/colexec/spilling_queue.go, line 76 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Nit: This might be a bit specific. Consider making it a testing callback (
onEnqueue
) that can optionally be set.
Done.
Build failed (retrying...): |
Need to fix the test. bors r- |
Canceled. |
d92d7c8
to
e2d602e
Compare
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.
bors r+
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @asubiotto)
pkg/sql/colexec/spilling_queue.go, line 76 at r3 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
Done.
I reverted the change to the original version because I don't see a clean way of forcing the spilling queue to use the disk use from inside of the callback (simply calling maybeSpillToDisk
doesn't work). I think it is reasonable because in case we need to have a more general callback, we can introduce it then.
Build succeeded: |
This commit refactors
enqueue
method of the spilling queue todeep-copy the passed-in batches if they are kept in memory. Previous
behavior was suboptimal because it was forcing the caller to always
allocate a new batch. Additionally, the spilling queue will now perform
a coalescing step by attempting to append as many tuples to the tail
in-memory batch as possible. The in-memory batches are allocated with
dynamically increasing capacity.
This allows us to significantly simplify the code of the router outputs
which were performing the coalescing step previously.
Additionally, this commit fixes a couple of uses of
enqueue
method(the router outputs and the merge joiner) in which they forgot to
enqueue a zero-length batch which is necessary when the disk queue is
initialized.
Fixes: #47062.
Release note: None