Skip to content

Commit 725501d

Browse files
craig[bot]Xiang-Gu
craig[bot]
andcommitted
Merge #109818
109818: schemachanger: Disable ALTER TABLE multiple commands by default r=Xiang-Gu a=Xiang-Gu Disable ALTER TABLE stmt with multiple ADD COLUMN, DROP COLUMN, ALTER PK commands (which is added in #99526) by default. Customers can enable it by setting `use_declarative_schema_changer` to `unsafe` or `unsafe_always`, or forcefully enable declarative schema changer on `ALTER TABLE` stmts by setting cluster setting `sql.schema.force_declarative_statements='+ALTER TABLE';`. The plan is to enable it again by default after v23.2 (be it v24.1 or v23.3). Fixes #108870 Epic: None Release note: None Co-authored-by: Xiang Gu <[email protected]>
2 parents 1f95b88 + 4983c74 commit 725501d

File tree

8 files changed

+206
-36
lines changed

8 files changed

+206
-36
lines changed

pkg/sql/logictest/testdata/logic_test/alter_table

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3186,6 +3186,9 @@ t_96728 t_96728_pkey PRIMARY KEY PRIMARY KEY (i ASC)
31863186
# This is only supported in declarative schema changer starting from V23_2.
31873187
subtest alter_table_with_multiple_commands_99035
31883188

3189+
statement ok
3190+
SET CLUSTER SETTING sql.schema.force_declarative_statements = "+ALTER TABLE";
3191+
31893192
statement ok
31903193
DROP TABLE IF EXISTS t_99035;
31913194
CREATE TABLE t_99035 (i INT PRIMARY KEY, j INT NOT NULL, FAMILY "primary" (i, j));
@@ -3328,6 +3331,11 @@ CREATE TABLE public.t_99035 (
33283331
CONSTRAINT check_j CHECK (j >= 0:::INT8)
33293332
)
33303333

3334+
statement ok
3335+
RESET CLUSTER SETTING sql.schema.force_declarative_statements;
3336+
3337+
subtest end
3338+
33313339
# This subtest disallows using builtin function `cluster_logical_timestamp()`
33323340
# as the default expression when backfilling a column.
33333341
subtest 98269

pkg/sql/logictest/testdata/logic_test/new_schema_changer

Lines changed: 101 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1531,3 +1531,104 @@ SET CLUSTER SETTING sql.schema.force_declarative_statements='+CREATE SEQUENCE'
15311531
skipif config local-mixed-22.2-23.1
15321532
statement ok
15331533
EXPLAIN (DDL) CREATE SEQUENCE sq2
1534+
1535+
subtest end
1536+
1537+
# This subtest ensures ALTER TABLE with multiple ADD COLUMN, DROP COLUMN, and/or
1538+
# ALTER PK is disabled with declarative schema changer by default, unless
1539+
# `use_declarative_schema_changer = (unsafe|unsafe_always)` or
1540+
# `sql.schema.force_declarative_statements='+ALTER TABLE'`.
1541+
# TODO (xiang): Enable it by default on the next major release after v23.2, be it v24.1 or v23.3.
1542+
subtest 108870
1543+
1544+
statement ok
1545+
SET sql_safe_updates = false;
1546+
SET use_declarative_schema_changer = 'on';
1547+
1548+
statement ok
1549+
SET CLUSTER SETTING sql.schema.force_declarative_statements = '';
1550+
1551+
statement ok
1552+
CREATE TABLE t_108870 (i INT PRIMARY KEY, j INT NOT NULL);
1553+
1554+
# Ensure ALTER TABLE with only ADD COLUMN, only DROP COLUMN, or only ALTER PK can use declarative schema changer.
1555+
statement ok
1556+
EXPLAIN (DDL) ALTER TABLE t_108870 DROP COLUMN j;
1557+
1558+
statement ok
1559+
EXPLAIN (DDL) ALTER TABLE t_108870 ADD COLUMN k INT DEFAULT 40;
1560+
1561+
statement ok
1562+
EXPLAIN (DDL) ALTER TABLE t_108870 ADD COLUMN k INT DEFAULT 40, ADD COLUMN p INT DEFAULT 50;
1563+
1564+
statement ok
1565+
EXPLAIN (DDL) ALTER TABLE t_108870 ALTER PRIMARY KEY USING COLUMNS (j);
1566+
1567+
# Ensure ALTER TABLE with multiple ADD COLUMN, DROP COLUMN, and/or ALTER PK cannot use declarative schema changer.
1568+
skipif config local-mixed-22.2-23.1
1569+
statement error pgcode 0A000 cannot explain a statement which is not supported by the declarative schema changer
1570+
EXPLAIN (DDL) ALTER TABLE t_108870 ALTER PRIMARY KEY USING COLUMNS (j), DROP COLUMN i;
1571+
1572+
skipif config local-mixed-22.2-23.1
1573+
statement error pgcode 0A000 cannot explain a statement which is not supported by the declarative schema changer
1574+
EXPLAIN (DDL) ALTER TABLE t_108870 DROP COLUMN j, ADD COLUMN k INT DEFAULT 30;
1575+
1576+
skipif config local-mixed-22.2-23.1
1577+
statement error pgcode 0A000 cannot explain a statement which is not supported by the declarative schema changer
1578+
EXPLAIN (DDL) ALTER TABLE t_108870 ADD COLUMN k INT DEFAULT 30, ALTER PRIMARY KEY USING COLUMNS (j);
1579+
1580+
skipif config local-mixed-22.2-23.1
1581+
statement error pgcode 0A000 cannot explain a statement which is not supported by the declarative schema changer
1582+
EXPLAIN (DDL) ALTER TABLE t_108870 ALTER PRIMARY KEY USING COLUMNS (j), DROP COLUMN i, ADD COLUMN k INT DEFAULT 30;
1583+
1584+
# Ensure we can enable it in declarative schema changer with cluster setting
1585+
# sql.schema.force_declarative_statements.
1586+
skipif config local-mixed-22.2-23.1
1587+
statement ok
1588+
SET CLUSTER SETTING sql.schema.force_declarative_statements='+ALTER TABLE';
1589+
1590+
statement ok
1591+
SET use_declarative_schema_changer = 'on';
1592+
1593+
skipif config local-mixed-22.2-23.1
1594+
statement ok
1595+
EXPLAIN (DDL) ALTER TABLE t_108870 ALTER PRIMARY KEY USING COLUMNS (j), DROP COLUMN i;
1596+
1597+
skipif config local-mixed-22.2-23.1
1598+
statement ok
1599+
EXPLAIN (DDL) ALTER TABLE t_108870 DROP COLUMN j, ADD COLUMN k INT DEFAULT 30;
1600+
1601+
skipif config local-mixed-22.2-23.1
1602+
statement ok
1603+
EXPLAIN (DDL) ALTER TABLE t_108870 ADD COLUMN k INT DEFAULT 30, ALTER PRIMARY KEY USING COLUMNS (j);
1604+
1605+
skipif config local-mixed-22.2-23.1
1606+
statement ok
1607+
EXPLAIN (DDL) ALTER TABLE t_108870 ALTER PRIMARY KEY USING COLUMNS (j), DROP COLUMN i, ADD COLUMN k INT DEFAULT 30;
1608+
1609+
# Ensure we can also enable it in declarative schema changer with session variable
1610+
# use_declarative_schema_changer.
1611+
skipif config local-mixed-22.2-23.1
1612+
statement ok
1613+
SET CLUSTER SETTING sql.schema.force_declarative_statements='';
1614+
1615+
statement ok
1616+
SET use_declarative_schema_changer = 'unsafe_always';
1617+
1618+
skipif config local-mixed-22.2-23.1
1619+
statement ok
1620+
EXPLAIN (DDL) ALTER TABLE t_108870 ALTER PRIMARY KEY USING COLUMNS (j), DROP COLUMN i;
1621+
1622+
skipif config local-mixed-22.2-23.1
1623+
statement ok
1624+
EXPLAIN (DDL) ALTER TABLE t_108870 DROP COLUMN j, ADD COLUMN k INT DEFAULT 30;
1625+
1626+
skipif config local-mixed-22.2-23.1
1627+
statement ok
1628+
EXPLAIN (DDL) ALTER TABLE t_108870 ADD COLUMN k INT DEFAULT 30, ALTER PRIMARY KEY USING COLUMNS (j);
1629+
1630+
skipif config local-mixed-22.2-23.1
1631+
statement ok
1632+
EXPLAIN (DDL) ALTER TABLE t_108870 ALTER PRIMARY KEY USING COLUMNS (j), DROP COLUMN i, ADD COLUMN k INT DEFAULT 30;
1633+
1634+
subtest end

pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go

Lines changed: 49 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -13,12 +13,15 @@ package scbuildstmt
1313
import (
1414
"math"
1515
"reflect"
16+
"strings"
1617

1718
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1819
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
20+
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
1921
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
2022
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
2123
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
24+
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors"
2225
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
2326
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl"
2427
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
@@ -182,6 +185,52 @@ func AlterTable(b BuildCtx, n *tree.AlterTable) {
182185
maybeDropRedundantPrimaryIndexes(b, tbl.TableID)
183186
maybeRewriteTempIDsInPrimaryIndexes(b, tbl.TableID)
184187
disallowDroppingPrimaryIndexReferencedInUDFOrView(b, tbl.TableID, n.String())
188+
// TODO (xiang): Remove the following line for the next major release after v23.2,
189+
// be it v24.1 or v23.3.
190+
disableAlterTableMultipleCommandsOnV232(b, n, tbl.TableID)
191+
}
192+
193+
// disableAlterTableMultipleCommandsOnV232 disables declarative schema changer
194+
// if processing this ALTER TABLE stmt requires building more than one new
195+
// primary indexes by default on v23.2, unless the mode is unsafe or ALTER TABLE
196+
// is forcefully turned on.
197+
func disableAlterTableMultipleCommandsOnV232(b BuildCtx, n *tree.AlterTable, tableID catid.DescID) {
198+
chain := getPrimaryIndexChain(b, tableID)
199+
chainTyp := chain.chainType()
200+
201+
// isAlterPKWithRowid returns true if the stmt is ALTER PK with rowid.
202+
isAlterPKWithRowid := func() bool {
203+
if chainTyp == twoNewPrimaryIndexesWithAlteredPKType {
204+
_, _, inter2StoredCols := getSortedColumnIDsInIndexByKind(b, tableID, chain.inter2Spec.primary.IndexID)
205+
_, _, finalStoredCols := getSortedColumnIDsInIndexByKind(b, tableID, chain.finalSpec.primary.IndexID)
206+
inter2StoredColsAsSet := catalog.MakeTableColSet(inter2StoredCols...)
207+
finalStoredColsAsSet := catalog.MakeTableColSet(finalStoredCols...)
208+
diffSet := inter2StoredColsAsSet.Difference(finalStoredColsAsSet)
209+
if diffSet.Len() == 1 {
210+
colName := mustRetrieveColumnNameElem(b, tableID, diffSet.Ordered()[0]).Name
211+
if strings.HasPrefix(colName, "rowid") {
212+
return true
213+
}
214+
}
215+
}
216+
return false
217+
}
218+
219+
if chainTyp == twoNewPrimaryIndexesWithAlteredPKType ||
220+
chainTyp == twoNewPrimaryIndexesWithAddAndDropColumnsType ||
221+
chainTyp == threeNewPrimaryIndexesType {
222+
if isAlterPKWithRowid() {
223+
// This is the only exception that involves building more than one new
224+
// primary indexes but we would enable by default, because it was already
225+
// supported in v23.1.
226+
return
227+
}
228+
newSchemaChangerMode := getDeclarativeSchemaChangerModeForStmt(b, n)
229+
if newSchemaChangerMode != sessiondatapb.UseNewSchemaChangerUnsafe &&
230+
newSchemaChangerMode != sessiondatapb.UseNewSchemaChangerUnsafeAlways {
231+
panic(scerrors.NotImplementedErrorf(n, "statement has been disabled on v23.2"))
232+
}
233+
}
185234
}
186235

187236
// disallowDroppingPrimaryIndexReferencedInUDFOrView prevents dropping old (current)

pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1506,7 +1506,7 @@ const (
15061506
// A set of five pre-defined acceptable types for primary index chains:
15071507
// 1). noNewPrimaryIndex: "old, nil, nil, nil" (e.g. no add/drop column nor alter PK)
15081508
// 2). oneNewPrimaryIndex: "old, nil, nil, final" (e.g. add column(s), or drop columns(s), or alter PK without rowid)
1509-
// 3). twoNewPrimaryIndexesWithAlteredPK: "old, nil, inter2, final" (e.g. alter PK with rowid)
1509+
// 3). twoNewPrimaryIndexesWithAlteredPK: "old, nil, inter2, final" (e.g. alter PK with rowid, or alter PK + drop column(s))
15101510
// 4). twoNewPrimaryIndexesWithAddAndDropColumns: "old, inter1, nil, final" (e.g. add & drop column(s))
15111511
// 5). threeNewPrimaryIndexes: "old, inter1, inter2, final" (e.g. add/drop column + alter PK)
15121512
type chainType int

pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go

Lines changed: 19 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -166,13 +166,7 @@ func isFullySupportedWithFalsePositiveInternal(
166166
// Process dispatches on the statement type to populate the BuilderState
167167
// embedded in the BuildCtx. Any error will be panicked.
168168
func Process(b BuildCtx, n tree.Statement) {
169-
newSchemaChangerMode := b.EvalCtx().SessionData().NewSchemaChangerMode
170-
// Check if the feature is either forced disabled or enabled,
171-
// using a cluster setting.
172-
disabledStatements := getSchemaChangerStatementControl(&b.ClusterSettings().SV)
173-
if forcedEnabled := disabledStatements.CheckStatementControl(n); forcedEnabled {
174-
newSchemaChangerMode = sessiondatapb.UseNewSchemaChangerUnsafe
175-
}
169+
newSchemaChangerMode := getDeclarativeSchemaChangerModeForStmt(b, n)
176170
// Run a few "quick checks" to see if the statement is not supported.
177171
if !IsFullySupportedWithFalsePositive(n, b.EvalCtx().Settings.Version.ActiveVersion(b),
178172
newSchemaChangerMode) {
@@ -191,6 +185,24 @@ func Process(b BuildCtx, n tree.Statement) {
191185
fn.Call(in)
192186
}
193187

188+
// getDeclarativeSchemaChangerModeForStmt returns the mode specific for `n`.
189+
// It almost always returns value of session variable
190+
// `use_declarative_schema_changer`, unless `n` is forcefully enabled (or
191+
// disabled) via cluster setting `sql.schema.force_declarative_statements`, in
192+
// which case it returns `unsafe` (or `off`).
193+
func getDeclarativeSchemaChangerModeForStmt(
194+
b BuildCtx, n tree.Statement,
195+
) sessiondatapb.NewSchemaChangerMode {
196+
ret := b.EvalCtx().SessionData().NewSchemaChangerMode
197+
// Check if the feature is either forcefully enabled or disabled, via a
198+
// cluster setting.
199+
stmtsForceControl := getStatementsForceControl(&b.ClusterSettings().SV)
200+
if forcedEnabled := stmtsForceControl.CheckControl(n); forcedEnabled {
201+
ret = sessiondatapb.UseNewSchemaChangerUnsafe
202+
}
203+
return ret
204+
}
205+
194206
var isV221Active = func(_ tree.NodeFormatter, _ sessiondatapb.NewSchemaChangerMode, activeVersion clusterversion.ClusterVersion) bool {
195207
return activeVersion.IsActive(clusterversion.TODODelete_V22_1)
196208
}

pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process_test.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import (
2222
func TestSupportedStatements(t *testing.T) {
2323
sv := &settings.Values{}
2424
// Non-existent tags should error out.
25-
require.Error(t, schemaChangerDisabledStatements.Validate(sv, "FAKE STATEMENT"))
25+
require.Error(t, forceDeclarativeStatements.Validate(sv, "FAKE STATEMENT"))
2626
// Generate the full set of statements
2727
allTags := strings.Builder{}
2828
noTags := strings.Builder{}
@@ -35,8 +35,8 @@ func TestSupportedStatements(t *testing.T) {
3535
ret := typTag.Func.Call([]reflect.Value{reflect.New(typ.Elem())})
3636
require.Equal(t, ret[0].String(), stmt.statementTag, "statement tag is different in AST")
3737
// Validate all tags are supported.
38-
require.NoError(t, schemaChangerDisabledStatements.Validate(sv, "+"+stmt.statementTag))
39-
require.NoError(t, schemaChangerDisabledStatements.Validate(sv, "!"+stmt.statementTag))
38+
require.NoError(t, forceDeclarativeStatements.Validate(sv, "+"+stmt.statementTag))
39+
require.NoError(t, forceDeclarativeStatements.Validate(sv, "!"+stmt.statementTag))
4040
// Validate all of them can be specified at once.
4141
if !first {
4242
allTags.WriteString(",")
@@ -48,6 +48,6 @@ func TestSupportedStatements(t *testing.T) {
4848
noTags.WriteString("!")
4949
noTags.WriteString(stmt.statementTag)
5050
}
51-
require.NoError(t, schemaChangerDisabledStatements.Validate(sv, allTags.String()))
52-
require.NoError(t, schemaChangerDisabledStatements.Validate(sv, noTags.String()))
51+
require.NoError(t, forceDeclarativeStatements.Validate(sv, allTags.String()))
52+
require.NoError(t, forceDeclarativeStatements.Validate(sv, noTags.String()))
5353
}

pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go

Lines changed: 20 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -19,18 +19,21 @@ import (
1919
"github.com/cockroachdb/errors"
2020
)
2121

22-
// schemaStatementControl track if a statement tag is enabled or disabled
23-
// forcefully by the user.
24-
type schemaStatementControl map[string]bool
22+
// statementsForceControl track if a statement tag is enabled or disabled
23+
// forcefully by the user to use declarative schema changer.
24+
type statementsForceControl map[string]bool
2525

26-
// schemaChangerDisabledStatements statements which are disabled
27-
// for the declarative schema changer. Users can specify statement
28-
// tags for each statement and a "!" symbol in front can have the opposite
29-
// effect to force enable fully unimplemented features.
30-
var schemaChangerDisabledStatements = settings.RegisterStringSetting(
26+
// forceDeclarativeStatements outlines statements which are forcefully enabled
27+
// and/or disabled with declarative schema changer, separated by comma.
28+
// Forcefully enabled statements are prefixed with "+";
29+
// Forcefully disabled statements are prefixed with "!";
30+
// E.g. `SET CLUSTER SETTING sql.schema.force_declarative_statements = "+ALTER TABLE,!CREATE SEQUENCE";`
31+
//
32+
// Note: We can only control statements implemented in declarative schema changer.
33+
var forceDeclarativeStatements = settings.RegisterStringSetting(
3134
settings.TenantWritable,
3235
"sql.schema.force_declarative_statements",
33-
"allows force enabling / disabling declarative schema changer for specific statements",
36+
"forcefully enable or disable declarative schema changer for specific statements",
3437
"",
3538
settings.WithValidateString(func(values *settings.Values, s string) error {
3639
if s == "" {
@@ -52,10 +55,10 @@ var schemaChangerDisabledStatements = settings.RegisterStringSetting(
5255
return nil
5356
}))
5457

55-
// CheckStatementControl if a statement is forced to disabled or enabled. If a
56-
// statement is disabled then an not implemented error will be panicked. Otherwise,
57-
// a flag is returned indicating if this statement has been *forced* to be enabled.
58-
func (c schemaStatementControl) CheckStatementControl(n tree.Statement) (forceEnabled bool) {
58+
// CheckControl checks if a statement is forced to be enabled or disabled. If
59+
// `n` is forcefully disabled, then a "NotImplemented" error will be panicked.
60+
// Otherwise, return whether `n` is forcefully enabled.
61+
func (c statementsForceControl) CheckControl(n tree.Statement) (forceEnabled bool) {
5962
// This map is only created *if* any force flags are set.
6063
if c == nil {
6164
return false
@@ -74,9 +77,9 @@ func (c schemaStatementControl) CheckStatementControl(n tree.Statement) (forceEn
7477
// GetSchemaChangerStatementControl returns a map of statements that
7578
// are explicitly disabled by administrators for the declarative schema
7679
// changer.
77-
func getSchemaChangerStatementControl(sv *settings.Values) schemaStatementControl {
78-
statements := schemaChangerDisabledStatements.Get(sv)
79-
var statementMap schemaStatementControl
80+
func getStatementsForceControl(sv *settings.Values) statementsForceControl {
81+
statements := forceDeclarativeStatements.Get(sv)
82+
var statementMap statementsForceControl
8083
for _, tag := range strings.Split(statements, ",") {
8184
tag = strings.ToUpper(strings.TrimSpace(tag))
8285
if len(tag) == 0 {
@@ -90,7 +93,7 @@ func getSchemaChangerStatementControl(sv *settings.Values) schemaStatementContro
9093
enabledOrDisabled = false
9194
}
9295
if statementMap == nil {
93-
statementMap = make(schemaStatementControl)
96+
statementMap = make(statementsForceControl)
9497
}
9598
statementMap[tag] = enabledOrDisabled
9699
}

pkg/sql/schemachanger/sctest/end_to_end.go

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -107,23 +107,20 @@ func EndToEndSideEffects(t *testing.T, relTestCaseDir string, factory TestServer
107107
// for end-to-end side-effect testing, so we ignore them.
108108
break
109109
case "test":
110-
stmts, execStmts := parseStmts()
110+
stmts, _ := parseStmts()
111111
require.Lessf(t, numTestStatementsObserved, 1, "only one test per-file.")
112112
numTestStatementsObserved++
113113
stmtSqls := make([]string, 0, len(stmts))
114114
for _, stmt := range stmts {
115115
stmtSqls = append(stmtSqls, stmt.SQL)
116116
}
117-
// Keep test cluster in sync.
118-
defer execStmts()
119117

120-
// Wait for any jobs due to previous schema changes to finish.
121-
sctestdeps.WaitForNoRunningSchemaChanges(t, tdb)
122-
var deps *sctestdeps.TestState
123118
// Create test dependencies and execute the schema changer.
124119
// The schema changer test dependencies do not hold any reference to the
125120
// test cluster, here the SQLRunner is only used to populate the mocked
126121
// catalog state.
122+
// It is declared here because it's used in its initialization below.
123+
var deps *sctestdeps.TestState
127124
// Set up a reference provider factory for the purpose of proper
128125
// dependency resolution.
129126
execCfg := s.ExecutorConfig().(sql.ExecutorConfig)

0 commit comments

Comments
 (0)