-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
sql: properly synchronize the internal executor iterator #62581
Conversation
Andrew, this solution almost works, but it doesn't work well with the vectorized engine where we use the context cancellation to shut everything down. For one of the queries mentioned in the issue we have the following plan:
When performing hash join on the empty cluster, the right side will be empty, and as a result, we will short-circuit the execution of the whole query without reading anything from the left side of the join. The whole query then shuts down which cancels the context. However, the left side has a connExecutor goroutine running the virtual table, and it will still proceed to fully execute the read, but that read will encounter a context cancellation error at the distSender level. Which makes the whole txn to be in a non-retriable state, and the whole query errors out. I'm not quite sure what to do here.
I'm curious whether you have some insights here. |
Huh, if the right side is empty than the query should just return no rows, right? I don't think I've fully digested what you're saying. When does the context plumbed into the virtual table get canceled? |
Yes, the query will return no rows, but the way the cleanup of the virtual table on the left currently works is that the left scan will still proceed until the very end because we don't have a way to stop the connExecutor from executing the query to completion. I originally removed some parts of the plan which probably made it confusing, so I put them back in. The table is not empty itself, everything is filtered out. I'm currently trying to plumb the shutdown of query execution of the left side when the iterator for it is closed. We'll see whether the hack at least works. |
Sorry, didn't answer your question - the context is canceled right before returning from the main query. The main query is ready to return no rows, but during the shutdown of the vectorized infrastructure, we cancel the context of the vectorized flow; then, after the vectorized infra is cleaned up, we call |
Just added another commit as a prototype, but it didn't work :/ Possibly I screwed something up though. |
Hm, possibly my understanding of the situation is wrong too because the error ( Oh, while typing it out I realized that I might see the problem - |
Yeah, that was it, apologies for panicking too early. At least the couple of queries in the issue now work. I'll polish up the PR tomorrow for review. However, I'm still concerned about the scenario I typed up above about the way the vectorized flows are shutdown. Possibly we might never actually run into such case, but I don't know all the details of how context cancellation might affect things. One example that comes to my mind is with the changefeeds (some details here). |
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'm not concerned about changefeeds and the internal executor. We could add some defensive checks to not execute sinkless changefeeds on an internal executor but I doubt it will happen.
This wasn't too bad
Reviewed 2 of 2 files at r1.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @yuzefovich)
pkg/sql/conn_executor_internal_test.go, line 305 at r2 (raw file):
}, dataCh: dataCh, waitCh: waitCh,
just pass a closed waitCh?
pkg/sql/internal.go, line 294 at r2 (raw file):
}() // TODO(yuzefovich): de-duplicate this with the code below.
I believe in your ability to make a closure
pkg/sql/internal.go, line 811 at r2 (raw file):
// the first object is sent on the data channel. first, ok := <-dataCh if ok && first.cols != nil {
can you explain this part to me?
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 mentioned changefeeds as an example of when the way of shutting down the vectorized flows (by context cancellation) interferes with some other features. (I didn't mean that the work we're doing here will affect the changefeeds.) In general I'm still concerned about the correctness.
Also, the CI seems to be unhappy, and in particular TestShowJobWhenComplete/show_jobs
currently fails easily (often, but not every time). I wonder whether you have some ideas of what we're missing because I don't see what we're doing wrong :/
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner)
pkg/sql/conn_executor_internal_test.go, line 305 at r2 (raw file):
Previously, ajwerner wrote…
just pass a closed waitCh?
Good point, done.
pkg/sql/internal.go, line 294 at r2 (raw file):
Previously, ajwerner wrote…
I believe in your ability to make a closure
Thanks! I wanted to see whether CI was happy :) it wasn't :(
pkg/sql/internal.go, line 811 at r2 (raw file):
Previously, ajwerner wrote…
can you explain this part to me?
This was the last piece that I added to make a couple of queries from the issue happy.
I just added a more explicit comment about what's going on here. Let me know if more info is worth it.
8128ee2
to
66d9f8d
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.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner)
pkg/sql/internal.go, line 733 at r4 (raw file):
// skipped. dataCh <- ieIteratorResult{err: res.Err()} <-waitCh
Note that some of these waits are not necessary, but I think it might be a good idea to follow the same pattern (of "send on dataCh, block on waitCh") everywhere.
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.
Also, the CI seems to be unhappy, and in particular TestShowJobWhenComplete/show_jobs
This seems better now, no? Is this RFAL?
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner)
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 think the failures are non-deterministic, but last time I looked TestShowJobWhenComplete/show_jobs
(among other things) was failing pretty reliably. I don't see where the problem might be, so any suggestions are welcome. I'll look into debugging this more later today.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner)
Hm, the CI I kicked off this morning succeeded, and now I can no longer repro manually Anyway, I think it is RFAL then. |
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.
Generally I'm concerned about the need for selects and detecting context cancellation on the two channels. Can you convince me it's not a problem?
Reviewed 1 of 4 files at r4.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @yuzefovich)
pkg/sql/internal.go, line 811 at r4 (raw file):
// the next piece from the data channel. // // Note that only only statements of ROWS type should send the cols, but
only only
pkg/sql/internal.go, line 816 at r4 (raw file):
resultColumns = first.cols } waitCh <- struct{}{}
does this need to be done in a select? If the context underneath is canceled, then it seems possible that nobody will receive this.
84f388f
to
f69b85b
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.
I believe your concern is totally on point, and I addressed it. I'm feeling optimistic that now we have the correct fix in place.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner)
pkg/sql/internal.go, line 816 at r4 (raw file):
Previously, ajwerner wrote…
does this need to be done in a select? If the context underneath is canceled, then it seems possible that nobody will receive this.
That's a very good point. I think you are totally right, and this is the thing that I missed.
I updated the code so that the reader always listens for context cancellation when operating with the channels. We don't need to do that on the writer (connExecutor) side because the reader will properly drain/close the channels in rowsIterator.Close
(Close
must be called whenever an iterator is returned by the internal executor).
c497c1a
to
60ef3ee
Compare
Ok, fixed some comments and a minor issue. PTAL. |
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 added two more commits which I speculate fix the problems. See what you think.
For what it's worth, I'm not super pleased with them. I'm thinking we should refactor some of this synchronization into an abstraction. Also the context management around here is not particularly principled.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @pbardea)
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 cleaned it up. I'm happier with it now.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @ajwerner and @pbardea)
I was missing the crucial observation you added to the commit message. I think now the requirement of listening on the ctx cancellation on the writer side makes sense. I squashed the two larger commits into one with some cosmetic modifications. The notable changes are moving the logic of draining Thanks a lot for baby-sitting this PR. PTAL. |
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 5 of 9 files at r5, 6 of 6 files at r6.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @yuzefovich)
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.
Actually, I feel like we can make this a tad better.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @yuzefovich)
pkg/sql/internal_result_channel.go, line 39 at r6 (raw file):
// is non-nil. Err will be non-nil if either close has been called or // the passed context is finished. nextResult(ctx context.Context) (_ ieIteratorResult, done bool, err error)
There's something that still smells a little bit about this interface. The semantics of done and err are a bit off. The semantics of close feel off too. I;m alarmed by the way that close on the async channel consumes everything and propagates errors from the sender while close on the sync channel just effectively no-ops. It's probably fine but I'm having a hard time reasoning about what we want here. Are we trying to wait for the writer or preempt it? It feels like the later as on the happy path we should always see done from nextResult
. I'm inclined to make the close behavior much more uniform between the two impls here.
`SetColumns` contract allows for the argument to be nil, yet the iterator of the streaming internal executor expects that column schema, if set, is non-nil. I don't think this could happen in practice, but theoretically previously we could encounter an assertion error due to none of the four fields in `ieIteratorResult` object being non-nil, and now this is fixed. Release note: None
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.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner)
pkg/sql/internal_result_channel.go, line 39 at r6 (raw file):
Previously, ajwerner wrote…
There's something that still smells a little bit about this interface. The semantics of done and err are a bit off. The semantics of close feel off too. I;m alarmed by the way that close on the async channel consumes everything and propagates errors from the sender while close on the sync channel just effectively no-ops. It's probably fine but I'm having a hard time reasoning about what we want here. Are we trying to wait for the writer or preempt it? It feels like the later as on the happy path we should always see done from
nextResult
. I'm inclined to make the close behavior much more uniform between the two impls here.
Updated the usage of context.Canceled
error per our discussion.
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 @ajwerner)
We recently merged an update to the internal executor to make it streaming. Currently it is implemented by having two goroutines (the iterator, "the consumer"; and the connExecutor, "the producer"). The communication between them is done on the buffered channel. As a result, both goroutines can run concurrently. However, a crucial point was overlooked - our `kv.Txn`s cannot be used concurrently. Imagine a plan that reads from two tables each of which is populated via the internal executor: if we read from the first, and then from the second concurrently, we will have the concurrent usage of the txn for that plan. This commit the problem by carving out a new abstraction to optionally synchronize the execution of the internal executor with its corresponding iterator. The abstraction comes in sync and async flavors. In the sync form, the ieResultChannel ensures that the reader and writer do not concurrently operate, and, additionally provides a mechanism whereby the reader may ensure that the writer observes an error when its attempts to publish the previous row returns. This last point is critical to ensure that transactions are not erroneously used after it has become unsafe. The async flavor is still used by the internal executor when it doesn't return an iterator directly and executes the query to completion itself. Release note: None (no stable release with this bug).
Thanks for the help with this PR! bors r+ |
Build succeeded: |
sql: fix an edge case in the internal executor
SetColumns
contract allows for the argument to be nil, yet theiterator of the streaming internal executor expects that column schema,
if set, is non-nil. I don't think this could happen in practice, but
theoretically previously we could encounter an assertion error due to
none of the four fields in
ieIteratorResult
object being non-nil, andnow this is fixed.
Release note: None
sql: add context to IncrementRowsAffected interface
Release note: None
sql: properly synchronize the internal executor iterator
We recently merged an update to the internal executor to make it
streaming. Currently it is implemented by having two goroutines (the
iterator, "the consumer"; and the connExecutor, "the producer"). The
communication between them is done on the buffered channel. As a result,
both goroutines can run concurrently.
However, a crucial point was overlooked - our
kv.Txn
s cannot be usedconcurrently. Imagine a plan that reads from two tables each of which is
populated via the internal executor: if we read from the first, and then
from the second concurrently, we will have the concurrent usage of the
txn for that plan.
This commit the problem by carving out a new abstraction to optionally
synchronize the execution of the internal executor with its corresponding
iterator. The abstraction comes in sync and async flavors. In the sync
form, the ieResultChannel ensures that the reader and writer do not
concurrently operate, and, additionally provides a mechanism whereby the
reader may ensure that the writer observes an error when its attempts to
publish the previous row returns. This last point is critical to ensure
that transactions are not erroneously used after it has become unsafe.
The async flavor is still used by the internal executor when it doesn't
return an iterator directly and executes the query to completion itself.
Fixes: #62415.
Release note: None (no stable release with this bug).