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

sql: disable gossip-based physical planning by default #135034

Open
wants to merge 1 commit into
base: master
Choose a base branch
from

Conversation

yuzefovich
Copy link
Member

@yuzefovich yuzefovich commented Nov 12, 2024

Now that we improved handling of draining procedure in the virtual clusters in 24.3 timeframe, I believe it's time to begin fully deprecating the gossip-based physical planning, and as the first step this commit introduces a cluster setting to control which method is used in single-tenant deployments (instance-based planning being the default). I plan to have the setting as an escape hatch in case we find problems when rolling out this change and to remove the setting altogether after 25.2.

One of the differences between two planning methods is slightly adjusted in this commit. In particular, in the instance-based planning we stitch together consecutive spans whenever EnsureSafeSplitKey for the keys in the spans returns the same prefix. The idea is that spans corresponding to different column families of the same SQL row must be placed on the same instance. However, the condition on the length of the keys returned by EnsureSafeSplitKey is too broad and captures more cases than needed (i.e. two keys that cannot be part of the same SQL row end up in the same SpanPartition). To partially alleviate this difference, this commit introduces "boundary granularity" knob which indicates whether consecutive spans might be part of the same SQL row, and we now use the stitching logic only if so. All callers of PartitionSpans have been audited, and we need the stitching logic only in two call sites that correspond to planning of table readers. Longer term, if we change the encoding so that column family IDs are encoded in a special way, we'll be able to tell definitively whether stitching is needed simply based on the key itself, and this additional logic could be removed.

Epic: None

Release note: None

Copy link

blathers-crl bot commented Nov 12, 2024

It looks like your PR touches production code but doesn't add or edit any test code. Did you consider adding tests to your PR?

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

Now that we improved handling of draining procedure in the virtual
clusters in 24.3 timeframe, I believe it's time to begin fully
deprecating the gossip-based physical planning, and as the first step
this commit introduces a cluster setting to control which method is used
in single-tenant deployments (instance-based planning being the
default). I plan to have the setting as an escape hatch in case we find
problems when rolling out this change and to remove the setting
altogether after 25.2.

One of the differences between two planning methods is slightly adjusted
in this commit. In particular, in the instance-based planning we stitch
together consecutive spans whenever `EnsureSafeSplitKey` for the keys in
the spans returns the same prefix. The idea is that spans corresponding
to different column families of the same SQL row must be placed on the
same instance. However, the condition on the length of the keys returned
by `EnsureSafeSplitKey` is too broad and captures more cases than
needed (i.e. two keys that cannot be part of the same SQL row end up in
the same `SpanPartition`). To partially alleviate this difference, this
commit introduces "boundary granularity" knob which indicates whether
consecutive spans _might_ be part of the same SQL row, and we now use
the stitching logic only if so. All callers of `PartitionSpans` have
been audited, and we need the stitching logic only in two callsites that
correspond to planning of table readers. Longer term, if we change the
encoding so that column family IDs are encoded in a special way, we'll
be able to tell definitively whether stitching is needed simply based on
the key itself, and this additional logic could be removed.

Epic: None

Release note: None
@yuzefovich yuzefovich marked this pull request as ready for review November 23, 2024 02:58
@yuzefovich yuzefovich requested review from a team as code owners November 23, 2024 02:58
@yuzefovich yuzefovich requested review from jeffswenson and wenyihu6 and removed request for a team, jeffswenson and wenyihu6 November 23, 2024 02:58
Copy link
Collaborator

@DrewKimball DrewKimball left a comment

Choose a reason for hiding this comment

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

:lgtm: Nice!

Reviewed 9 of 9 files at r1, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @michae2 and @yuzefovich)


pkg/sql/fingerprint_span.go line 121 at r1 (raw file):

	}

	spanPartitions, err := dsp.PartitionSpans(ctx, planCtx, []roachpb.Span{span}, PartitionSpansBoundDefault)

Note: we can reach here through the builtin function crdb_internal.fingerprint()


pkg/sql/distsql_physical_planner.go line 1567 at r1 (raw file):

			// address #112702, we'll be able to make the check here precise.
			if safeKey, err := keys.EnsureSafeSplitKey(span.Key); err == nil && len(safeKey) > 0 {
				if safeKey.Equal(lastKey) {

I know this is how it was already, but why don't we compare the key against the last end key (if it's set) instead? Won't this only catch the case when the preceding span covers a single row?


pkg/sql/distsql_physical_planner.go line 1569 at r1 (raw file):

				if safeKey.Equal(lastKey) {
					if log.V(1) {
						log.Infof(ctx, "stitchin span %s into the previous span partition", span)

[nit] stitchin -> stitching

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.

3 participants