-
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
colexecwindow: fix disk spilling in some cases #74491
Conversation
This commit fixes a possible data corruption (which would either result in a silent wrong query result or an internal error) scenario that could occur when the data is serialized/deserialized in the vectorized engine. This would occur when the deserialized vectors are appended to, and it was the most likely to occur with Bytes-like types (because their `Set`s can behave like appends to a certain degree). We need to deserialize the data in two paths - in the inbox after reading from network and during the disk spilling. I believe that the former is safe (since we don't modify those batches) and the latter is mostly safe (since we tend to not modify the batches that we read from disk). I think the only exception is the window functions. Consider the following scenario: a batch with two Bytes vectors is serialized. Say - the first vector is `{data:[foo], offsets:[0, 3]}` - the second vector is `{data:[bar], offsets:[0, 3]}`. After serializing both of them we will have a flat buffer with something like: `buf = {1foo031bar03}` (ones represent the lengths of each vector). Now, when the first vector is being deserialized, it's data slice will be something like: `data` = `[foo031bar03]`, `len(data) = 3`, `cap(data) > 3`. If we don't explicitly cap the slice and deserialize it into a Bytes vector, then later when we append to that vector, we will overwrite the data that is actually a part of the second serialized vector, thus, corrupting it (or the next batch). Release note (bug fix): Previously, CockroachDB could return incorrect results or internal errors on queries with window functions returning INT, FLOAT, BYTES, STRING, UUID, or JSON type when the disk spilling occurred. The bug was introduced in 21.2.0 and is now fixed.
18a0143
to
9570486
Compare
`bufferedWindowOp` is special in a sense that its output vector is appended by the `vectorTypeEnforcer` that is its input operator. That output vector, thus, is part of the input batch, and the vector is updated in incremental fashion (as the results are ready). It is also possible that the input batch needs to be spilled to disk for the operator to make progress. Previously, the output vector could be in an invalid state (if it was bytes-like) because less elements were set on the vector than the length of the batch. This is now fixed by making the output vector valid before spilling. Release note: None (because the previous commit contains very similar info).
This commit fixes a problem with MIN/MAX optimized window functions in the vectorized engine. The problem was that we forgot to make an explicit copy of the value before rewinding the spilling buffer (which can make the previously retrieved value invalid). Release note (bug fix): CockroachDB could previously incorrectly calculate MIN/MAX when used as window functions in some cases after spilling to disk. The bug was introduced in 21.2.0 and is now fixed.
Release note: 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.
Reviewed 2 of 2 files at r1, 1 of 1 files at r2, 3 of 3 files at r3, 1 of 1 files at r4, all commit messages.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @cucaroach and @yuzefovich)
pkg/sql/colexec/colexecwindow/buffered_window.go, line 248 at r2 (raw file):
// TODO(yuzefovich): it is quite unfortunate that the output // vector is being spilled to disk. Consider refactoring this. switch b.outputColFam {
Does the test in commit 4 demonstrate that this works as intended, or is another test needed?
pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go, line 284 at r3 (raw file):
} // Any values that could not fit in the queue would also have been // dominated by the current one, so reset omittedIndex.
For my education, what does dominated mean in this context?
pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go, line 275 at r4 (raw file):
// calls below might reuse the same underlying vector. var valCopy _GOTYPE execgen.COPYVAL(valCopy, val)
Does the test in commit 4 demonstrate that this works as intended, or is another test needed?
pkg/sql/distsql/columnar_operators_test.go, line 1224 at r4 (raw file):
fmt.Printf("argIdxs: %v\n", argsIdxs) frame := windowerSpec.WindowFns[0].Frame fmt.Printf("frame mode: %v\n", frame.Mode)
nit: seems like these things don't change, so the print statements could be lifted out of the for loop.
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! 1 of 0 LGTMs obtained (waiting on @cucaroach and @rharding6373)
pkg/sql/colexec/colexecwindow/buffered_window.go, line 248 at r2 (raw file):
Previously, rharding6373 (Rachael Harding) wrote…
Does the test in commit 4 demonstrate that this works as intended, or is another test needed?
Yeah, the test in commit 4 was able to find all of the bugs fixed in the first three commits, so I think that test is sufficient.
pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go, line 284 at r3 (raw file):
Previously, rharding6373 (Rachael Harding) wrote…
For my education, what does dominated mean in this context?
Here is an example: imagine we have the following vector {1, 3, 5, 2}
and say we're computing min
window function with the frame as ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING
(which means that we're computing min
over the current value, value right before it if present, and value right after it if present).
Here is how we'll compute min
using the "removable" strategy. We're maintaining a priority queue of values that are currently within the frame (where we're keeping track of the value as well as the index - the index is needed so that we could remove it when the corresponding value goes outside of the window frame). Values in min
queue are increasing in their value and in their index.
- computing
min
foridx = 0
:- queue is empty, values
1
and3
enter the queue (because they just entered the window frame), so we now havequeue = (1, 0), (3, 1)
.min
is1
.
- queue is empty, values
min
foridx = 1
:- window frame is
idx = {0, 1, 2}
, so nothing needs to be removed from the queue,5
enters the queue, so we havequeue = (1, 0), (3, 1), (5, 2)
.min
is1
.
- window frame is
min
foridx = 2
:- window frame is
idx = {1, 2, 3}
, soidx = 0
has just exited from the frame and we have to remote if from the queue, nowqueue = (3, 1), (5, 2)
.2
just entered the frame, but it dominates both values in the queue (because it has higher priority - lower value in this case - and higher index - so it'll stay inside the window frame for longer), nowqueue = (2, 3)
.min
is2
.
- window frame is
min
foridx = 3
:- window frame is
idx = {2, 3}
, nothing needs to be removed from the queue and nothing is added, somin
is2
, and we're done.
- window frame is
Hope this makes sense.
pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go, line 275 at r4 (raw file):
Previously, rharding6373 (Rachael Harding) wrote…
Does the test in commit 4 demonstrate that this works as intended, or is another test needed?
Same as above, I think test in the commit 4 is sufficient.
pkg/sql/distsql/columnar_operators_test.go, line 1224 at r4 (raw file):
Previously, rharding6373 (Rachael Harding) wrote…
nit: seems like these things don't change, so the print statements could be lifted out of the for loop.
Note that this printing out happens only in case an unexpected error is encountered and we call t.Fatal
below. We do not log all this information in the happy path, so I think the layout is as desirable.
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! 1 of 0 LGTMs obtained (waiting on @cucaroach and @rharding6373)
pkg/sql/colexec/colexecwindow/buffered_window.go, line 248 at r2 (raw file):
Previously, yuzefovich (Yahor Yuzefovich) wrote…
Yeah, the test in commit 4 was able to find all of the bugs fixed in the first three commits, so I think that test is sufficient.
BTW I stressed that test overnight, and it didn't find any problems with over 50k runs, so I have high confidence that we have flushed out all the edge cases.
Build succeeded: |
colserde: fix possible data corruption scenario during disk spilling
This commit fixes a possible data corruption (which would either result
in a silent wrong query result or an internal error) scenario that could
occur when the data is serialized/deserialized in the vectorized engine.
This would occur when the deserialized vectors are appended to, and it
was the most likely to occur with Bytes-like types (because their
Set
scan behave like appends to a certain degree).
We need to deserialize the data in two paths - in the inbox after
reading from network and during the disk spilling. I believe that the
former is safe (since we don't modify those batches) and the latter is
mostly safe (since we tend to not modify the batches that we read from
disk). I think the only exception is the window functions.
Consider the following scenario: a batch with two Bytes vectors is
serialized. Say
{data:[foo], offsets:[0, 3]}
{data:[bar], offsets:[0, 3]}
.After serializing both of them we will have a flat buffer with something
like:
buf = {1foo031bar03}
(ones represent the lengths of each vector).Now, when the first vector is being deserialized, it's data slice will be
something like:
data
=[foo031bar03]
,len(data) = 3
,cap(data) > 3
.If we don't explicitly cap the slice and deserialize it into a Bytes
vector, then later when we append to that vector, we will overwrite the
data that is actually a part of the second serialized vector, thus,
corrupting it (or the next batch).
Release note (bug fix): Previously, CockroachDB could return incorrect
results or internal errors on queries with window functions returning
INT, FLOAT, BYTES, STRING, UUID, or JSON type when the disk spilling
occurred. The bug was introduced in 21.2.0 and is now fixed.
colexecwindow: make bytes-like output vector valid before spilling
bufferedWindowOp
is special in a sense that its output vector isappended by the
vectorTypeEnforcer
that is its input operator. Thatoutput vector, thus, is part of the input batch, and the vector is
updated in incremental fashion (as the results are ready). It is also
possible that the input batch needs to be spilled to disk for the
operator to make progress. Previously, the output vector could be in an
invalid state (if it was bytes-like) because less elements were set on
the vector than the length of the batch. This is now fixed by making the
output vector valid before spilling.
Fixes: #70715.
Release note: None (because the previous commit contains very similar
info).
colexecwindow: fix min/max optimized window functions
This commit fixes a problem with MIN/MAX optimized window functions in
the vectorized engine. The problem was that we forgot to make an
explicit copy of the value before rewinding the spilling buffer (which
can make the previously retrieved value invalid).
Fixes: #74476.
Release note (bug fix): CockroachDB could previously incorrectly
calculate MIN/MAX when used as window functions in some cases after
spilling to disk. The bug was introduced in 21.2.0 and is now fixed.
distsql: force disk spilling in TestWindowFunctionsAgainstProcessor
Release note: None