Skip to content
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 premature calls to IdempotentClose in hash router outputs #49333

Merged
merged 1 commit into from
May 28, 2020

Conversation

yuzefovich
Copy link
Member

Previously, we were passing in toClose slice of idempotent closers to
every outbox that is the output of a hash router. However, it is
possible that one stream will be closed before others, and this would
prompt the corresponding outbox to close all of the closers prematurely.
Other streams might still be active and ready to consume more data, but
that single outbox would close everything. This is now fixed by making
hash router responsible for closing the whole toClose slice, and it
will do so right before exiting Run method.

Fixes: #49315.

Release note (bug fix): Previously, CockroachDB could return an internal
error or incorrect results on queries when they were run via the
vectorized execution engine and had a hash router in the DistSQL plan.
This could only occur with vectorize=on.

@yuzefovich yuzefovich requested review from asubiotto and a team May 20, 2020 17:44
@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Contributor

@asubiotto asubiotto left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As we discussed, outboxes only close the closers after a zero-batch is received. If a hashrouter is the input, this will only happen once all streams have received all batches, so I don't think this patch fixes the linked issue.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @asubiotto)

@yuzefovich
Copy link
Member Author

yuzefovich commented May 26, 2020

I added some print statements, and it appears that outbox 0xc00635aa00 is calling (*mergeJoinBase).IdempotentClose too early. Here are the logs:

created hash router 0xc00567dc20 with outputs [0xc00635a820 0xc00635a8c0 0xc00635a960]
created outbox 0xc00635aa00 with input 0xc00635a8c0
...
outbox 0xc00635aa00 is being closed
mergeJoinBase 0xc0062a2800 is being closed
...
routerOutputOp 0xc00635a8c0 received zero-length batch
...
hash router 0xc00567dc20 is done

I'll continue on looking. (Note this is the behavior on master, without this patch with some extra printf statements.)

@yuzefovich
Copy link
Member Author

Ok, I think I understand what's going on, and I believe my initial patch correctly fixes the issue (a side note: making DrainMeta calls only from the same goroutine as Next maybe would also resolve the issue).

Screen Shot 2020-05-26 at 2 45 39 PM

We have this plan, and MergeJoiner/4 is the one that is hitting the nil pointer because it is closed prematurely. Consider the following scenario:

  1. HashJoiner/10 consumes the right input which is empty, so it propagates zero batch downstream
  2. outbox that is the output of Sorter/19 gets that zero batch and terminates gracefully
  3. as part of the termination, that outbox will drain all metadataSources
  4. one of those sources is an inbox that is one of three inputs to unordered synchronizer which is the left input to HashJoiner/10
  5. that inbox is connected to the outbox that is one of the outputs of the hash router on MergeJoiner/4, and the inbox sends drain signal to the outbox
  6. that outbox gets the drain signal, so it terminates itself
  7. as part of that termination it calls IdempotentClose on all things in toClose slice it was passed in in the constructor
  8. one of the things in toClose is the merge joiner, so it gets closed.

And this is where the problem is - the output of MergeJoiner/4 is the hash router which has three outputs, and only one of the router outputs is closed while two others can still be active. We should be shutting down the merge joiner only when the hash router is getting drained (meaning all of its three outputs are getting drained as well), and this PR fixes this.

Previously, we were passing in `toClose` slice of idempotent closers to
every outbox that is the output of a hash router. However, it is
possible that one stream will be closed before others, and this would
prompt the corresponding outbox to close all of the closers prematurely.
Other streams might still be active and ready to consume more data, but
that single outbox would close everything. This is now fixed by making
hash router responsible for closing the whole `toClose` slice, and it
will do so right before exiting `Run` method.

Release note (bug fix): Previously, CockroachDB could return an internal
error or incorrect results on queries when they were run via the
vectorized execution engine and had a hash router in the DistSQL plan.
This could only occur with `vectorize=on`.
Copy link
Contributor

@asubiotto asubiotto left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for digging deeper into this! It makes sense.

:lgtm:

Reviewed 1 of 4 files at r1.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @yuzefovich)


pkg/sql/colflow/vectorized_flow.go, line 671 at r1 (raw file):

			// router is responsible for closing all of the idempotent closers.
			if _, err := s.setupRemoteOutputStream(
				ctx, flowCtx, op, outputTyps, stream, metadataSourcesQueue, nil /* toClose */, factory,

Hmm, were we previously passing in toClose to multiple outboxes? I guess it doesn't matter since they are idempotent closers.

Copy link
Member Author

@yuzefovich yuzefovich left a 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: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @asubiotto)


pkg/sql/colflow/vectorized_flow.go, line 671 at r1 (raw file):

Previously, asubiotto (Alfonso Subiotto Marqués) wrote…

Hmm, were we previously passing in toClose to multiple outboxes? I guess it doesn't matter since they are idempotent closers.

Exactly, we were passing full toClose to all outboxes, so the first one to shutdown would initiate the closure of toClose things and others would initiate a noop.

@craig
Copy link
Contributor

craig bot commented May 28, 2020

Build failed (retrying...)

@craig
Copy link
Contributor

craig bot commented May 28, 2020

Build succeeded

@craig craig bot merged commit 1019064 into cockroachdb:master May 28, 2020
@yuzefovich yuzefovich deleted the fix-router-close branch May 28, 2020 02:45
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

roachtest: tpcdsvec failed
3 participants