Skip to content

distsql: avoid planning on a draining node.#128283

Merged
craig[bot] merged 2 commits intocockroachdb:masterfrom
shubhamdhama:draining-use-in-distsql-100578
Sep 4, 2024
Merged

distsql: avoid planning on a draining node.#128283
craig[bot] merged 2 commits intocockroachdb:masterfrom
shubhamdhama:draining-use-in-distsql-100578

Conversation

@shubhamdhama
Copy link
Copy Markdown
Contributor

@shubhamdhama shubhamdhama commented Aug 4, 2024

Commit-wise description


Commit 1: sql: add is_draining to system.sql_instances table.

This is a preliminary commit toward improving the DistSQL to avoid planning
on an instance that is draining. The idea here is to set this column to
true as soon as the draining request is received by the server and use that
through instance reader in the planner at gateway.

Adding this column requires writing a migration for the upgrade. And since
this new column is nullable we don't need to backfill and we are good with
just swapping the descriptors.

Informs: #100578
Release note: None

Development notes:

To update pkg/sql/catalog/bootstrap/testdata/testdata I had to manually
update the hash values for system/tenant in this file and pass the
--rewrite option to do the rest:

dev test //pkg/sql/catalog/bootstrap:bootstrap_test \
        --filter=TestInitialValuesToString/testdata --rewrite

To update tests in
pkg/sql/catalog/systemschema_test/testdata/bootstrap_{system,tenant}

dev test \
        //pkg/sql/catalog/systemschema_test:systemschema_test_test --rewrite

Commit 2: distsql: avoid planning on a draining node.

This change would prevent distsql planner to include a draining node in a
plan by leveraging the system.sql_instances table's is_draining field.

Somewhat related work to avoid planning on an unavailable node: #124986

Fixes: #100578
Epic: CRDB-31575
Release note: None

@shubhamdhama shubhamdhama requested review from a team as code owners August 4, 2024 19:24
@shubhamdhama shubhamdhama requested review from dt and yuzefovich and removed request for a team August 4, 2024 19:25
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

pkg/sql/distsql_physical_planner_test.go line 832 at r2 (raw file):

			return checkQueryPlan(t, r, expectedPlans)
		}, 4*time.Second)

The test for external-process mode is failing because the plan still includes the drained 5th node. This happens because setting is_draining to true isn't being reflected on the gateway node. However, this works correctly on a roachprod cluster. Despite extensive investigation, I couldn't figure out why this is happening, so I would really appreciate any help.

Code quote:

		testutils.SucceedsWithin(t, func() error {
			// TODO: REMOVE
			res := r.QueryStr(t, "SELECT now(), id, encode(session_id, 'hex'), is_draining FROM system.sql_instances WHERE addr IS NOT NULL")
			t.Logf("res : %v", res)

			return checkQueryPlan(t, r, expectedPlans)
		}, 4*time.Second)

@rafiss rafiss removed the request for review from a team August 21, 2024 20:23
@shubhamdhama shubhamdhama force-pushed the draining-use-in-distsql-100578 branch from cc10bf1 to 661e2dc Compare August 26, 2024 09:46
Copy link
Copy Markdown
Contributor Author

@shubhamdhama shubhamdhama left a comment

Choose a reason for hiding this comment

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

This is also ready for review.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @dt, @jeffswenson, @stevendanna, and @yuzefovich)


pkg/sql/distsql_physical_planner_test.go line 832 at r2 (raw file):

Previously, shubhamdhama (Shubham Dhama) wrote…

The test for external-process mode is failing because the plan still includes the drained 5th node. This happens because setting is_draining to true isn't being reflected on the gateway node. However, this works correctly on a roachprod cluster. Despite extensive investigation, I couldn't figure out why this is happening, so I would really appreciate any help.

Oh btw, this was fixed. Earlier, I was draining system tenant instance which I now changed with the correct sql instance.

Copy link
Copy Markdown
Collaborator

@jeffswenson jeffswenson left a comment

Choose a reason for hiding this comment

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

LGTM

I think you should also get a review from someone on sql foundations or sql queries. The functionality straddles the two teams. Historically sql foundations owned the sql_instances infrastructure and sql queries still owns distsql planning which is impacted by this change.

@shubhamdhama shubhamdhama force-pushed the draining-use-in-distsql-100578 branch 2 times, most recently from 85f61c1 to 29d22c9 Compare August 28, 2024 16:58
@shubhamdhama shubhamdhama requested a review from a team August 28, 2024 17:06
@rytaft rytaft requested review from a team and DrewKimball and removed request for a team and yuzefovich August 28, 2024 17:14
@shubhamdhama shubhamdhama force-pushed the draining-use-in-distsql-100578 branch from 29d22c9 to 2f8894c Compare August 29, 2024 11:07
Copy link
Copy Markdown
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: I just have a few questions.

Reviewed 6 of 22 files at r1, 6 of 18 files at r3, 13 of 13 files at r5, 4 of 4 files at r6, 4 of 4 files at r7, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @dt, @jeffswenson, @shubhamdhama, and @stevendanna)


pkg/sql/distsql_physical_planner.go line 1725 at r7 (raw file):

	}

	// GetAvailableInstances returns mostly healthy nodes, except those that have

[nit] Why the comment change?


pkg/sql/sqlinstance/instancestorage/instancereader.go line 138 at r5 (raw file):

		Locality:        row.locality,
		BinaryVersion:   row.binaryVersion,
		Region:          row.region,

Is it problematic that we weren't filling in Region before?


pkg/upgrade/upgrades/v24_3_sql_instances_add_draining.go line 39 at r6 (raw file):

		mutableDesc.TableDescriptor = *expectedDesc
		mutableDesc.Version = version
		return txn.Descriptors().WriteDesc(ctx, false, mutableDesc, txn.KV())

Will WriteDesc increment the descriptor version?

Copy link
Copy Markdown
Contributor Author

@shubhamdhama shubhamdhama left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @DrewKimball, @dt, @jeffswenson, and @stevendanna)


pkg/sql/distsql_physical_planner.go line 1725 at r7 (raw file):

Previously, DrewKimball (Drew Kimball) wrote…

[nit] Why the comment change?

Done. (Nice catch, in my previous revision I removed this function).


pkg/sql/sqlinstance/instancestorage/instancereader.go line 138 at r5 (raw file):

Previously, DrewKimball (Drew Kimball) wrote…

Is it problematic that we weren't filling in Region before?

It wasn't problematic earlier, but I figured it's most likely a miss.


pkg/upgrade/upgrades/v24_3_sql_instances_add_draining.go line 39 at r6 (raw file):

Previously, DrewKimball (Drew Kimball) wrote…

Will WriteDesc increment the descriptor version?

Yes,

// WriteDesc constructs a new Batch, calls WriteDescToBatch and runs it.
func (tc *Collection) WriteDesc(
ctx context.Context, kvTrace bool, desc catalog.MutableDescriptor, txn *kv.Txn,
) error {
b := txn.NewBatch()
if err := tc.WriteDescToBatch(ctx, kvTrace, desc, b); err != nil {
return err
}

// WriteDescToBatch calls MaybeIncrementVersion, adds the descriptor to the
// collection as an uncommitted descriptor, and writes it into b.
func (tc *Collection) WriteDescToBatch(
ctx context.Context, kvTrace bool, desc catalog.MutableDescriptor, b *kv.Batch,
) error {
if desc.GetID() == descpb.InvalidID {
return errors.AssertionFailedf("cannot write descriptor with an empty ID: %v", desc)
}
desc.MaybeIncrementVersion()
if !tc.skipValidationOnWrite && tc.validationModeProvider.ValidateDescriptorsOnWrite() {
if err := validate.Self(tc.version, desc); err != nil {
return err
}
}

@shubhamdhama shubhamdhama force-pushed the draining-use-in-distsql-100578 branch from 2f8894c to 6337d93 Compare September 2, 2024 06:30
@rimadeodhar rimadeodhar requested a review from fqazi September 3, 2024 16:51
@rimadeodhar
Copy link
Copy Markdown
Collaborator

@fqazi: Please could you take a look from the SQL Foundations side? Thanks.

@fqazi fqazi requested a review from DrewKimball September 3, 2024 17:12
Copy link
Copy Markdown
Collaborator

@fqazi fqazi left a comment

Choose a reason for hiding this comment

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

:lgtm:

One minor nit

Reviewed 6 of 22 files at r1, 4 of 18 files at r3, 3 of 13 files at r5, 2 of 4 files at r6, 10 of 10 files at r8, 4 of 4 files at r9, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball, @dt, @jeffswenson, @shubhamdhama, and @stevendanna)


pkg/upgrade/upgrades/v24_3_sql_instances_add_draining.go line 37 at r8 (raw file):

		}
		version := mutableDesc.Version
		mutableDesc.TableDescriptor = *expectedDesc

Use protoutil.Clone here

@shubhamdhama shubhamdhama force-pushed the draining-use-in-distsql-100578 branch from 6337d93 to f1046f1 Compare September 3, 2024 17:26
@shubhamdhama
Copy link
Copy Markdown
Contributor Author

pkg/upgrade/upgrades/v24_3_sql_instances_add_draining.go line 37 at r8 (raw file):

Previously, fqazi (Faizan Qazi) wrote…

Use protoutil.Clone here

Done.

@shubhamdhama shubhamdhama force-pushed the draining-use-in-distsql-100578 branch from f1046f1 to bebbfa5 Compare September 4, 2024 06:58
@shubhamdhama
Copy link
Copy Markdown
Contributor Author

Thanks everyone for the review.

bors r=jeffswenson,DrewKimball,fqazi

Copy link
Copy Markdown
Collaborator

@stevendanna stevendanna 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 working on this. Overall this looks reasonable to me.

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

bors r-

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Sep 4, 2024

Canceled.

This is a preliminary commit toward improving the DistSQL to avoid planning
on an instance that is draining. The idea here is to set this column to
true as soon as the draining request is received by the server and use that
through instance reader in the planner at gateway.

Adding this column requires writing a migration for the upgrade. And since
this new column is nullable we don't need to backfill and we are good with
just swapping the descriptors.

Informs: cockroachdb#100578
Release note: None

Development notes:

To update `pkg/sql/catalog/bootstrap/testdata/testdata` I had to manually
update the hash values for `system`/`tenant` in this file and pass the
`--rewrite` option to do the rest:
```
dev test //pkg/sql/catalog/bootstrap:bootstrap_test \
	--filter=TestInitialValuesToString/testdata --rewrite
```

To update tests in
`pkg/sql/catalog/systemschema_test/testdata/bootstrap_{system,tenant}`
```
dev test \
	//pkg/sql/catalog/systemschema_test:systemschema_test_test --rewrite
```
@shubhamdhama shubhamdhama force-pushed the draining-use-in-distsql-100578 branch from bebbfa5 to 95e3751 Compare September 4, 2024 12:08
This change would prevent distsql planner to include a draining node in a
plan by leveraging the system.sql_instances table's `is_draining` field.

Somewhat related work to avoid planning on an unavailable node: cockroachdb#124986

Fixes: cockroachdb#100578
Epic: CRDB-31575
Release note: None
@shubhamdhama shubhamdhama force-pushed the draining-use-in-distsql-100578 branch from 95e3751 to 9ef988d Compare September 4, 2024 13:16
@shubhamdhama
Copy link
Copy Markdown
Contributor Author

TFTR.

bors r=jeffswenson,DrewKimball,fqazi,stevendanna

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Sep 4, 2024

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.

sql: distsql planning should avoid sending flows to SQL servers in the process of starting up, and in the process of shutting down

7 participants