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

distsql: adds support for distributed bulk ops in multi-tenant #76566

Merged
merged 2 commits into from
Feb 25, 2022

Conversation

rharding6373
Copy link
Collaborator

@rharding6373 rharding6373 commented Feb 15, 2022

distsql: adds support for distributed bulk ops in multi-tenant

Previously, distsql did not support distributed queries in multi-tenant
environments. Queries were always planned using a single SQL pod, even
if using multiple pods could yield better parallelism.

This change adds an interface inside distsql that allows different
implementations for functions that differ between single- and
multi-tenant. Currently, it only has SetupAllNodesPlanning, which for
single-tenant remains the same, but in multi-tenant returns all
available SQL pods at the time it is invoked.

This PR also fixes some places where SQL pods need to communicate with
other SQL pods using the PodNodeDialer instead of the NodeDialer that
handles SQL <-> KV communication in multi-tenant mode.

Fixes #74548

References #47900

Release note (sql change): Adds support for distributed import queries in
multi-tenant environments, which allows import queries to have improved
parallelism by utilizing all available SQL pods in the tenant.

importccl, backupccl: add import and restore tenant tests with multiple SQL pods

These tests validate that imports and restores can be
distributed across multiple SQL pods in a single tenant.

Release note: None

@rharding6373 rharding6373 requested a review from a team as a code owner February 15, 2022 05:19
@rharding6373 rharding6373 requested a review from a team February 15, 2022 05:19
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch 2 times, most recently from 55371f5 to 4014ce6 Compare February 15, 2022 05:36
@@ -47,7 +47,7 @@ func StartDistChangefeed(
dsp := execCtx.DistSQLPlanner()
evalCtx := execCtx.ExtendedEvalContext()
planCtx := dsp.NewPlanningCtx(ctx, evalCtx, nil /* planner */, noTxn,
execCtx.ExecCfg().Codec.ForSystemTenant() /* distribute */)
execCtx.ExecCfg().Codec.ForSystemTenant(), false /* tenantDistributionEnabled */)
Copy link
Contributor

Choose a reason for hiding this comment

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

why would we want to disable tenant distribution?

Copy link
Collaborator Author

@rharding6373 rharding6373 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 @miretskiy, @RaduBerinde, @stevendanna, and @yuzefovich)


pkg/ccl/changefeedccl/changefeeddist/distflow.go, line 50 at r2 (raw file):

Previously, miretskiy (Yevgeniy Miretskiy) wrote…

why would we want to disable tenant distribution?

It wasn't enabled before -- distribute = ForSystemTenant() meant that it was only distributed for the non-tenant case (not that it mattered, because NewPlanningCtx() also overrode distribution to disable it in the tenant case). The biggest reason not to enable it right now is we haven't tested tenant distribution with changefeeds yet. It might be interesting to turn tenant distribution on for some of these operations and see what breaks, but I haven't checked tests for use cases like changefeeds to see if there's already coverage for multi-pod tenants -- is there?

@miretskiy
Copy link
Contributor

miretskiy commented Feb 15, 2022 via email

Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

Nice work!

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


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

) *PlanningCtx {
	// Tenants can only distribute plans with tenantDistributionEnabled.
	distribute = distribute && (tenantDistributionEnabled || evalCtx.Codec.ForSystemTenant())

I don't think we need both flags. We should either let the caller pass either true, false, or codec.ForSystemTenant() for the distribute flag as necessary, or define a tri-state enum Distribute, DistributeOnlySystemTenant, DontDistribute


pkg/sql/distsql_plan_bulk.go, line 23 at r1 (raw file):

type distSQLTenantPlanner interface {
	SetupAllNodesPlanning(

[nit] add a comment explaining what the method does


pkg/sql/distsql_plan_bulk.go, line 28 at r1 (raw file):

}

type singleTenantPlanner struct {

"single tenant" is not a great name, since you can be the system tenant in a multi-tenant configuration.
These should be systemTenantPlanner and nonSystemTenantPlanner (or just tenantPlanner).


pkg/sql/distsql_plan_bulk.go, line 41 at r1 (raw file):

	ctx context.Context, evalCtx *extendedEvalContext, execCfg *ExecutorConfig,
) (*PlanningCtx, []base.SQLInstanceID, error) {
	return dsp.distSQLTenantPlanner.SetupAllNodesPlanning(ctx, evalCtx, execCfg)

This is a bit mind-bending.. The tenant planner embeds a *DistSQLPLanner and the DistSQLPlanner contains the tenant planner as a field.

Maybe for now we should just have two internal setupAllNodesPlanningSystem and setupAllNodesPlanningTenant and have SetupAllNodesPlanning call the right one.

Otherwise, we should make a decision as to which interface is "above" the other and disentangle them. If we want the new interface to be "above" then it would be public and it would implement the public SetupAllNodesPlanning call (and could internally use the DistSQLPlanner). If it is "below", then it would be used by DistSQLPlanner.SetupAllNodesPlanning but it wouldn't use a DistSQLPlanner directly. It would be initialized with whatever necessary information and return the instance IDs (with DistsqlPlanner.SetupAllNodesPlanning() setting up the planning context separately). The system-tenant implementation of this interface would also contain CheckInstanceHealthAndVersion and other code that is system-tenant-only.


pkg/sql/distsql_plan_bulk.go, line 44 at r1 (raw file):

}

func (dsp *singleTenantPlanner) SetupAllNodesPlanning(

[nit] add a comment mentioning this is part of the distSQLTenantPlanner interface.


pkg/sql/distsql_plan_bulk.go, line 80 at r1 (raw file):

}

func (dsp *multiTenantPlanner) SetupAllNodesPlanning(

[nit] add a comment mentioning this is part of the distSQLTenantPlanner interface.


pkg/ccl/importccl/import_stmt_test.go, line 6944 at r2 (raw file):

	t1.Exec(t, createStmt)
	t1.Exec(t, importStmt, userfileURI, userfile2URI)

[nit] I wonder if there is a way to check that we're using two nodes.

Copy link
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!

Reviewed 19 of 19 files at r1, 1 of 1 files at r2, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @miretskiy, @RaduBerinde, @rharding6373, and @stevendanna)


pkg/ccl/changefeedccl/changefeeddist/distflow.go, line 50 at r1 (raw file):

	evalCtx := execCtx.ExtendedEvalContext()
	planCtx := dsp.NewPlanningCtx(ctx, evalCtx, nil /* planner */, noTxn,
		execCtx.ExecCfg().Codec.ForSystemTenant(), false /* tenantDistributionEnabled */)

nit: it'd be nice to keep the previous inlined comment (i.e. /* distribute */).


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

	// sqlInstanceProvider has information about SQL instances in a multi-tenant
	// environment

nit: missing periods.


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

	sqlInstanceProvider sqlinstance.Provider

	// bulk IO interface

nit: maybe add a few more words in the comment?


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

Previously, RaduBerinde wrote…

I don't think we need both flags. We should either let the caller pass either true, false, or codec.ForSystemTenant() for the distribute flag as necessary, or define a tri-state enum Distribute, DistributeOnlySystemTenant, DontDistribute

I like the idea of a tri-state enum - this way each caller has to explicitly think whether the distribution of its operation is supported in a multi-tenant setting.


pkg/sql/distsql_plan_bulk.go, line 41 at r1 (raw file):

Previously, RaduBerinde wrote…

This is a bit mind-bending.. The tenant planner embeds a *DistSQLPLanner and the DistSQLPlanner contains the tenant planner as a field.

Maybe for now we should just have two internal setupAllNodesPlanningSystem and setupAllNodesPlanningTenant and have SetupAllNodesPlanning call the right one.

Otherwise, we should make a decision as to which interface is "above" the other and disentangle them. If we want the new interface to be "above" then it would be public and it would implement the public SetupAllNodesPlanning call (and could internally use the DistSQLPlanner). If it is "below", then it would be used by DistSQLPlanner.SetupAllNodesPlanning but it wouldn't use a DistSQLPlanner directly. It would be initialized with whatever necessary information and return the instance IDs (with DistsqlPlanner.SetupAllNodesPlanning() setting up the planning context separately). The system-tenant implementation of this interface would also contain CheckInstanceHealthAndVersion and other code that is system-tenant-only.

+1 on the idea of setupAllNodesPlanningSystem and setupAllNodesPlanningTenant.


pkg/sql/distsql_running.go, line 334 at r1 (raw file):

			// NB: we wouldn't hit this in practice but if we did the actual
			// error would be opaque.
			return nil, nil, nil, errorutil.UnsupportedWithMultiTenancy(47900)

nit: maybe add a link to this issue in the PR description.


pkg/ccl/importccl/import_stmt_test.go, line 6944 at r2 (raw file):

Previously, RaduBerinde wrote…

[nit] I wonder if there is a way to check that we're using two nodes.

I agree that it seems important for the test. Maybe we can check the trace?

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch from 4014ce6 to 4e0fed8 Compare February 16, 2022 22:54
Copy link
Collaborator Author

@rharding6373 rharding6373 left a comment

Choose a reason for hiding this comment

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

TFTRs! I added multi-pod backup/restore tests to the second commit, but still need to figure out in-test verification of bulk op distribution.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @miretskiy, @RaduBerinde, @rharding6373, @stevendanna, and @yuzefovich)


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

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: maybe add a few more words in the comment?

Removed in refactor.


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

Previously, yuzefovich (Yahor Yuzefovich) wrote…

I like the idea of a tri-state enum - this way each caller has to explicitly think whether the distribution of its operation is supported in a multi-tenant setting.

Done


pkg/sql/distsql_plan_bulk.go, line 23 at r1 (raw file):

Previously, RaduBerinde wrote…

[nit] add a comment explaining what the method does

No longer applicable.


pkg/sql/distsql_plan_bulk.go, line 28 at r1 (raw file):

Previously, RaduBerinde wrote…

"single tenant" is not a great name, since you can be the system tenant in a multi-tenant configuration.
These should be systemTenantPlanner and nonSystemTenantPlanner (or just tenantPlanner).

Renamed the functions below.


pkg/sql/distsql_plan_bulk.go, line 41 at r1 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

+1 on the idea of setupAllNodesPlanningSystem and setupAllNodesPlanningTenant.

I removed the confusing interface for now. Thanks for the suggestion!


pkg/sql/distsql_plan_bulk.go, line 44 at r1 (raw file):

Previously, RaduBerinde wrote…

[nit] add a comment mentioning this is part of the distSQLTenantPlanner interface.

No longer applicable.


pkg/sql/distsql_plan_bulk.go, line 80 at r1 (raw file):

Previously, RaduBerinde wrote…

[nit] add a comment mentioning this is part of the distSQLTenantPlanner interface.

No longer applicable


pkg/ccl/backupccl/backup_test.go, line 7059 at r4 (raw file):

		})

		t.Run("system-backup", func(t *testing.T) {

Need to verify that the backup and restore are distributed in the test (I've only verified this manually via distsql plans)


pkg/ccl/importccl/import_stmt_test.go, line 6944 at r2 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

I agree that it seems important for the test. Maybe we can check the trace?

Hm, although when I log the plan diagram internally it looks right: https://cockroachdb.github.io/distsqlplan/decode.html#eJyckDFPwzAQhXd-xs2micPmmSVLQawog2tfq0huzvjOCCnKf0dxhpIqINrx3t3z9_xG4I8ABkDBQB739owM5h00KGigUxATOWSmNMtjOWr9F5haQT_ELLPcKXCUEMwI0ktAMPCG1rfnSEmerdiqBgUexfahvJ4Z07EPaKrK49HmIP6wi_kQerdLRFIJsuwcf0I3KaAsFxCLPSEYPalfwlwy5IGSx4R-Be-mjbh7eqRYNVeH2-hmhdY39aDv6KG5s4jNNBxpYPzXP-u5KPQnXFplysnhayJXMMv4UnxF8MiybPUytENZlYA_zfpP89PKXF-bm5vI3fTwHQAA__-3d_dH

Tracing doesn't indicate anything happening on Node 2...

Still a work in progress.

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch from 4e0fed8 to 50aa18d Compare February 17, 2022 01:41
Copy link
Member

@dt dt 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 @miretskiy, @RaduBerinde, @rharding6373, @stevendanna, and @yuzefovich)


pkg/ccl/importccl/import_stmt_test.go, line 6944 at r2 (raw file):

Previously, rharding6373 (Rachael Harding) wrote…

Hm, although when I log the plan diagram internally it looks right: https://cockroachdb.github.io/distsqlplan/decode.html#eJyckDFPwzAQhXd-xs2micPmmSVLQawog2tfq0huzvjOCCnKf0dxhpIqINrx3t3z9_xG4I8ABkDBQB739owM5h00KGigUxATOWSmNMtjOWr9F5haQT_ELLPcKXCUEMwI0ktAMPCG1rfnSEmerdiqBgUexfahvJ4Z07EPaKrK49HmIP6wi_kQerdLRFIJsuwcf0I3KaAsFxCLPSEYPalfwlwy5IGSx4R-Be-mjbh7eqRYNVeH2-hmhdY39aDv6KG5s4jNNBxpYPzXP-u5KPQnXFplysnhayJXMMv4UnxF8MiybPUytENZlYA_zfpP89PKXF-bm5vI3fTwHQAA__-3d_dH

Tracing doesn't indicate anything happening on Node 2...

Still a work in progress.

One idea: make a table with a column who's default value is row int primary key, instance int default crdb_internal.node_id() (or add a companion builtin crdb_internal.instance_id?) then IMPORT INTO that table two files that just have rows "1", "2", "3" and "4", "5", "6".

Now right now this IMPORT will fail because the crdb_internal.node_id builtin is too volatile. But we could add a TestingAddImportBuiltin(name, *CustomFunc) to pkg/sql/row/expr_walker.go to inject an entry into the supportedImportFuncOverrides map, to allow calling it in the test.

then we just inspect the populated instance IDs in the table e.g.select r from [select count(*) as r, instance_id from t group by instance_id] should look like 3, 3?

Copy link
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.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @miretskiy, @RaduBerinde, @rharding6373, @stevendanna, and @yuzefovich)


pkg/ccl/importccl/import_processor.go, line 177 at r8 (raw file):

// Start is part of the RowSource interface.
func (idp *readImportDataProcessor) Start(ctx context.Context) {
	log.Infof(ctx, "starting read import")

These calls need to happen after StartInternal call below. There, we derive a tracing span with a new context, so in order for these messages appear in the trace, we need to use the new context (the third commit had the right version).


pkg/ccl/importccl/import_stmt_test.go, line 6951 at r8 (raw file):

	t1.Exec(t, "SET tracing = off")
	t2.Exec(t, "SET tracing = off")
	rows := t1.Query(t, "SELECT concat(location, ' ', tag, ' ', span::string, ' ', operation, ' ', message, ' ', timestamp::string) FROM [SHOW TRACE FOR SESSION]")

I recently encountered a case where SHOW TRACE FOR SESSION also didn't include some of the events that I knew for sure occurred. I talked with Andrei about it, he suggested that the session tracing might be broken and he wants to get rid off it.

First, I'd use EXPLAIN ANALYZE (DEBUG) on the query for which you want to see the trace to make sure that events do get into the trace there. Next, I'd refactor the test to use WithStatementTrace testing knob (take a look at TestLargeKeys in kvstreamer/large_keys_test.go for an example).

Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

r1 + r2 LGTM

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @miretskiy, @RaduBerinde, @rharding6373, @stevendanna, and @yuzefovich)

Copy link
Member

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

I meant first two commits LGTM

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @miretskiy, @RaduBerinde, @rharding6373, @stevendanna, and @yuzefovich)

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch 2 times, most recently from c51a0e7 to 90e6636 Compare February 23, 2022 23:34
Copy link
Collaborator Author

@rharding6373 rharding6373 left a comment

Choose a reason for hiding this comment

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

TFTRs! RFAL

I've been unable to verify in-test that the bulk ops are distributed, so I left TODOs in the interest of getting this in before code freeze. I have done manual verification using logs, and here is e.g. a distributed restore plan diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJy0ksFuszAQhO__U0Rz9q-ASVKJW6Veckmr5lhxoPYqRaU2tY3UKuLdKyBSAk1NUujRLN_uzOzuYd9zxACD0pI26RtZxE8IwcDBECFhKIwWZK02dWnf_LiWH4gDhkwVpas_JwxCG0K8h8tcToixLfLM3Sq5FalzZLYFiXkABkkuzfJmDJ-Jl1K9WiQVgy7dodmxx_PnzKRqRx0sqRIG69IdIQ4r9oOiY5NSaSPJkPzepa_5kazThu5SlzZ6ox5yqvIogk8rYqP_62K-umh01BkdXr8R3t1IOPFGwqk3svzFRvj1sYR_e6h86lgWIw_1_IhCK9u3dL5zUIskuaPWkdWlEfRgtGjGtM_7hmtuQpJ1bfXwWKu2VAu8HF6NgRd-OOzDwSnM_TD3wjcdOOjD0ZjA_PBAYH54ILDFmMCWYzz74QHPfnjA8-oqz0n17ysAAP__L_l-2w==

Backups are currently not distributed because they rely on PartitionSpans, which I will fix for the tenant case in a follow-up PR.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @miretskiy, @RaduBerinde, @rharding6373, @stevendanna, and @yuzefovich)


pkg/ccl/importccl/import_processor.go, line 177 at r8 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

These calls need to happen after StartInternal call below. There, we derive a tracing span with a new context, so in order for these messages appear in the trace, we need to use the new context (the third commit had the right version).

Done.


pkg/ccl/importccl/import_stmt_test.go, line 6944 at r2 (raw file):

Previously, dt (David Taylor) wrote…

One idea: make a table with a column who's default value is row int primary key, instance int default crdb_internal.node_id() (or add a companion builtin crdb_internal.instance_id?) then IMPORT INTO that table two files that just have rows "1", "2", "3" and "4", "5", "6".

Now right now this IMPORT will fail because the crdb_internal.node_id builtin is too volatile. But we could add a TestingAddImportBuiltin(name, *CustomFunc) to pkg/sql/row/expr_walker.go to inject an entry into the supportedImportFuncOverrides map, to allow calling it in the test.

then we just inspect the populated instance IDs in the table e.g.select r from [select count(*) as r, instance_id from t group by instance_id] should look like 3, 3?

The consensus from offline discussions has been to leave verification of distribution in the test as a TODO for now since tracing is broken for import and fixing it would be a bit of effort, as would David's suggestion, and EXPLAIN doesn't work well for bulk ops. I think using tracing would be ideal since it would translate to backup/restore well, as well.


pkg/ccl/importccl/import_stmt_test.go, line 6951 at r8 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

I recently encountered a case where SHOW TRACE FOR SESSION also didn't include some of the events that I knew for sure occurred. I talked with Andrei about it, he suggested that the session tracing might be broken and he wants to get rid off it.

First, I'd use EXPLAIN ANALYZE (DEBUG) on the query for which you want to see the trace to make sure that events do get into the trace there. Next, I'd refactor the test to use WithStatementTrace testing knob (take a look at TestLargeKeys in kvstreamer/large_keys_test.go for an example).

Thanks for the tips.

@rharding6373 rharding6373 requested a review from a team February 23, 2022 23:46
Copy link
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.

Reviewed 5 of 19 files at r3, 7 of 16 files at r5, 15 of 15 files at r9, 6 of 6 files at r10, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @miretskiy, @RaduBerinde, @rharding6373, and @stevendanna)


-- commits, line 29 at r10:
nit: s/Theses/These/.


pkg/sql/apply_join.go, line 309 at r9 (raw file):

		params.ctx, &plannerCopy, plannerCopy.execCfg.NodeID, plannerCopy.SessionData().DistSQLMode, plan.main,
	)
	var distributeType DistributionType

nit: slightly more concise version which seems to be more common in our codebase:

distributeType := DistributionTypeNode
if distributePlan.WillDistribute() {
  distributeType = DistributionTypeSystemTenantOnly
}

pkg/sql/importer/import_processor.go, line 185 at r10 (raw file):

	ctx = idp.StartInternal(ctx, readImportDataProcessorName)
	log.Infof(ctx, "starting read import")
	log.VEvent(ctx, 2, "starting read import")

These seem redundant, no?

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch from 90e6636 to d9629c7 Compare February 24, 2022 06:43
Copy link
Collaborator Author

@rharding6373 rharding6373 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 @miretskiy, @RaduBerinde, @rharding6373, @stevendanna, and @yuzefovich)


pkg/sql/apply_join.go, line 309 at r9 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: slightly more concise version which seems to be more common in our codebase:

distributeType := DistributionTypeNode
if distributePlan.WillDistribute() {
  distributeType = DistributionTypeSystemTenantOnly
}

Done.


pkg/sql/importer/import_processor.go, line 185 at r10 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

These seem redundant, no?

A bit, I suppose. Removed.

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch from d9629c7 to 5e97873 Compare February 24, 2022 17:22
Copy link
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.

:lgtm:

Reviewed 9 of 9 files at r11, 3 of 4 files at r12, 1 of 1 files at r13, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @miretskiy, @RaduBerinde, @rharding6373, and @stevendanna)


pkg/sql/distsql_plan_ctas.go, line 34 at r13 (raw file):

	recv *DistSQLReceiver,
) {
	var distribute DistributionType

nit: there are a couple more places we could make things more concise and consistent.

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch 2 times, most recently from 0d605ba to b7eff0f Compare February 24, 2022 20:42
Copy link
Collaborator Author

@rharding6373 rharding6373 left a comment

Choose a reason for hiding this comment

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

TFTRs and all the help getting this together! Refactored the backup/restore test due to some resource limitations and disable race testing to avoid flakiness due to the test size. It looks like remaining failures have been known flakes.

bors r+

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @miretskiy, @RaduBerinde, @rharding6373, @stevendanna, and @yuzefovich)


pkg/sql/distsql_plan_ctas.go, line 34 at r13 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: there are a couple more places we could make things more concise and consistent.

Thanks, looks like I missed a bunch...

@craig
Copy link
Contributor

craig bot commented Feb 25, 2022

Build failed (retrying...):

@craig
Copy link
Contributor

craig bot commented Feb 25, 2022

Build failed:

@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch from b7eff0f to 1d6bcf9 Compare February 25, 2022 02:23
@rharding6373
Copy link
Collaborator Author

Failed due to a flaky test.

bors r+

@craig
Copy link
Contributor

craig bot commented Feb 25, 2022

Merge conflict.

Previously, distsql did not support distributed queries in multi-tenant
environments. Queries were always planned using a single SQL pod, even
if using multiple pods could yield better parallelism.

This change adds an interface inside distsql that allows different
implementations for functions that differ between single- and
multi-tenant. Currently, it only has `SetupAllNodesPlanning`, which for
single-tenant remains the same, but in multi-tenant returns all
available SQL pods at the time it is invoked.

This PR also fixes some places where SQL pods need to communicate with
other SQL pods using the PodNodeDialer instead of the NodeDialer that
handles SQL <-> KV communication in multi-tenant mode.

Fixes cockroachdb#74548

References cockroachdb#47900

Release note (sql change): Adds support for distributed import queries in
multi-tenant environments, which allows import queries to have improved
parallelism by utilizing all available SQL pods in the tenant.
…le SQL pods

These tests validate that imports and restores can be
distributed across multiple SQL pods in a single tenant.

Release note: None
@rharding6373 rharding6373 force-pushed the distsql_multi_tenant_bulkio branch from 1d6bcf9 to 6b556c9 Compare February 25, 2022 16:57
@rharding6373
Copy link
Collaborator Author

bors r+

@craig
Copy link
Contributor

craig bot commented Feb 25, 2022

Build succeeded:

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: enable simple DistSQL distribution for IMPORT in multi-tenant configurations
6 participants