Skip to content

Commit

Permalink
sql: add ALTER RANGE RELOCATE
Browse files Browse the repository at this point in the history
Fixes #54971

This commit introduces a new ALTER RANGE RELOCATE
command, which will allow an admin to move a lease
or replica for a specific range. Unlike ALTER TABLE RELOCATE
this command works on a range_id, which makes it a lot easier
to use since the user does not have to worry about range keys, which
are difficult to deal with in an emergncy situation.

Release note (sql change): Introduce new SQL syntax ALTER RANGE RELOCATE
to move a lease or replica between stores. This is helpful in an emergency
situation to relocate data in the cluster.
  • Loading branch information
lunevalex committed Nov 22, 2021
1 parent 4c8c0d4 commit da69e5f
Show file tree
Hide file tree
Showing 37 changed files with 769 additions and 15 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/bnf/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
3 changes: 3 additions & 0 deletions docs/generated/sql/bnf/alter_range_relocate_lease_stmt.bnf
Original file line number Diff line number Diff line change
@@ -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
5 changes: 5 additions & 0 deletions docs/generated/sql/bnf/alter_range_relocate_stmt.bnf
Original file line number Diff line number Diff line change
@@ -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
2 changes: 2 additions & 0 deletions docs/generated/sql/bnf/alter_range_stmt.bnf
Original file line number Diff line number Diff line change
@@ -1,2 +1,4 @@
alter_range_stmt ::=
alter_zone_range_stmt
| alter_range_relocate_lease_stmt
| alter_range_relocate_stmt
22 changes: 22 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -1085,6 +1085,7 @@ unreserved_keyword ::=
| 'REGIONS'
| 'REINDEX'
| 'RELEASE'
| 'RELOCATE'
| 'RENAME'
| 'REPEATABLE'
| 'REPLACE'
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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'
Expand Down
12 changes: 12 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
44 changes: 44 additions & 0 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)...))
}
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/catalog/colinfo/result_columns.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/distsql_spec_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/opt/exec/execbuilder/relational.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/opt/exec/execbuilder/statement.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/opt/exec/explain/emit.go
Original file line number Diff line number Diff line change
Expand Up @@ -271,6 +271,7 @@ func (e *emitter) nodeName(n *Node) (string, error) {
}

var nodeNames = [...]string{
alterRangeRelocateOp: "relocate",
alterTableRelocateOp: "relocate",
alterTableSplitOp: "split",
alterTableUnsplitAllOp: "unsplit all",
Expand Down Expand Up @@ -847,6 +848,7 @@ func (e *emitter) emitNodeAttributes(n *Node) error {
alterTableUnsplitOp,
alterTableUnsplitAllOp,
alterTableRelocateOp,
alterRangeRelocateOp,
controlJobsOp,
controlSchedulesOp,
cancelQueriesOp,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/exec/explain/plan_gist_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/opt/exec/explain/result_columns.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,9 @@ func getResultColumns(
case alterTableRelocateOp:
return colinfo.AlterTableRelocateColumns, nil

case alterRangeRelocateOp:
return colinfo.AlterRangeRelocateColumns, nil

case exportOp:
return colinfo.ExportColumns, nil

Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/opt/exec/factory.opt
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/expr_format.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/opt/memo/logical_props_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/opt/ops/statement.opt
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
1 change: 1 addition & 0 deletions pkg/sql/opt/optbuilder/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
63 changes: 63 additions & 0 deletions pkg/sql/opt/optbuilder/alter_range.go
Original file line number Diff line number Diff line change
@@ -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
}
5 changes: 4 additions & 1 deletion pkg/sql/opt/optbuilder/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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(),
Expand Down Expand Up @@ -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)

Expand Down
Loading

0 comments on commit da69e5f

Please sign in to comment.