Skip to content

distsql: improve unhealthy node detection in SQL instances-based planning.#124986

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
shubhamdhama:120774-improve-unhealthy-node-detection
Jul 9, 2024
Merged

distsql: improve unhealthy node detection in SQL instances-based planning.#124986
craig[bot] merged 1 commit intocockroachdb:masterfrom
shubhamdhama:120774-improve-unhealthy-node-detection

Conversation

@shubhamdhama
Copy link
Copy Markdown
Contributor

@shubhamdhama shubhamdhama commented Jun 3, 2024

The distsql planner currently uses different strategies to avoid planning
on unhealthy nodes in single and multi-tenant deployments. For
single-tenant deployments, we check for a healthy gRPC connection and use
information from the gossip protocol, which allows for quick detection. For
multi-tenant deployments, we rely on the sql_instances system table cache
to check if an instance is healthy. However, this cache may not update for
over half a minute (40 seconds in my tests) after a node goes down, leading
to planning on an unhealthy instance, which are later retried-as-local at the gateway.

To prevent planning on dead SQL instances, the resolver now ensures it does
not consider such instances by first establishing a healthy gRPC
connection. This means that in shared-process mode, if an instance is found
to be unhealthy, the gateway will be used. In external mode, such a node
would be excluded from the candidates for locality-based matching or
round-robin selection.

Fixes: #120774

Epic CRDB-39091

Release note: None

@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@shubhamdhama shubhamdhama marked this pull request as ready for review June 10, 2024 10:13
@shubhamdhama shubhamdhama requested review from a team as code owners June 10, 2024 10:13
@shubhamdhama shubhamdhama requested review from DrewKimball and removed request for a team June 10, 2024 10:13
@yuzefovich yuzefovich requested review from yuzefovich and removed request for DrewKimball June 10, 2024 15:35
@yuzefovich
Copy link
Copy Markdown
Member

Hey @shubhamdhama, is this ready for a look? I see that there are a couple of check boxes still. In any case, I'll take a look tomorrow.

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

Hi @yuzefovich, Yes, I'm still working on the above pending tasks and some unit test failures. However, I would appreciate your review on this anyway.

Copy link
Copy Markdown
Member

@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.

Nice! This is definitely on the right track. I have some nits, but my main question is when should we be performing the filtering of unhealthy instances and whether we should cache that information. Currently, in this PR we perform it "on demand", whenever a particular instance might be considered for physical planning, and in some cases we will perform it many times for the same instance. In the single-tenant setup we maintain a map from the NodeID to health status (stored in PlanningCtx.nodeStatuses) where we perform the health check at most once for a particular node and then use the result of that check going forward, throughout the whole physical planning for this query.

I think we might want to replicate that - i.e. rather that filtering out instances from the list, we'd maintain nodeStatuses for instances and use that as the cache for the health of instances. If we haven't tried dialing an instance, then nodeStatuses[instanceID] will be empty; if we have dialed, then we'll get the health status from the map. Only gateway instance is considered healthy right away (this is already done in NewPlanningCtxWithOracle). If we go with this approach, then some of my inlined comments on the PR will become irrelevant.

Additionally, there is a second aspect of health that we check in the system tenant - isAvailable is also called. In single-tenant setup it performs the "node liveness" check (which in case of the DistSQL means that the node is "alive and connected"). We currently hard-code return true for multi-tenant setup only because at the time of writing that code (and actually still at the moment) DistSQL is disabled on serverless (i.e. in the external process mode). I feel like ConnHealthTryDialInstance approach actually takes care of this "isAvailable" too, so perhaps we just want to update some comments in server_sql.go where isAvailable function is created.

Reviewed 4 of 4 files at r1, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @shubhamdhama)


pkg/rpc/nodedialer/nodedialer.go line 239 at r1 (raw file):

// heartbeat. If no healthy connection is found, it will attempt to dial the
// instance.
func (n *Dialer) ConnHealthTryDialInstance(instanceInfo sqlinstance.InstanceInfo) error {

nit: rather than plumb the whole InstanceInfo struct (which makes rpc/nodedialer depend on sqlinstance package, which seems somewhat wrong), we could pass addr and id arguments explicitly.


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

	ctx context.Context,
) func(nodeID roachpb.NodeID) (base.SQLInstanceID, SpanPartitionReason) {
	allHealthy, err := dsp.sqlAddressResolver.GetAllInstances(ctx)

nit: let's rename allHealthy to allInstances.


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

				nodeID, rpc.DefaultClass); err == nil {
				return sqlInstance, SpanPartitionReason_TARGET_HEALTHY
			}

If we fail to dial here, should we remove sqlInstance from healthyNodes? This way we can avoid repeated dials for the same instance. The downside is that if the instance does come up in the middle of the physical planning, we won't use it (which seems acceptable to me).


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

	// cache. The filtering out of these nodes is deferred to the resolver using
	// the filterUnhealthyNodes function.
	instances, err := dsp.sqlAddressResolver.GetAllInstances(ctx)

There are a couple other places (GetAllInstancesByLocality and setupAllNodesPlanningTenant) where we call GetAllInstances and probably should add filtering too.


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

			// sql_instances cache. Therefore, we filter out these unhealthy nodes by
			// dialing them.
			instances = dsp.filterUnhealthyInstances(instances)

resolver function can be called many times during physical planning of a single statement, so filtering (which includes possibly dialing the instances) also will be performed many times, and I don't think that's desirable. I think we should perform filtering only once unconditionally, right above "healthy SQL instances ..." log message. (Perhaps we could add another log message to highlight which instances were deemed unhealthy.)


pkg/sql/distsql_physical_planner_test.go line 486 at r1 (raw file):

}

func TestDistSQLDeadHostsInSharedMode(t *testing.T) {

nit: there is a lot of duplication with TestDistSQLDeadHosts. I think it'd be nice to extract the shared code into a helper function that performs the necessary set up.


pkg/sql/distsql_physical_planner_test.go line 490 at r1 (raw file):

	defer log.Scope(t).Close(t)

	skip.UnderShort(t, "takes 20s")

nit: we might to do skip.UnderDuress since this test seems heavy and could be flaky under special configs (race or deadlock).


pkg/sql/distsql_physical_planner_test.go line 541 at r1 (raw file):

	r.Exec(t, fmt.Sprintf("INSERT INTO t SELECT i, i*i FROM generate_series(1, %d) AS g(i)", n))

	r.Exec(t, "SET DISTSQL = ON")

nit: do we need distsql = on? I'd expect that the aggregation query would be distributed even with the default auto.


pkg/sql/distsql_physical_planner_test.go line 563 at r1 (raw file):

	// Verify the plan should include all 5 nodes.
	r.CheckQueryResults(t,
		"SELECT info FROM [EXPLAIN (DISTSQL, SHAPE) SELECT sum(xsquared) FROM t] WHERE info LIKE 'Diagram%'",

nit: I think SHAPE doesn't do anything here (i.e. it's only supported for EXPLAIN (DDL, SHAPE)).

@shubhamdhama shubhamdhama force-pushed the 120774-improve-unhealthy-node-detection branch 2 times, most recently from d6f0e57 to 39aa31b Compare June 27, 2024 06:10
@shubhamdhama shubhamdhama requested a review from yuzefovich June 27, 2024 17:44
@shubhamdhama
Copy link
Copy Markdown
Contributor Author

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

Previously, yuzefovich (Yahor Yuzefovich) wrote…

If we fail to dial here, should we remove sqlInstance from healthyNodes? This way we can avoid repeated dials for the same instance. The downside is that if the instance does come up in the middle of the physical planning, we won't use it (which seems acceptable to me).

I think this is resolved with nodeStatuses cache.

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

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

Previously, yuzefovich (Yahor Yuzefovich) wrote…

There are a couple other places (GetAllInstancesByLocality and setupAllNodesPlanningTenant) where we call GetAllInstances and probably should add filtering too.

Done.

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

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

Previously, yuzefovich (Yahor Yuzefovich) wrote…

resolver function can be called many times during physical planning of a single statement, so filtering (which includes possibly dialing the instances) also will be performed many times, and I don't think that's desirable. I think we should perform filtering only once unconditionally, right above "healthy SQL instances ..." log message. (Perhaps we could add another log message to highlight which instances were deemed unhealthy.)

Partially done. I have kept the filtering within the resolver as we are doing with resolver for mixed-process case like we discussed it's probably a good idea to let it detect unhealthy nodes when resolver is called to react to unhealthy nodes between the creation of resolver and when it's called.

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

pkg/sql/distsql_physical_planner_test.go line 486 at r1 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: there is a lot of duplication with TestDistSQLDeadHosts. I think it'd be nice to extract the shared code into a helper function that performs the necessary set up.

I have gathered three related tests for three modes (single tenant, shared and external) into one TestDistSQLUnavailableHosts. I have named it this way as I'm thinking of moving tests related to draining into this one too (the one that's present currently for single tenant and other ones I would be writing in follow-up PR).

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

pkg/sql/distsql_physical_planner_test.go line 541 at r1 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: do we need distsql = on? I'd expect that the aggregation query would be distributed even with the default auto.

Right we don't.

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

pkg/sql/distsql_physical_planner_test.go line 563 at r1 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: I think SHAPE doesn't do anything here (i.e. it's only supported for EXPLAIN (DDL, SHAPE)).

Actually it does something, I'm not sure why, but it gives consistent URL for plans that looks completely same otherwise I have two plans that looks the same but different URL (I lost the URLs but I can reproduce and share them if you are interested).

@shubhamdhama
Copy link
Copy Markdown
Contributor Author

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

		skip.UnderDuress(t, "takes 20s")

		tc, r := startAndSetupCluster(t, base.SharedTestTenantAlwaysEnabled)

Btw I hope it's fine to use this directly as the value for DefaultTestTenant. Asking because of this note

// Note: this value should not be used for checked in test code
// unless there is a good reason to do so. We want the common case
// to use TestTenantProbabilistic or TestTenantProbabilisticOnly.
TestTenantAlwaysEnabled = DefaultTestTenantOptions{testBehavior: ttEnabled, allowAdditionalTenants: true}

Code quote:

SharedTestTenantAlwaysEnabled

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.

Thanks for the detailed review!

Now some other tests are failing (in non-sql_test target), I'll figure out why. In the meantime I can use another round of review to check it's still on the right direction. And sorry for the delay in updating this PR, I was thinking of fixing the draining case in this one but that may take bit longer we can get this one going. I'll try to follow up quickly for next review iterations.

On a side note, if I stress run shared tenant mode it's unexpectedly failing with less than 5 nodes in the plan (even before killing the 5th node). Again, I am yet to dig deeper into this one too.

                                Wraps: (2) query 'SELECT info FROM [EXPLAIN (DISTSQL, SHAPE) SELECT sum(xsquared) FROM t] WHERE info LIKE 'Diagram%'':
                                  | expected:
                                  | Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJyklV9vmzAUxd_3KdCVpraSq2AghPDUqmFqtPTPQqZNqqLKC7cMleDUNmqrKt99IjQNYQVB4CERxj7n8Lv25Q3kUwwu-N7Eu5hpUfLAtW_Tmyvtzvt9OzkfX2vHo7E_839MiOZfnt96J9r7VJkuj1_kU8oEBif5GjXXfl16Uy-XmYy_e9rRKGKhYMuvR0Ag4QFesyVKcO-AAgEDCJhAwAICfZgTWAm-QCm5yKa8bRaMgxdwdQJRskpVNjwnsOACwX0DFakYwYUZ-xPjFFmAoqcDgQAVi-KNjTpT96tHfAUCFzxOl4l0tW1sIOCvWDbSo7rdo6f5X8_QYb4mwFO1s5SKhQguLWQcj8DV16R5zPMwFBgyxUWvv5_S_3l1fEZPKm2Nkm2_0nbnliZcBJi_5s5qvq4PRvV2ycxSMrpPhDYvHD28cD1D39bOal472iZpAZHdrXb2nq3RHJDRAZD1AchuDshok7QAaNAN0GDP1mwOyOwAyP4A5DQHZLZJWgDkdAPk7NlazQFZHQA574CMxnSsNjELdIbd6AzbtOQpyhVPJJZ65OdOesnplGbNFIMQ884reSoWeCv4YjM3v73ZCG0GApQqf0rzm3GyfSSVQLb8-KIUlWitkrGnRItK_bKSUZ-pTSizVsqqVqJlJevQ17PLSv1aJbs6k1FWsg_NNCgrDWqVnOpMZlnJOTSTU1Ya1m8DvTqU9d_erN_mNamG2dF5iPnzfRSAC_r7dfrJz_aCbAELZXZ-_b_8eSM7e11lp--BxRIJXLFHHKFCsYySSKpoAa4SKa7XX_4FAAD__5H-gCw=
                                  | got:
                                  | Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJyklF1ro04Uxu__n2I48KctTImjxmS9amlcGjZ92ZhlF0oos_HUSo1jZ0baUvLdF7VpjDSiiRcJzsvveeY543kH9RyDC7438S5mJEoeBPk-vbkid96f28n5-Jocj8b-zP85ocS_PL_1TsjHUpUtj1_Vc8YlBiflHj0nvy-9qVdiJuMfHjkaRTyUfPn_EVBIRIDXfIkK3DtgQMEEChZQsGFOIZVigUoJmU-_F4vHwSu4BoUoSTOdD88pLIREcN9BRzpGcGHG_8Y4RR6g7BlAIUDNo7iQ0Gf6Pn3CN6BwIeJsmSiXrC0DBT_l-UiPGU6PnZZ_PdMgPAkII0I_ooT5ioLI9EZeaR4iuKzidzwC11jR9pbPw1BiyLWQPXvbsf_r6viMneyUNWuy9k7ZjVqWCBlgeeSN1HzVbGzYzZhVMzbcMsbal5DtX8KeaayraButK8e6OK0E1D-scv0tWbN9QOYBAdmfATntAzK7OK0E5BwWkLMla7UPyDogIOczoGH7gKwuTisBDQ4LaNCl5UxRpSJRWOsBXysZNaVTljcLDEIsO4sSmVzgrRSLYm35elOAioEAlS5nWfkyTtZTSkvky8-OWSWxRpK5RWJVkl0nmY2kbx08WY0kezeJ1Un2vqfr10n9RpKz25NZJzn7enLqpEEjabjbk1UnDff1NMjv6EMsXu6jAFwwPp7TL37WD-QbeKjyD8V_FC8FdvaW5tf8gccKKVzxJxyhRrmMkkjpaAGulhmuVv_9CwAA__-0ofXg

PS: Oops, I guess I wasn't supposed to hit "send" button for each comment on Reviewable :)

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

Copy link
Copy Markdown
Member

@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.

Nice! I think this is almost ready to go, I have a few more comments and nits.

Reviewed 7 of 7 files at r2, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @shubhamdhama)


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

Previously, shubhamdhama (Shubham Dhama) wrote…

Partially done. I have kept the filtering within the resolver as we are doing with resolver for mixed-process case like we discussed it's probably a good idea to let it detect unhealthy nodes when resolver is called to react to unhealthy nodes between the creation of resolver and when it's called.

SGTM.


pkg/sql/distsql_physical_planner.go line 1606 at r2 (raw file):

	if log.ExpensiveLogEnabled(ctx, 2) {
		log.VEventf(ctx, 2, "healthy SQL instances available for distributed planning: %v", allInstances)

nit: let's reword the comment a bit: s/healthy/all/.


pkg/sql/distsql_physical_planner.go line 1676 at r2 (raw file):

	instances []sqlinstance.InstanceInfo, nodeStatusesCache map[base.SQLInstanceID]NodeStatus,
) (healthyInstances []sqlinstance.InstanceInfo, unhealthyInstances []sqlinstance.InstanceInfo) {
	healthyInstances = make([]sqlinstance.InstanceInfo, 0, len(instances))

Let's avoid this allocation by modifying the passed-in argument instances in-place to become the healthy instances.


pkg/sql/distsql_physical_planner.go line 1776 at r2 (raw file):

			if len(instances) == 0 {
				log.Eventf(ctx, "no healthy sql instances available for planning, using the gateway")
				return dsp.gatewaySQLInstanceID, SpanPartitionReason_GATEWAY_ON_ERROR

nit: I think this (and below) should be SpanPartitionReason_GATEWAY_TARGET_UNHEALTHY.


pkg/sql/distsql_plan_bulk.go line 110 at r2 (raw file):

		return nil, nil, err
	}
	pods, _ = dsp.filterUnhealthyInstances(pods, nil /* nodeStatusesCache */)

nit: even though it doesn't matter right now, it seems cleaner to pass planCtx.nodeStatuses as nodeStatusesCache argument.


pkg/sql/distsql_physical_planner_test.go line 486 at r1 (raw file):

Previously, shubhamdhama (Shubham Dhama) wrote…

I have gathered three related tests for three modes (single tenant, shared and external) into one TestDistSQLUnavailableHosts. I have named it this way as I'm thinking of moving tests related to draining into this one too (the one that's present currently for single tenant and other ones I would be writing in follow-up PR).

SGTM


pkg/sql/distsql_physical_planner_test.go line 563 at r1 (raw file):

Previously, shubhamdhama (Shubham Dhama) wrote…

Actually it does something, I'm not sure why, but it gives consistent URL for plans that looks completely same otherwise I have two plans that looks the same but different URL (I lost the URLs but I can reproduce and share them if you are interested).

Yeah, good point. In explain.MakeFlags(), if SHAPE option is used, we set a few flags to hide values (to make some things deterministic). It's not used widely, so I was surprised about SHAPE being used in a very old test, but the test was recently adjusted a bit.


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

Previously, shubhamdhama (Shubham Dhama) wrote…

Btw I hope it's fine to use this directly as the value for DefaultTestTenant. Asking because of this note

// Note: this value should not be used for checked in test code
// unless there is a good reason to do so. We want the common case
// to use TestTenantProbabilistic or TestTenantProbabilisticOnly.
TestTenantAlwaysEnabled = DefaultTestTenantOptions{testBehavior: ttEnabled, allowAdditionalTenants: true}

Yeah, the idea behind SharedTestTenantAlwaysEnabled and ExternalTestTenantAlwaysEnabled was that we wouldn't use them in the tests like this. The proposed approach is to use base.TestControlsTenantsExplicitly and then manually start the correct tenant in the sub-tests. It is a bit more tedious, but take a look at TestClusterInflightTraces in util/tracing/collector/collector_test.go for an example.

(The thinking behind this is that tenants being started automatically by the testserver are random, so if you need a tenant for a sub-test, you shouldn't override the randomized behavior and, instead, you should opt out of it completely and manually control the tenant creation.)

@yuzefovich
Copy link
Copy Markdown
Member

PS: Oops, I guess I wasn't supposed to hit "send" button for each comment on Reviewable :)

No problem!

@shubhamdhama shubhamdhama force-pushed the 120774-improve-unhealthy-node-detection branch from 39aa31b to b8f6572 Compare July 2, 2024 15:44
@shubhamdhama shubhamdhama requested a review from a team as a code owner July 2, 2024 15:44
@shubhamdhama shubhamdhama requested review from msbutler and removed request for a team July 2, 2024 15:44
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! 0 of 0 LGTMs obtained (waiting on @msbutler and @yuzefovich)


pkg/sql/distsql_physical_planner.go line 1776 at r2 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: I think this (and below) should be SpanPartitionReason_GATEWAY_TARGET_UNHEALTHY.

Good catch, it seems there is a better option: SpanPartitionReason_GATEWAY_NO_HEALTHY_INSTANCES

@craig craig bot merged commit 3119004 into cockroachdb:master Jul 9, 2024
shubhamdhama added a commit to shubhamdhama/cockroach that referenced this pull request Jul 11, 2024
This PR fixes the two flaky tests as reported in the linked issue:

*TestDistSQLUnavailableHosts/unhealthy-nodes-in-single-tenant*

The issue was that the first plan of `plans` slice was incorrect and
included the 5th node.

Although this test shouldn't be affected by the changes in cockroachdb#124986, I made
slight improvements during refactoring. I added a plan check after stopping
the 5th node and before running the query to avoid updating the lease and
to understand how the query is planned.

This issue wasn't detected even in local stress testing. My hypothesis is
that leases are usually updated quickly enough that the 5th node's ranges
are moved to the 2nd or 3rd node most of the time. But on teamcity it hits
the case where the leases haven't moved and they are planned on the
gateway.

*TestDistSQLUnavailableHosts/unhealthy-nodes-in-shared-mode*

The test runs fast enough that if a heartbeat hasn't occurred yet,
`ConnHealthTryDialInstance` returns `ErrNotHeartbeated`. The fix here is in
most cases, this error can be ignored, and the node can be considered
healthy. If the node is indeed unhealthy, we would anyway retry-as-local.

Fixes: cockroachdb#126908

Epic: CRDB-40179

Release note: None
shubhamdhama added a commit to shubhamdhama/cockroach that referenced this pull request Jul 11, 2024
This PR fixes the two flaky tests as reported in the linked issue:

*TestDistSQLUnavailableHosts/unhealthy-nodes-in-single-tenant*

The issue was that the first plan of `plans` slice was incorrect and
included the 5th node.

Although this test shouldn't be affected by the changes in cockroachdb#124986, I made
slight improvements during refactoring. I added a plan check after stopping
the 5th node and before running the query to avoid updating the lease and
to understand how the query is planned.

This issue wasn't detected even in local stress testing. My hypothesis is
that leases are usually updated quickly enough that the 5th node's ranges
are moved to the 2nd or 3rd node most of the time. But on teamcity it hits
the case where the leases haven't moved and they are planned on the
gateway.

*TestDistSQLUnavailableHosts/unhealthy-nodes-in-shared-mode*

The test runs fast enough that if a heartbeat hasn't occurred yet,
`ConnHealthTryDialInstance` returns `ErrNotHeartbeated`. The fix here is in
most cases, this error can be ignored, and the node can be considered
healthy. If the node is indeed unhealthy, we would anyway retry-as-local.

Fixes: cockroachdb#126908

Epic: CRDB-40179

Release note: None
shubhamdhama added a commit to shubhamdhama/cockroach that referenced this pull request Jul 11, 2024
This PR fixes the two flaky tests as reported in the linked issue:

*TestDistSQLUnavailableHosts/unhealthy-nodes-in-single-tenant*

The issue was that the first plan of `plans` slice was incorrect and
included the 5th node.

Although this test shouldn't be affected by the changes in cockroachdb#124986, I made
slight improvements during refactoring. I added a plan check after stopping
the 5th node and before running the query to avoid updating the lease and
to understand how the query is planned.

This issue wasn't detected even in local stress testing. My hypothesis is
that leases are usually updated quickly enough that the 5th node's ranges
are moved to the 2nd or 3rd node most of the time. But on teamcity it hits
the case where the leases haven't moved and they are planned on the
gateway.

*TestDistSQLUnavailableHosts/unhealthy-nodes-in-shared-mode*

The test runs fast enough that if a heartbeat hasn't occurred yet,
`ConnHealthTryDialInstance` returns `ErrNotHeartbeated`. The fix here is in
most cases, this error can be ignored, and the node can be considered
healthy. If the node is indeed unhealthy, we would anyway retry-as-local.

Fixes: cockroachdb#126908

Epic: CRDB-40179

Release note: None
craig bot pushed a commit that referenced this pull request Jul 12, 2024
126976: distsql: Fix flaky TestDistSQLUnavailableHosts. r=yuzefovich a=shubhamdhama

This PR fixes the two flaky tests as reported in the linked issue:

**TestDistSQLUnavailableHosts/unhealthy-nodes-in-single-tenant**

The issue was that the first plan of `plans` slice was incorrect and included the 5th node.

Some more context: Although this test shouldn't be affected by the core changes in #124986, I made slight improvements during refactoring. I added a plan check after stopping the 5th node and before running the query to avoid updating the lease and to understand how the query is planned.

This issue wasn't detected even in local stress testing. My hypothesis is that leases are usually updated quickly enough that the 5th node's ranges are moved to the 2nd or 3rd node most of the time. But on teamcity it hits the case where the leases haven't moved and range is planned on the gateway.

**TestDistSQLUnavailableHosts/unhealthy-nodes-in-shared-mode**

The test runs fast enough that if a heartbeat hasn't occurred yet, `ConnHealthTryDialInstance` returns `ErrNotHeartbeated`. The fix here is in most cases, this error can be ignored, and the node can be considered healthy. If the node is indeed unhealthy, we would anyway retry-as-local.

Fixes: #126908

Epic: CRDB-40179

Release note: None

Co-authored-by: Shubham Dhama <shubham.dhama@cockroachlabs.com>
blathers-crl bot pushed a commit that referenced this pull request Jul 12, 2024
This PR fixes the two flaky tests as reported in the linked issue:

*TestDistSQLUnavailableHosts/unhealthy-nodes-in-single-tenant*

The issue was that the first plan of `plans` slice was incorrect and
included the 5th node.

Although this test shouldn't be affected by the changes in #124986, I made
slight improvements during refactoring. I added a plan check after stopping
the 5th node and before running the query to avoid updating the lease and
to understand how the query is planned.

This issue wasn't detected even in local stress testing. My hypothesis is
that leases are usually updated quickly enough that the 5th node's ranges
are moved to the 2nd or 3rd node most of the time. But on teamcity it hits
the case where the leases haven't moved and they are planned on the
gateway.

*TestDistSQLUnavailableHosts/unhealthy-nodes-in-shared-mode*

The test runs fast enough that if a heartbeat hasn't occurred yet,
`ConnHealthTryDialInstance` returns `ErrNotHeartbeated`. The fix here is in
most cases, this error can be ignored, and the node can be considered
healthy. If the node is indeed unhealthy, we would anyway retry-as-local.

Fixes: #126908

Epic: CRDB-40179

Release note: None
@andrewbaptist
Copy link
Copy Markdown

I just noticed this PR merged, but I'm not sure if this is the right approach. The goal was to centralize "all" liveness checks into the NodeVitality calls. With the upcoming work on leader leases, we are going to make an even more robust version of this. Specifically I don't mind the addition of the ConnHealthTryDialInstance call, but ideally we should do the call from within IsLive check for DistSQL.

The problem right now is that NodeVitality may not "function correctly" on SQL pods, although I don't think that is easily solvable, it just can't look at the liveness records from gossip. Ideally we shouldn't need the connHealthCheckerSystem and instanceConnHealthChecker inside the DistSQLPlanner and it could all be part of the isAvailable callback which either directly or indirectly calls into NodeVitality.

I know this is already merged, so I'm not sure if it is worth changing now, but if might be a lot cleaner to not have to split this out the way it is done.

@yuzefovich
Copy link
Copy Markdown
Member

@andrewbaptist IIUC what you're saying is that this code

var isAvailable func(sqlInstanceID base.SQLInstanceID) bool
nodeLiveness, hasNodeLiveness := cfg.nodeLiveness.Optional(47900)
if hasNodeLiveness {
isAvailable = func(sqlInstanceID base.SQLInstanceID) bool {
return nodeLiveness.GetNodeVitalityFromCache(roachpb.NodeID(sqlInstanceID)).IsLive(livenesspb.DistSQL)
}
} else {
// We're on a SQL tenant, so this is the only node DistSQL will ever
// schedule on - always returning true is fine.
isAvailable = func(sqlInstanceID base.SQLInstanceID) bool {
return true
}
}

ideally should become the only necessary function for DistSQL planner to decide whether remote nodes / instances are healthy and available for DistSQL work, right? It sounds like this NodeVitality thingy already does (or will do shortly) what we need in the system tenant, so we could easily get rid off connHealthCheckerSystem altogether, but NodeVitality will be hard to make it work for SQL pods.

If my understanding is correct, then I don't see a difficulty with this PR being merged - it improved the situation in the multi-tenant environment, especially so in the external process mode (i.e. serverless). Given that KV layer doesn't really control when SQL pods go up or down, I imagine that we'll always need some mechanism at the SQL layer to ensure that SQL pods are healthy, and this mechanism after this change is "try dial the pod". It sounds like shortly we could simplify the DistSQL physical planner health check in the system tenant, but some folks think that we should fully deprecate the system tenant code path altogether and only keep the "instance-based DistSQL planning", so I'm curious in your thoughts on how we can improve the situation for SQL pods where we don't have access to gossip-based info.

@andrewbaptist
Copy link
Copy Markdown

I was a little curious about if this was possible and did a small refactoring of this code a little to make it more clear the difference between checking SQL vs KV node health. I moved things around but should not have changed the logic at all in #127293. I think after this change it should be possible to move the code from CheckKVNodeHealth into the IsLive function and we would only retain the CheckSQLNodeHealth.

I agree that the NodeVitality doesn't quite work for SQL instances. Additionally the newly added checks might be insufficient. For instance if we support draining of SQL instances should we add that check?

Longer term, I'd like to think about what NodeVitality for SQL instances should look like especially if we decide to have more SQL-to-SQL connections. Also I'm a little concerned that without the correct locality information in SQL connections we might not plan as efficiently if we have a multi-region setup.

@yuzefovich
Copy link
Copy Markdown
Member

Thanks for the patch! I'll take a look tomorrow.

For instance if we support draining of SQL instances should we add that check?

Indeed, this is a known limitation #100578, and Shubham will be tackling it shortly.

shubhamdhama added a commit to shubhamdhama/cockroach that referenced this pull request Aug 4, 2024
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 added in cockroachdb#126765.

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

Fixes: cockroachdb#100578
Epic: CRDB-31575
Release note: None
shubhamdhama added a commit to shubhamdhama/cockroach that referenced this pull request Aug 26, 2024
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 added in cockroachdb#126765.

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

Fixes: cockroachdb#100578
Epic: CRDB-31575
Release note: None
shubhamdhama added a commit to shubhamdhama/cockroach that referenced this pull request Aug 29, 2024
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 added a commit to shubhamdhama/cockroach that referenced this pull request Sep 2, 2024
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 added a commit to shubhamdhama/cockroach that referenced this pull request Sep 3, 2024
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 added a commit to shubhamdhama/cockroach that referenced this pull request Sep 4, 2024
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 added a commit to shubhamdhama/cockroach that referenced this pull request Sep 4, 2024
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
craig bot pushed a commit that referenced this pull request Sep 4, 2024
128283: distsql: avoid planning on a draining node. r=jeffswenson,DrewKimball,fqazi,stevendanna a=shubhamdhama

Commit-wise description

---
**Commit 1: [`sql: add is_draining to system.sql_instances table.`](https://github.com/cockroachdb/cockroach/pull/128283/commits/4b0de41bc39)**

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.`](https://github.com/cockroachdb/cockroach/pull/128283/commits/6337d932854)**

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

Co-authored-by: Shubham Dhama <shubham.dhama@cockroachlabs.com>
@shubhamdhama shubhamdhama deleted the 120774-improve-unhealthy-node-detection branch March 1, 2025 10: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.

distsql: improve unhealthy node detection in SQL instances-based planning

5 participants