diff --git a/docs/generated/sql/bnf/BUILD.bazel b/docs/generated/sql/bnf/BUILD.bazel index 41413203ce8e..8f80a01399d4 100644 --- a/docs/generated/sql/bnf/BUILD.bazel +++ b/docs/generated/sql/bnf/BUILD.bazel @@ -18,6 +18,8 @@ FILES = [ "alter_index_stmt", "alter_partition_stmt", "alter_primary_key", + "alter_range_relocate_lease_stmt", + "alter_range_relocate_stmt", "alter_range_stmt", "alter_rename_view_stmt", "alter_role_stmt", diff --git a/docs/generated/sql/bnf/alter_range_relocate_lease_stmt.bnf b/docs/generated/sql/bnf/alter_range_relocate_lease_stmt.bnf new file mode 100644 index 000000000000..95b9c4bea0c7 --- /dev/null +++ b/docs/generated/sql/bnf/alter_range_relocate_lease_stmt.bnf @@ -0,0 +1,3 @@ +alter_range_relocate_lease_stmt ::= + 'ALTER' 'RANGE' relocate_kw 'LEASE' 'TO' iconst64 'FOR' select_stmt + | 'ALTER' 'RANGE' iconst64 relocate_kw 'LEASE' 'TO' iconst64 diff --git a/docs/generated/sql/bnf/alter_range_relocate_stmt.bnf b/docs/generated/sql/bnf/alter_range_relocate_stmt.bnf new file mode 100644 index 000000000000..af265852b67f --- /dev/null +++ b/docs/generated/sql/bnf/alter_range_relocate_stmt.bnf @@ -0,0 +1,5 @@ +alter_range_relocate_stmt ::= + 'ALTER' 'RANGE' relocate_kw voters_kw 'FROM' iconst64 'TO' iconst64 'FOR' select_stmt + | 'ALTER' 'RANGE' iconst64 relocate_kw voters_kw 'FROM' iconst64 'TO' iconst64 + | 'ALTER' 'RANGE' relocate_kw 'NON_VOTERS' 'FROM' iconst64 'TO' iconst64 'FOR' select_stmt + | 'ALTER' 'RANGE' iconst64 relocate_kw 'NON_VOTERS' 'FROM' iconst64 'TO' iconst64 diff --git a/docs/generated/sql/bnf/alter_range_stmt.bnf b/docs/generated/sql/bnf/alter_range_stmt.bnf index ee71f8482572..f77f9af42222 100644 --- a/docs/generated/sql/bnf/alter_range_stmt.bnf +++ b/docs/generated/sql/bnf/alter_range_stmt.bnf @@ -1,2 +1,4 @@ alter_range_stmt ::= alter_zone_range_stmt + | alter_range_relocate_lease_stmt + | alter_range_relocate_stmt diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index f95a931f3590..0f176f3c99e6 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1085,6 +1085,7 @@ unreserved_keyword ::= | 'REGIONS' | 'REINDEX' | 'RELEASE' + | 'RELOCATE' | 'RENAME' | 'REPEATABLE' | 'REPLACE' @@ -1343,6 +1344,8 @@ alter_database_stmt ::= alter_range_stmt ::= alter_zone_range_stmt + | alter_range_relocate_lease_stmt + | alter_range_relocate_stmt alter_partition_stmt ::= alter_zone_partition_stmt @@ -1833,6 +1836,16 @@ alter_database_primary_region_stmt ::= alter_zone_range_stmt ::= 'ALTER' 'RANGE' zone_name set_zone_config +alter_range_relocate_lease_stmt ::= + 'ALTER' 'RANGE' relocate_kw 'LEASE' 'TO' iconst64 'FOR' select_stmt + | 'ALTER' 'RANGE' iconst64 relocate_kw 'LEASE' 'TO' iconst64 + +alter_range_relocate_stmt ::= + 'ALTER' 'RANGE' relocate_kw voters_kw 'FROM' iconst64 'TO' iconst64 'FOR' select_stmt + | 'ALTER' 'RANGE' iconst64 relocate_kw voters_kw 'FROM' iconst64 'TO' iconst64 + | 'ALTER' 'RANGE' relocate_kw 'NON_VOTERS' 'FROM' iconst64 'TO' iconst64 'FOR' select_stmt + | 'ALTER' 'RANGE' iconst64 relocate_kw 'NON_VOTERS' 'FROM' iconst64 'TO' iconst64 + alter_zone_partition_stmt ::= 'ALTER' 'PARTITION' partition_name 'OF' 'TABLE' table_name set_zone_config | 'ALTER' 'PARTITION' partition_name 'OF' 'INDEX' table_index_name set_zone_config @@ -2307,6 +2320,15 @@ survival_goal_clause ::= primary_region_clause ::= 'PRIMARY' 'REGION' opt_equal region_name +relocate_kw ::= + 'TESTING_RELOCATE' + | 'EXPERIMENTAL_RELOCATE' + | 'RELOCATE' + +voters_kw ::= + 'VOTERS' + | + alter_default_privileges_target_object ::= 'TABLES' | 'SEQUENCES' diff --git a/pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported b/pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported index 0dd834618590..1eec331aca5f 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported +++ b/pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported @@ -66,6 +66,18 @@ ALTER TABLE kv EXPERIMENTAL_RELOCATE LEASE VALUES (1, 'k') statement error operation is unsupported in multi-tenancy mode SELECT crdb_internal.check_consistency(true, '', '') +statement error operation is unsupported in multi-tenancy mode +ALTER RANGE 1 RELOCATE LEASE TO 2 + +statement error operation is unsupported in multi-tenancy mode +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT range_id from crdb_internal.ranges where table_name = 'kv' + +statement error operation is unsupported in multi-tenancy mode +ALTER RANGE 1 RELOCATE FROM 1 TO 2 + +statement error operation is unsupported in multi-tenancy mode +ALTER RANGE RELOCATE FROM 1 TO 2 FOR SELECT range_id from crdb_internal.ranges where table_name = 'kv' + # Can not query inflight traces on sql pods statement error table crdb_internal.cluster_inflight_traces is not implemented on tenants diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index 795f7099bfa8..94449910f5e1 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -948,3 +948,47 @@ func TestLeasesDontThrashWhenNodeBecomesSuspect(t *testing.T) { return errors.Errorf("Expected server 1 to have at lease 1 lease.") }) } + +// TestAlterRangeRelocate verifies that the ALTER_RANGE commands work as expected. +func TestAlterRangeRelocate(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numStores = 6 + tc := testcluster.StartTestCluster(t, numStores, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + }, + ) + defer tc.Stopper().Stop(ctx) + + _, rhsDesc := tc.SplitRangeOrFatal(t, keys.UserTableDataMin) + tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Targets(1, 2)...) + + // We start with having the range under test on (1,2,3). + db := tc.ServerConn(0) + // Move 2 -> 4. + _, err := db.Exec("ALTER RANGE " + rhsDesc.RangeID.String() + " RELOCATE FROM 2 TO 4") + require.NoError(t, err) + require.NoError(t, tc.WaitForVoters(rhsDesc.StartKey.AsRawKey(), tc.Targets(0, 2, 3)...)) + // Move lease 1 -> 4. + _, err = db.Exec("ALTER RANGE " + rhsDesc.RangeID.String() + " RELOCATE LEASE TO 4") + require.NoError(t, err) + testutils.SucceedsSoon(t, func() error { + repl := tc.GetFirstStoreFromServer(t, 3).LookupReplica(rhsDesc.StartKey) + if !repl.OwnsValidLease(ctx, tc.Servers[0].Clock().NowAsClockTimestamp()) { + return errors.Errorf("Expected lease to transfer to node 4") + } + // Do this to avoid snapshot problems below when we do another replica move. + if repl != tc.GetRaftLeader(t, rhsDesc.StartKey) { + return errors.Errorf("Expected node 4 to be the raft leader") + } + return nil + }) + + // Move lease 3 -> 5. + _, err = db.Exec("ALTER RANGE RELOCATE FROM 3 TO 5 FOR (SELECT range_id from crdb_internal.ranges where range_id = " + rhsDesc.RangeID.String() + ")") + require.NoError(t, err) + require.NoError(t, tc.WaitForVoters(rhsDesc.StartKey.AsRawKey(), tc.Targets(0, 3, 4)...)) +} diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 5c6318c11407..a81e5325764b 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -150,6 +150,7 @@ go_library( "refresh_materialized_view.go", "region_util.go", "relocate.go", + "relocate_range.go", "rename_column.go", "rename_database.go", "rename_index.go", diff --git a/pkg/sql/catalog/colinfo/result_columns.go b/pkg/sql/catalog/colinfo/result_columns.go index 18004ae197a0..c6f77d5fcbc2 100644 --- a/pkg/sql/catalog/colinfo/result_columns.go +++ b/pkg/sql/catalog/colinfo/result_columns.go @@ -222,6 +222,14 @@ var AlterTableScatterColumns = ResultColumns{ {Name: "pretty", Typ: types.String}, } +// AlterRangeRelocateColumns are the result columns of an +// ALTER RANGE .. RELOCATE statement. +var AlterRangeRelocateColumns = ResultColumns{ + {Name: "range_id", Typ: types.Int}, + {Name: "pretty", Typ: types.String}, + {Name: "result", Typ: types.String}, +} + // ScrubColumns are the result columns of a SCRUB statement. var ScrubColumns = ResultColumns{ {Name: "job_uuid", Typ: types.Uuid}, diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 8d8cdc49ccd7..19726d4c1851 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -1075,6 +1075,12 @@ func (e *distSQLSpecExecFactory) ConstructAlterTableRelocate( return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning: alter table relocate") } +func (e *distSQLSpecExecFactory) ConstructAlterRangeRelocate( + input exec.Node, relocateLease bool, relocateNonVoters bool, toStoreID int64, fromStoreID int64, +) (exec.Node, error) { + return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning: alter range relocate") +} + func (e *distSQLSpecExecFactory) ConstructBuffer(input exec.Node, label string) (exec.Node, error) { return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning: buffer") } diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 1a5acc16d72b..621b47301c3e 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -313,6 +313,9 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) { case *memo.AlterTableRelocateExpr: ep, err = b.buildAlterTableRelocate(t) + case *memo.AlterRangeRelocateExpr: + ep, err = b.buildAlterRangeRelocate(t) + case *memo.ControlJobsExpr: ep, err = b.buildControlJobs(t) diff --git a/pkg/sql/opt/exec/execbuilder/statement.go b/pkg/sql/opt/exec/execbuilder/statement.go index c3a493758a85..1be5ab973159 100644 --- a/pkg/sql/opt/exec/execbuilder/statement.go +++ b/pkg/sql/opt/exec/execbuilder/statement.go @@ -231,6 +231,24 @@ func (b *Builder) buildAlterTableRelocate(relocate *memo.AlterTableRelocateExpr) return planWithColumns(node, relocate.Columns), nil } +func (b *Builder) buildAlterRangeRelocate(relocate *memo.AlterRangeRelocateExpr) (execPlan, error) { + input, err := b.buildRelational(relocate.Input) + if err != nil { + return execPlan{}, err + } + node, err := b.factory.ConstructAlterRangeRelocate( + input.root, + relocate.RelocateLease, + relocate.RelocateNonVoters, + relocate.ToStoreID, + relocate.FromStoreID, + ) + if err != nil { + return execPlan{}, err + } + return planWithColumns(node, relocate.Columns), nil +} + func (b *Builder) buildControlJobs(ctl *memo.ControlJobsExpr) (execPlan, error) { input, err := b.buildRelational(ctl.Input) if err != nil { diff --git a/pkg/sql/opt/exec/explain/emit.go b/pkg/sql/opt/exec/explain/emit.go index f46ed407aad5..1b6627c9b9e7 100644 --- a/pkg/sql/opt/exec/explain/emit.go +++ b/pkg/sql/opt/exec/explain/emit.go @@ -271,6 +271,7 @@ func (e *emitter) nodeName(n *Node) (string, error) { } var nodeNames = [...]string{ + alterRangeRelocateOp: "relocate", alterTableRelocateOp: "relocate", alterTableSplitOp: "split", alterTableUnsplitAllOp: "unsplit all", @@ -847,6 +848,7 @@ func (e *emitter) emitNodeAttributes(n *Node) error { alterTableUnsplitOp, alterTableUnsplitAllOp, alterTableRelocateOp, + alterRangeRelocateOp, controlJobsOp, controlSchedulesOp, cancelQueriesOp, diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index 93b2b916f70b..4b95f19259eb 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -30,7 +30,7 @@ import ( ) func init() { - if numOperators != 57 { + if numOperators != 58 { // If this error occurs please make sure the new op is the last one in order // to not invalidate existing plan gists/hashes. If we are just adding an // operator at the end there's no need to update version below and we can diff --git a/pkg/sql/opt/exec/explain/result_columns.go b/pkg/sql/opt/exec/explain/result_columns.go index 1945780a9fd0..5ae050ba4f8d 100644 --- a/pkg/sql/opt/exec/explain/result_columns.go +++ b/pkg/sql/opt/exec/explain/result_columns.go @@ -162,6 +162,9 @@ func getResultColumns( case alterTableRelocateOp: return colinfo.AlterTableRelocateColumns, nil + case alterRangeRelocateOp: + return colinfo.AlterRangeRelocateColumns, nil + case exportOp: return colinfo.ExportColumns, nil diff --git a/pkg/sql/opt/exec/factory.opt b/pkg/sql/opt/exec/factory.opt index 9bc1f03f5d3e..e21b5b0bfb72 100644 --- a/pkg/sql/opt/exec/factory.opt +++ b/pkg/sql/opt/exec/factory.opt @@ -731,3 +731,12 @@ define Export { FileFormat string Options []exec.KVOption } + +# AlterTableRelocate implements ALTER RANGE RELOCATE. +define AlterRangeRelocate { + input exec.Node + relocateLease bool + relocateNonVoters bool + toStoreID int64 + fromStoreID int64 +} diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index b3a5b9b5cd0e..eda2076eefa3 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -211,7 +211,7 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { *InsertExpr, *UpdateExpr, *UpsertExpr, *DeleteExpr, *SequenceSelectExpr, *WindowExpr, *OpaqueRelExpr, *OpaqueMutationExpr, *OpaqueDDLExpr, *AlterTableSplitExpr, *AlterTableUnsplitExpr, *AlterTableUnsplitAllExpr, - *AlterTableRelocateExpr, *ControlJobsExpr, *CancelQueriesExpr, + *AlterTableRelocateExpr, *AlterRangeRelocateExpr, *ControlJobsExpr, *CancelQueriesExpr, *CancelSessionsExpr, *CreateViewExpr, *ExportExpr: fmt.Fprintf(f.Buffer, "%v", e.Op()) FormatPrivate(f, e.Private(), required) diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 196d3d8d520f..6588f0f003ee 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -1034,6 +1034,12 @@ func (b *logicalPropsBuilder) buildAlterTableRelocateProps( b.buildBasicProps(relocate, relocate.Columns, rel) } +func (b *logicalPropsBuilder) buildAlterRangeRelocateProps( + relocate *AlterRangeRelocateExpr, rel *props.Relational, +) { + b.buildBasicProps(relocate, relocate.Columns, rel) +} + func (b *logicalPropsBuilder) buildControlJobsProps(ctl *ControlJobsExpr, rel *props.Relational) { b.buildBasicProps(ctl, opt.ColList{}, rel) } diff --git a/pkg/sql/opt/ops/statement.opt b/pkg/sql/opt/ops/statement.opt index 015914d251ed..e6d8e7f8f6b0 100644 --- a/pkg/sql/opt/ops/statement.opt +++ b/pkg/sql/opt/ops/statement.opt @@ -292,3 +292,25 @@ define CreateStatisticsPrivate { # Syntax is the tree.CreateStats AST node. Syntax CreateStats } + +# AlterTableRelocateRange represents an `ALTER RANGE .. RELOCATE ..` statement. +[Relational, Mutation] +define AlterRangeRelocate { + # The input expression provides range IDs as integers. + Input RelExpr + _ AlterRangeRelocatePrivate +} + +[Private] +define AlterRangeRelocatePrivate { + RelocateLease bool + RelocateNonVoters bool + ToStoreID int64 + FromStoreID int64 + + # Columns stores the column IDs for the statement result columns. + Columns ColList + + # Props stores the required physical properties for the input expression. + Props PhysProps +} diff --git a/pkg/sql/opt/optbuilder/BUILD.bazel b/pkg/sql/opt/optbuilder/BUILD.bazel index 82f87dea5f60..b76c8761005d 100644 --- a/pkg/sql/opt/optbuilder/BUILD.bazel +++ b/pkg/sql/opt/optbuilder/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "optbuilder", srcs = [ + "alter_range.go", "alter_table.go", "arbiter_set.go", "builder.go", diff --git a/pkg/sql/opt/optbuilder/alter_range.go b/pkg/sql/opt/optbuilder/alter_range.go new file mode 100644 index 000000000000..d4603ae44e8d --- /dev/null +++ b/pkg/sql/opt/optbuilder/alter_range.go @@ -0,0 +1,63 @@ +// Copyright 2018 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package optbuilder + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" + "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" +) + +// buildAlterTableRelocate builds an ALTER RANGE RELOCATE (LEASE). +func (b *Builder) buildAlterRangeRelocate( + relocate *tree.RelocateRange, inScope *scope, +) (outScope *scope) { + + if err := b.catalog.RequireAdminRole(b.ctx, "ALTER RANGE RELOCATE"); err != nil { + panic(err) + } + + // Disable optimizer caching, as we do for other ALTER statements. + b.DisableMemoReuse = true + + outScope = inScope.push() + b.synthesizeResultColumns(outScope, colinfo.AlterTableRelocateColumns) + + cmdName := "RELOCATE" + if relocate.RelocateLease { + cmdName += " LEASE" + } + colNames := []string{"range ids"} + colTypes := []*types.T{types.Int} + + outScope = inScope.push() + b.synthesizeResultColumns(outScope, colinfo.AlterRangeRelocateColumns) + + // We don't allow the input statement to reference outer columns, so we + // pass a "blank" scope rather than inScope. + inputScope := b.buildStmt(relocate.Rows, colTypes, b.allocScope()) + checkInputColumns(cmdName, inputScope, colNames, colTypes, 1) + + outScope.expr = b.factory.ConstructAlterRangeRelocate( + inputScope.expr.(memo.RelExpr), + &memo.AlterRangeRelocatePrivate{ + RelocateLease: relocate.RelocateLease, + RelocateNonVoters: relocate.RelocateNonVoters, + ToStoreID: relocate.ToStoreID, + FromStoreID: relocate.FromStoreID, + Columns: colsToColList(outScope.cols), + Props: physical.MinRequired, + }, + ) + return outScope +} diff --git a/pkg/sql/opt/optbuilder/builder.go b/pkg/sql/opt/optbuilder/builder.go index 6c6a3ab94993..870eeea8b71b 100644 --- a/pkg/sql/opt/optbuilder/builder.go +++ b/pkg/sql/opt/optbuilder/builder.go @@ -268,7 +268,7 @@ func (b *Builder) buildStmt( // A blocklist of statements that can't be used from inside a view. switch stmt := stmt.(type) { case *tree.Delete, *tree.Insert, *tree.Update, *tree.CreateTable, *tree.CreateView, - *tree.Split, *tree.Unsplit, *tree.Relocate, + *tree.Split, *tree.Unsplit, *tree.Relocate, *tree.RelocateRange, *tree.ControlJobs, *tree.ControlSchedules, *tree.CancelQueries, *tree.CancelSessions: panic(pgerror.Newf( pgcode.Syntax, "%s cannot be used inside a view definition", stmt.StatementTag(), @@ -323,6 +323,9 @@ func (b *Builder) buildStmt( case *tree.Relocate: return b.buildAlterTableRelocate(stmt, inScope) + case *tree.RelocateRange: + return b.buildAlterRangeRelocate(stmt, inScope) + case *tree.ControlJobs: return b.buildControlJobs(stmt, inScope) diff --git a/pkg/sql/opt/optbuilder/testdata/alter_range b/pkg/sql/opt/optbuilder/testdata/alter_range new file mode 100644 index 000000000000..0f7b64949019 --- /dev/null +++ b/pkg/sql/opt/optbuilder/testdata/alter_range @@ -0,0 +1,77 @@ +exec-ddl +CREATE TABLE abc (a INT PRIMARY KEY, b INT, c STRING, INDEX b (b), UNIQUE INDEX bc (b,c)) +---- + +# Tests for ALTER RANGE RELOCATE. +build +ALTER RANGE 1 RELOCATE FROM 1 TO 2 +---- +alter-range-relocate &{false false 2 1 [3 4 5] []} + ├── columns: range_id:3 pretty:4 result:5 + └── values + ├── columns: column1:6!null + └── (1,) + +build +ALTER RANGE RELOCATE FROM 1 TO 2 FOR SELECT a from abc +---- +alter-range-relocate &{false false 2 1 [3 4 5] []} + ├── columns: range_id:3 pretty:4 result:5 + └── project + ├── columns: a:6!null + └── scan abc + └── columns: a:6!null b:7 c:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + +build +ALTER RANGE a RELOCATE FROM 1 TO 2 +---- +error (42601): at or near "relocate": syntax error + +build +ALTER RANGE RELOCATE FROM 1 TO 2 FOR SELECT c from abc +---- +error (42601): RELOCATE data column 1 (range ids) must be of type int, not type string + +build +ALTER RANGE 1 RELOCATE NON_VOTERS FROM 1 TO 2 +---- +alter-range-relocate &{false true 2 1 [3 4 5] []} + ├── columns: range_id:3 pretty:4 result:5 + └── values + ├── columns: column1:6!null + └── (1,) + +build +ALTER RANGE RELOCATE NON_VOTERS FROM 1 TO 2 FOR SELECT a from abc +---- +alter-range-relocate &{false true 2 1 [3 4 5] []} + ├── columns: range_id:3 pretty:4 result:5 + └── project + ├── columns: a:6!null + └── scan abc + └── columns: a:6!null b:7 c:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + +# Tests for ALTER RANGE RELOCATE LEASE. +build +ALTER RANGE 1 RELOCATE LEASE TO 2 +---- +alter-range-relocate &{true false 2 0 [3 4 5] []} + ├── columns: range_id:3 pretty:4 result:5 + └── values + ├── columns: column1:6!null + └── (1,) + +build +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT a from abc +---- +alter-range-relocate &{true false 2 0 [3 4 5] []} + ├── columns: range_id:3 pretty:4 result:5 + └── project + ├── columns: a:6!null + └── scan abc + └── columns: a:6!null b:7 c:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + +build +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT c from abc +---- +error (42601): RELOCATE LEASE data column 1 (range ids) must be of type int, not type string diff --git a/pkg/sql/opt/ordering/ordering.go b/pkg/sql/opt/ordering/ordering.go index 3542475cc6d3..eafe8c817895 100644 --- a/pkg/sql/opt/ordering/ordering.go +++ b/pkg/sql/opt/ordering/ordering.go @@ -256,6 +256,11 @@ func init() { buildChildReqOrdering: alterTableRelocateBuildChildReqOrdering, buildProvidedOrdering: noProvidedOrdering, } + funcMap[opt.AlterRangeRelocateOp] = funcs{ + canProvideOrdering: canNeverProvideOrdering, + buildChildReqOrdering: alterRangeRelocateBuildChildReqOrdering, + buildProvidedOrdering: noProvidedOrdering, + } funcMap[opt.ControlJobsOp] = funcs{ canProvideOrdering: canNeverProvideOrdering, buildChildReqOrdering: controlJobsBuildChildReqOrdering, diff --git a/pkg/sql/opt/ordering/statement.go b/pkg/sql/opt/ordering/statement.go index 235f880ce4ed..57b1e73a126e 100644 --- a/pkg/sql/opt/ordering/statement.go +++ b/pkg/sql/opt/ordering/statement.go @@ -48,6 +48,15 @@ func alterTableRelocateBuildChildReqOrdering( return parent.(*memo.AlterTableRelocateExpr).Props.Ordering } +func alterRangeRelocateBuildChildReqOrdering( + parent memo.RelExpr, required *props.OrderingChoice, childIdx int, +) props.OrderingChoice { + if childIdx != 0 { + return props.OrderingChoice{} + } + return parent.(*memo.AlterRangeRelocateExpr).Props.Ordering +} + func controlJobsBuildChildReqOrdering( parent memo.RelExpr, required *props.OrderingChoice, childIdx int, ) props.OrderingChoice { diff --git a/pkg/sql/opt/xform/physical_props.go b/pkg/sql/opt/xform/physical_props.go index 8e000a410943..4d7941279478 100644 --- a/pkg/sql/opt/xform/physical_props.go +++ b/pkg/sql/opt/xform/physical_props.go @@ -67,6 +67,8 @@ func BuildChildPhysicalProps( childProps.Presentation = parent.(*memo.AlterTableUnsplitExpr).Props.Presentation case opt.AlterTableRelocateOp: childProps.Presentation = parent.(*memo.AlterTableRelocateExpr).Props.Presentation + case opt.AlterRangeRelocateOp: + childProps.Presentation = parent.(*memo.AlterRangeRelocateExpr).Props.Presentation case opt.ControlJobsOp: childProps.Presentation = parent.(*memo.ControlJobsExpr).Props.Presentation case opt.CancelQueriesOp: diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index cd8753b72b0b..1175fa84df3c 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -1970,6 +1970,23 @@ func (ef *execFactory) ConstructAlterTableRelocate( }, nil } +// ConstructAlterRangeRelocate is part of the exec.Factory interface. +func (ef *execFactory) ConstructAlterRangeRelocate( + input exec.Node, relocateLease bool, relocateNonVoters bool, toStoreID int64, fromStoreID int64, +) (exec.Node, error) { + if !ef.planner.ExecCfg().Codec.ForSystemTenant() { + return nil, errorutil.UnsupportedWithMultiTenancy(54250) + } + + return &relocateRange{ + rows: input.(planNode), + relocateLease: relocateLease, + relocateNonVoters: relocateNonVoters, + toStoreID: roachpb.StoreID(toStoreID), + fromStoreID: roachpb.StoreID(fromStoreID), + }, nil +} + // ConstructControlJobs is part of the exec.Factory interface. func (ef *execFactory) ConstructControlJobs( command tree.JobCommand, input exec.Node, reason tree.TypedExpr, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index e00f0199767a..d80808c232ac 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -778,7 +778,7 @@ func (u *sqlSymUnion) setVar() *tree.SetVar { %token ELSE ENCODING ENCRYPTION_PASSPHRASE END ENUM ENUMS ESCAPE EXCEPT EXCLUDE EXCLUDING %token EXISTS EXECUTE EXECUTION EXPERIMENTAL %token EXPERIMENTAL_FINGERPRINTS EXPERIMENTAL_REPLICA -%token EXPERIMENTAL_AUDIT +%token EXPERIMENTAL_AUDIT EXPERIMENTAL_RELOCATE %token EXPIRATION EXPLAIN EXPORT EXTENSION EXTRACT EXTRACT_DURATION %token FAILURE FALSE FAMILY FETCH FETCHVAL FETCHTEXT FETCHVAL_PATH FETCHTEXT_PATH @@ -829,7 +829,7 @@ func (u *sqlSymUnion) setVar() *tree.SetVar { %token RANGE RANGES READ REAL REASON REASSIGN RECURSIVE RECURRING REF REFERENCES REFRESH %token REGCLASS REGION REGIONAL REGIONS REGNAMESPACE REGPROC REGPROCEDURE REGROLE REGTYPE REINDEX -%token REMOVE_PATH RENAME REPEATABLE REPLACE REPLICATION +%token RELOCATE REMOVE_PATH RENAME REPEATABLE REPLACE REPLICATION %token RELEASE RESET RESTORE RESTRICT RESTRICTED RESUME RETURNING RETRY REVISION_HISTORY %token REVOKE RIGHT ROLE ROLES ROLLBACK ROLLUP ROUTINES ROW ROWS RSHIFT RULE RUNNING @@ -841,7 +841,7 @@ func (u *sqlSymUnion) setVar() *tree.SetVar { %token START STATISTICS STATUS STDIN STREAM STRICT STRING STORAGE STORE STORED STORING SUBSTRING %token SURVIVE SURVIVAL SYMMETRIC SYNTAX SYSTEM SQRT SUBSCRIPTION STATEMENTS -%token TABLE TABLES TABLESPACE TEMP TEMPLATE TEMPORARY TENANT TESTING_RELOCATE EXPERIMENTAL_RELOCATE TEXT THEN +%token TABLE TABLES TABLESPACE TEMP TEMPLATE TEMPORARY TENANT TESTING_RELOCATE TEXT THEN %token TIES TIME TIMETZ TIMESTAMP TIMESTAMPTZ TO THROTTLING TRAILING TRACE %token TRANSACTION TRANSACTIONS TREAT TRIGGER TRIM TRUE %token TRUNCATE TRUSTED TYPE TYPES @@ -902,6 +902,8 @@ func (u *sqlSymUnion) setVar() *tree.SetVar { // ALTER RANGE %type alter_zone_range_stmt +%type alter_range_relocate_lease_stmt +%type alter_range_relocate_stmt // ALTER TABLE %type alter_onetable_stmt @@ -1778,6 +1780,10 @@ alter_database_primary_region_stmt: // // Commands: // ALTER RANGE ... CONFIGURE ZONE +// ALTER RANGE RELOCATE from to FOR +// ALTER RANGE r RELOCATE from to +// ALTER RANGE RELOCATE LEASE to FOR +// ALTER RANGE r RELOCATE LEASE to // // Zone configurations: // DISCARD @@ -1788,6 +1794,8 @@ alter_database_primary_region_stmt: // %SeeAlso: ALTER TABLE alter_range_stmt: alter_zone_range_stmt +| alter_range_relocate_lease_stmt +| alter_range_relocate_stmt | ALTER RANGE error // SHOW HELP: ALTER RANGE // %Help: ALTER INDEX - change the definition of an index @@ -1903,6 +1911,7 @@ alter_unsplit_index_stmt: relocate_kw: TESTING_RELOCATE | EXPERIMENTAL_RELOCATE +| RELOCATE voters_kw: VOTERS {} @@ -1968,6 +1977,74 @@ alter_zone_range_stmt: $$.val = s } +alter_range_relocate_lease_stmt: + ALTER RANGE relocate_kw LEASE TO iconst64 FOR select_stmt + { + $$.val = &tree.RelocateRange{ + Rows: $8.slct(), + ToStoreID: $6.int64(), + RelocateLease: true, + RelocateNonVoters: false, + } + } +| ALTER RANGE iconst64 relocate_kw LEASE TO iconst64 + { + $$.val = &tree.RelocateRange{ + Rows: &tree.Select{ + Select: &tree.ValuesClause{Rows: []tree.Exprs{tree.Exprs{tree.NewDInt(tree.DInt($3.int64()))}}}, + }, + ToStoreID: $7.int64(), + RelocateLease: true, + RelocateNonVoters: false, + } + } + +alter_range_relocate_stmt: +ALTER RANGE relocate_kw voters_kw FROM iconst64 TO iconst64 FOR select_stmt + { + $$.val = &tree.RelocateRange{ + Rows: $10.slct(), + FromStoreID: $6.int64(), + ToStoreID: $8.int64(), + RelocateLease: false, + RelocateNonVoters: false, + } + } +| ALTER RANGE iconst64 relocate_kw voters_kw FROM iconst64 TO iconst64 + { + $$.val = &tree.RelocateRange{ + Rows: &tree.Select{ + Select: &tree.ValuesClause{Rows: []tree.Exprs{tree.Exprs{tree.NewDInt(tree.DInt($3.int64()))}}}, + }, + FromStoreID: $7.int64(), + ToStoreID: $9.int64(), + RelocateLease: false, + RelocateNonVoters: false, + } + } +| ALTER RANGE relocate_kw NON_VOTERS FROM iconst64 TO iconst64 FOR select_stmt + { + $$.val = &tree.RelocateRange{ + Rows: $10.slct(), + FromStoreID: $6.int64(), + ToStoreID: $8.int64(), + RelocateLease: false, + RelocateNonVoters: true, + } + } +| ALTER RANGE iconst64 relocate_kw NON_VOTERS FROM iconst64 TO iconst64 + { + $$.val = &tree.RelocateRange{ + Rows: &tree.Select{ + Select: &tree.ValuesClause{Rows: []tree.Exprs{tree.Exprs{tree.NewDInt(tree.DInt($3.int64()))}}}, + }, + FromStoreID: $7.int64(), + ToStoreID: $9.int64(), + RelocateLease: false, + RelocateNonVoters: true, + } +} + set_zone_config: CONFIGURE ZONE to_or_eq a_expr { @@ -13350,6 +13427,7 @@ unreserved_keyword: | REGIONS | REINDEX | RELEASE +| RELOCATE | RENAME | REPEATABLE | REPLACE diff --git a/pkg/sql/parser/testdata/alter_range b/pkg/sql/parser/testdata/alter_range index 059c78f4a7c8..0c3f15e53683 100644 --- a/pkg/sql/parser/testdata/alter_range +++ b/pkg/sql/parser/testdata/alter_range @@ -85,3 +85,59 @@ ALTER RANGE meta CONFIGURE ZONE USING DEFAULT ALTER RANGE meta CONFIGURE ZONE USING DEFAULT -- fully parenthesized ALTER RANGE meta CONFIGURE ZONE USING DEFAULT -- literals removed ALTER RANGE _ CONFIGURE ZONE USING DEFAULT -- identifiers removed + +parse +ALTER RANGE 1 RELOCATE LEASE TO 2 +---- +ALTER RANGE RELOCATE LEASE TO 2 FOR VALUES (1) -- normalized! +ALTER RANGE RELOCATE LEASE TO 2 FOR VALUES ((1)) -- fully parenthesized +ALTER RANGE RELOCATE LEASE TO 2 FOR VALUES (_) -- literals removed +ALTER RANGE RELOCATE LEASE TO 2 FOR VALUES (1) -- identifiers removed + +parse +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT range_id FROM foo +---- +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT range_id FROM foo +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT (range_id) FROM foo -- fully parenthesized +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT range_id FROM foo -- literals removed +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT _ FROM _ -- identifiers removed + +parse +ALTER RANGE 1 RELOCATE FROM 3 TO 2 +---- +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR VALUES (1) -- normalized! +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR VALUES ((1)) -- fully parenthesized +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR VALUES (_) -- literals removed +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR VALUES (1) -- identifiers removed + +parse +ALTER RANGE 1 RELOCATE VOTERS FROM 3 TO 2 +---- +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR VALUES (1) -- normalized! +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR VALUES ((1)) -- fully parenthesized +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR VALUES (_) -- literals removed +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR VALUES (1) -- identifiers removed + +parse +ALTER RANGE 1 RELOCATE NON_VOTERS FROM 3 TO 2 +---- +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR VALUES (1) -- normalized! +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR VALUES ((1)) -- fully parenthesized +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR VALUES (_) -- literals removed +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR VALUES (1) -- identifiers removed + +parse +ALTER RANGE RELOCATE FROM 3 TO 2 FOR SELECT range_id FROM foo +---- +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR SELECT range_id FROM foo -- normalized! +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR SELECT (range_id) FROM foo -- fully parenthesized +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR SELECT range_id FROM foo -- literals removed +ALTER RANGE RELOCATE VOTERS FROM 3 TO 2 FOR SELECT _ FROM _ -- identifiers removed + +parse +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR SELECT range_id FROM foo +---- +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR SELECT range_id FROM foo +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR SELECT (range_id) FROM foo -- fully parenthesized +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR SELECT range_id FROM foo -- literals removed +ALTER RANGE RELOCATE NON_VOTERS FROM 3 TO 2 FOR SELECT _ FROM _ -- identifiers removed diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index 32c1f486606c..0c02251090db 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -195,6 +195,7 @@ var _ planNode = &reassignOwnedByNode{} var _ planNode = &refreshMaterializedViewNode{} var _ planNode = &recursiveCTENode{} var _ planNode = &relocateNode{} +var _ planNode = &relocateRange{} var _ planNode = &renameColumnNode{} var _ planNode = &renameDatabaseNode{} var _ planNode = &renameIndexNode{} diff --git a/pkg/sql/plan_columns.go b/pkg/sql/plan_columns.go index 5d10c14d07cd..b620c981e9d9 100644 --- a/pkg/sql/plan_columns.go +++ b/pkg/sql/plan_columns.go @@ -108,6 +108,8 @@ func getPlanColumns(plan planNode, mut bool) colinfo.ResultColumns { return n.getColumns(mut, colinfo.ExplainPlanColumns) case *relocateNode: return n.getColumns(mut, colinfo.AlterTableRelocateColumns) + case *relocateRange: + return n.getColumns(mut, colinfo.AlterRangeRelocateColumns) case *scatterNode: return n.getColumns(mut, colinfo.AlterTableScatterColumns) case *showFingerprintsNode: diff --git a/pkg/sql/relocate.go b/pkg/sql/relocate.go index 7720db1941f2..989e7dc6ee86 100644 --- a/pkg/sql/relocate.go +++ b/pkg/sql/relocate.go @@ -91,17 +91,10 @@ func (n *relocateNode) Next(params runParams) (bool, error) { nodeID, ok := n.run.storeMap[storeID] if !ok { // Lookup the store in gossip. - var storeDesc roachpb.StoreDescriptor - gossipStoreKey := gossip.MakeStoreKey(storeID) - g, err := params.extendedEvalCtx.ExecCfg.Gossip.OptionalErr(54250) + storeDesc, err := lookupStoreDesc(storeID, params) if err != nil { return false, err } - if err := g.GetInfoProto( - gossipStoreKey, &storeDesc, - ); err != nil { - return false, errors.Wrapf(err, "error looking up store %d", storeID) - } nodeID = storeDesc.Node.NodeID n.run.storeMap[storeID] = nodeID } @@ -161,6 +154,21 @@ func (n *relocateNode) Close(ctx context.Context) { n.rows.Close(ctx) } +func lookupStoreDesc(storeID roachpb.StoreID, params runParams) (*roachpb.StoreDescriptor, error) { + var storeDesc roachpb.StoreDescriptor + gossipStoreKey := gossip.MakeStoreKey(storeID) + g, err := params.extendedEvalCtx.ExecCfg.Gossip.OptionalErr(54250) + if err != nil { + return nil, err + } + if err := g.GetInfoProto( + gossipStoreKey, &storeDesc, + ); err != nil { + return nil, errors.Wrapf(err, "error looking up store %d", storeID) + } + return &storeDesc, nil +} + func lookupRangeDescriptor( ctx context.Context, db *kv.DB, rowKey []byte, ) (roachpb.RangeDescriptor, error) { diff --git a/pkg/sql/relocate_range.go b/pkg/sql/relocate_range.go new file mode 100644 index 000000000000..1d51e4f75d25 --- /dev/null +++ b/pkg/sql/relocate_range.go @@ -0,0 +1,198 @@ +// Copyright 2017 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/errors" +) + +type relocateRange struct { + optColumnsSlot + + rows planNode + relocateLease bool + relocateNonVoters bool + toStoreID roachpb.StoreID + fromStoreID roachpb.StoreID + run relocateRunState +} + +// relocateRunState contains the run-time state of +// relocateRange during local execution. +type relocateRunState struct { + toStoreDesc *roachpb.StoreDescriptor + fromStoreDesc *roachpb.StoreDescriptor + results relocateResults +} + +// relocateResults captures the results of the last relocate run +type relocateResults struct { + rangeID roachpb.RangeID + rangeDesc *roachpb.RangeDescriptor + err error +} + +// relocateRequest is an internal data structure that describes a relocation. +type relocateRequest struct { + rangeID roachpb.RangeID + relocateLease bool + relocateNonVoters bool + toStoreDesc *roachpb.StoreDescriptor + fromStoreDesc *roachpb.StoreDescriptor +} + +func (n *relocateRange) startExec(params runParams) error { + if n.toStoreID <= 0 { + return errors.Errorf("invalid target to store ID %d for RELOCATE", n.toStoreID) + } + if !n.relocateLease && n.fromStoreID <= 0 { + return errors.Errorf("invalid target from store ID %d for RELOCATE", n.fromStoreID) + } + // Lookup all the store descriptors upfront, so we dont have to do it for each + // range we are working with. + var err error + n.run.toStoreDesc, err = lookupStoreDesc(n.toStoreID, params) + if err != nil { + return err + } + if !n.relocateLease { + n.run.fromStoreDesc, err = lookupStoreDesc(n.fromStoreID, params) + if err != nil { + return err + } + } + return nil +} + +func (n *relocateRange) Next(params runParams) (bool, error) { + if ok, err := n.rows.Next(params); err != nil || !ok { + return ok, err + } + datum := n.rows.Values()[0] + if datum == tree.DNull { + return true, nil + } + rangeID := roachpb.RangeID(tree.MustBeDInt(datum)) + + rangeDesc, err := relocate(params, relocateRequest{ + rangeID: rangeID, + relocateLease: n.relocateLease, + relocateNonVoters: n.relocateNonVoters, + fromStoreDesc: n.run.fromStoreDesc, + toStoreDesc: n.run.toStoreDesc, + }) + + // record the results of the relocation run, so we can output it. + n.run.results = relocateResults{ + rangeID: rangeID, + rangeDesc: rangeDesc, + err: err, + } + return true, nil +} + +func (n *relocateRange) Values() tree.Datums { + result := "ok" + if n.run.results.err != nil { + result = n.run.results.err.Error() + } + pretty := "" + if n.run.results.rangeDesc != nil { + pretty = keys.PrettyPrint(nil /* valDirs */, n.run.results.rangeDesc.StartKey.AsRawKey()) + } + return tree.Datums{ + tree.NewDInt(tree.DInt(n.run.results.rangeID)), + tree.NewDString(pretty), + tree.NewDString(result), + } +} + +func (n *relocateRange) Close(ctx context.Context) { + n.rows.Close(ctx) +} + +func relocate(params runParams, req relocateRequest, +) (*roachpb.RangeDescriptor, error) { + rangeDesc, err := lookupRangeDescriptorByRangeID(params.ctx, params.extendedEvalCtx.ExecCfg.DB, req.rangeID) + if err != nil { + return nil, errors.Wrapf(err, "error looking up range descriptor") + } + + if req.relocateLease { + err := params.p.ExecCfg().DB.AdminTransferLease(params.ctx, rangeDesc.StartKey, req.toStoreDesc.StoreID) + return rangeDesc, err + } else { + toTarget := roachpb.ReplicationTarget{NodeID: req.toStoreDesc.Node.NodeID, StoreID: req.toStoreDesc.StoreID} + fromTarget := roachpb.ReplicationTarget{NodeID: req.fromStoreDesc.Node.NodeID, StoreID: req.fromStoreDesc.StoreID} + if req.relocateNonVoters { + _, err := params.p.ExecCfg().DB.AdminChangeReplicas( + params.ctx, rangeDesc.StartKey, *rangeDesc, []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_NON_VOTER, Target: toTarget}, + {ChangeType: roachpb.REMOVE_NON_VOTER, Target: fromTarget}, + }, + ) + return rangeDesc, err + } else { + _, err := params.p.ExecCfg().DB.AdminChangeReplicas( + params.ctx, rangeDesc.StartKey, *rangeDesc, []roachpb.ReplicationChange{ + {ChangeType: roachpb.ADD_VOTER, Target: toTarget}, + {ChangeType: roachpb.REMOVE_VOTER, Target: fromTarget}, + }, + ) + return rangeDesc, err + } + } +} + +func lookupRangeDescriptorByRangeID( + ctx context.Context, db *kv.DB, rangeID roachpb.RangeID, +) (*roachpb.RangeDescriptor, error) { + var descriptor roachpb.RangeDescriptor + sentinelErr := errors.Errorf("sentinel") + err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return txn.Iterate(ctx, keys.MetaMin, keys.MetaMax, 100, + func(rows []kv.KeyValue) error { + var desc roachpb.RangeDescriptor + for _, row := range rows { + err := row.ValueProto(&desc) + if err != nil { + return errors.Wrapf(err, "unable to unmarshal range descriptor from %s", row.Key) + } + // In small enough clusters it's possible for the same range + // descriptor to be stored in both meta1 and meta2. This + // happens when some range spans both the meta and the user + // keyspace. Consider when r1 is [/Min, + // /System/NodeLiveness); we'll store the range descriptor + // in both /Meta2/ and in /Meta1/KeyMax[1]. + // + // [1]: See kvserver.rangeAddressing. + if desc.RangeID == rangeID { + descriptor = desc + return sentinelErr + } + } + return nil + }) + }) + if errors.Is(err, sentinelErr) { + return &descriptor, nil + } + if err != nil { + return nil, err + } + return nil, errors.Errorf("Descriptor for range %d is not found", rangeID) +} diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 4ca70455e454..fea320bbabc3 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "alter_database.go", "alter_default_privileges.go", "alter_index.go", + "alter_range.go", "alter_role.go", "alter_schema.go", "alter_sequence.go", diff --git a/pkg/sql/sem/tree/alter_range.go b/pkg/sql/sem/tree/alter_range.go new file mode 100644 index 000000000000..2f2482dac39b --- /dev/null +++ b/pkg/sql/sem/tree/alter_range.go @@ -0,0 +1,44 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tree + +import "strconv" + +// RelocateRange represents an `ALTER RANGE .. RELOCATE ..` +// statement. +type RelocateRange struct { + Rows *Select + ToStoreID int64 + FromStoreID int64 + RelocateLease bool + RelocateNonVoters bool +} + +// Format implements the NodeFormatter interface. +func (node *RelocateRange) Format(ctx *FmtCtx) { + ctx.WriteString("ALTER RANGE RELOCATE ") + if node.RelocateLease { + ctx.WriteString("LEASE ") + } else if node.RelocateNonVoters { + ctx.WriteString("NON_VOTERS ") + } else { + ctx.WriteString("VOTERS ") + } + if !node.RelocateLease { + ctx.WriteString("FROM ") + ctx.WriteString(strconv.FormatInt(node.FromStoreID, 10)) + ctx.WriteString(" ") + } + ctx.WriteString("TO ") + ctx.WriteString(strconv.FormatInt(node.ToStoreID, 10)) + ctx.WriteString(" FOR ") + ctx.FormatNode(node.Rows) +} diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index cadd103b346d..3897300e90d9 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -130,7 +130,7 @@ func CanWriteData(stmt Statement) bool { case *CopyFrom, *Import, *Restore: return true // CockroachDB extensions. - case *Split, *Unsplit, *Relocate, *Scatter: + case *Split, *Unsplit, *Relocate, *RelocateRange, *Scatter: return true } return false @@ -977,6 +977,22 @@ func (n *Relocate) StatementTag() string { return "EXPERIMENTAL_RELOCATE VOTERS" } +// StatementReturnType implements the Statement interface. +func (*RelocateRange) StatementReturnType() StatementReturnType { return Rows } + +// StatementType implements the Statement interface. +func (*RelocateRange) StatementType() StatementType { return TypeDML } + +// StatementTag returns a short string identifying the type of statement. +func (n *RelocateRange) StatementTag() string { + if n.RelocateLease { + return "RELOCATE LEASE" + } else if n.RelocateNonVoters { + return "RELOCATE NON_VOTERS" + } + return "RELOCATE VOTERS" +} + // StatementReturnType implements the Statement interface. func (*ReplicationStream) StatementReturnType() StatementReturnType { return Rows } @@ -1718,6 +1734,7 @@ func (n *Prepare) String() string { return AsString(n) } func (n *ReassignOwnedBy) String() string { return AsString(n) } func (n *ReleaseSavepoint) String() string { return AsString(n) } func (n *Relocate) String() string { return AsString(n) } +func (n *RelocateRange) String() string { return AsString(n) } func (n *RefreshMaterializedView) String() string { return AsString(n) } func (n *RenameColumn) String() string { return AsString(n) } func (n *RenameDatabase) String() string { return AsString(n) } diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 9eaa8b8fcf4c..76d54cdd4721 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -185,6 +185,9 @@ func (v *planVisitor) visitInternal(plan planNode, name string) { case *relocateNode: n.rows = v.visit(n.rows) + case *relocateRange: + n.rows = v.visit(n.rows) + case *insertNode, *insertFastPathNode: if ins, ok := n.(*insertNode); ok { ins.source = v.visit(ins.source) @@ -406,6 +409,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&recursiveCTENode{}): "recursive cte", reflect.TypeOf(&refreshMaterializedViewNode{}): "refresh materialized view", reflect.TypeOf(&relocateNode{}): "relocate", + reflect.TypeOf(&relocateRange{}): "relocate range", reflect.TypeOf(&renameColumnNode{}): "rename column", reflect.TypeOf(&renameDatabaseNode{}): "rename database", reflect.TypeOf(&renameIndexNode{}): "rename index",