Skip to content

Commit

Permalink
Merge #107815
Browse files Browse the repository at this point in the history
107815: sql/schemachanger: disable CREATE SEQUENCE / CREATE SCHEMA by default in declarative schema changer r=fqazi a=fqazi

For 23.2 and keeping things more conservative, we will keep CREATE SEQUENCE / CREATE SCHEMA running
in the older declarative schema changer. This patch will do the following:

1. CREATE SEQUENCE / CREATE SCHEMA are disabled by default
2. A new cluster setting sql.schema.force_declarative_statements can now be used to enable/disable declarative features in a more granular matter.


Fixes: #107734

Co-authored-by: Faizan Qazi <[email protected]>
  • Loading branch information
craig[bot] and fqazi committed Aug 10, 2023
2 parents d9d421e + ab1a740 commit 67e16cb
Show file tree
Hide file tree
Showing 16 changed files with 319 additions and 86 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -521,6 +521,7 @@ ALL_TESTS = [
"//pkg/sql/schemachanger/corpus:corpus_test",
"//pkg/sql/schemachanger/rel:rel_test",
"//pkg/sql/schemachanger/scbackup:scbackup_test",
"//pkg/sql/schemachanger/scbuild/internal/scbuildstmt:scbuildstmt_test",
"//pkg/sql/schemachanger/scbuild:scbuild_test",
"//pkg/sql/schemachanger/scdecomp:scdecomp_test",
"//pkg/sql/schemachanger/scexec/backfiller:backfiller_test",
Expand Down Expand Up @@ -1974,6 +1975,7 @@ GO_TARGETS = [
"//pkg/sql/schemachanger/scbackup:scbackup",
"//pkg/sql/schemachanger/scbackup:scbackup_test",
"//pkg/sql/schemachanger/scbuild/internal/scbuildstmt:scbuildstmt",
"//pkg/sql/schemachanger/scbuild/internal/scbuildstmt:scbuildstmt_test",
"//pkg/sql/schemachanger/scbuild:scbuild",
"//pkg/sql/schemachanger/scbuild:scbuild_test",
"//pkg/sql/schemachanger/scdecomp:scdecomp",
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/event_log
Original file line number Diff line number Diff line change
Expand Up @@ -708,8 +708,8 @@ FROM system.eventlog
WHERE "eventType" = 'create_schema'
ORDER BY "timestamp", info
----
1 {"EventType": "create_schema", "Owner": "root", "SchemaName": "test.sc", "Statement": "CREATE SCHEMA test.sc", "Tag": "CREATE SCHEMA", "User": "root"}
1 {"EventType": "create_schema", "Owner": "root", "SchemaName": "test.s", "Statement": "CREATE SCHEMA test.s", "Tag": "CREATE SCHEMA", "User": "root"}
1 {"EventType": "create_schema", "Owner": "root", "SchemaName": "test.sc", "Statement": "CREATE SCHEMA \"\".sc", "Tag": "CREATE SCHEMA", "User": "root"}
1 {"EventType": "create_schema", "Owner": "root", "SchemaName": "test.s", "Statement": "CREATE SCHEMA \"\".s", "Tag": "CREATE SCHEMA", "User": "root"}
1 {"EventType": "create_schema", "Owner": "u", "SchemaName": "test.u", "Statement": "CREATE SCHEMA AUTHORIZATION u", "Tag": "CREATE SCHEMA", "User": "root"}

statement ok
Expand Down
30 changes: 30 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/new_schema_changer
Original file line number Diff line number Diff line change
Expand Up @@ -1501,3 +1501,33 @@ public admin

statement ok
COMMIT


subtest statement-control

# Disable ALTER table completely and validate it works
statement ok
SET use_declarative_schema_changer = 'on'

statement ok
SET CLUSTER SETTING sql.schema.force_declarative_statements='!ALTER TABLE'

statement ok
CREATE TABLE stmt_ctrl(n int primary key)

statement error pgcode 0A000 cannot explain a statement which is not supported by the declarative schema changer
EXPLAIN (DDL) ALTER TABLE stmt_ctrl ADD COLUMN j BOOL

statement ok
ALTER TABLE stmt_ctrl ADD COLUMN fallback_works BOOL

# Validate that CREATE SEQUENCE is disabled and can be re-enabled with the same flag.
statement error pgcode 0A000 cannot explain a statement which is not supported by the declarative schema changer
EXPLAIN (DDL) CREATE SEQUENCE sq2

statement ok
SET CLUSTER SETTING sql.schema.force_declarative_statements='+CREATE SEQUENCE'

skipif config local-mixed-22.2-23.1
statement ok
EXPLAIN (DDL) CREATE SEQUENCE sq2
15 changes: 14 additions & 1 deletion pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "scbuildstmt",
Expand Down Expand Up @@ -28,6 +28,7 @@ go_library(
"drop_view.go",
"helpers.go",
"process.go",
"statement_control.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild/internal/scbuildstmt",
visibility = ["//pkg/sql/schemachanger/scbuild:__subpackages__"],
Expand All @@ -38,6 +39,7 @@ go_library(
"//pkg/keys",
"//pkg/security/username",
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catenumpb",
Expand Down Expand Up @@ -78,3 +80,14 @@ go_library(
"@com_github_lib_pq//oid",
],
)

go_test(
name = "scbuildstmt_test",
srcs = ["process_test.go"],
args = ["-test.timeout=295s"],
embed = [":scbuildstmt"],
deps = [
"//pkg/settings",
"@com_github_stretchr_testify//require",
],
)
58 changes: 37 additions & 21 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ import (
type supportedStatement struct {
// fn is a function to perform a schema change.
fn interface{}
// statementTag tag for this statement.
statementTag string
// checks contains a coarse-grained function to filter out most
// unsupported statements.
// It's possible for certain unsupported statements to pass it but will
Expand All @@ -44,28 +46,32 @@ var supportedStatements = map[reflect.Type]supportedStatement{
// Alter table will have commands individually whitelisted via the
// supportedAlterTableStatements list, so wwe will consider it fully supported
// here.
reflect.TypeOf((*tree.AlterTable)(nil)): {fn: AlterTable, on: true, checks: alterTableChecks},
reflect.TypeOf((*tree.CreateIndex)(nil)): {fn: CreateIndex, on: true, checks: isV231Active},
reflect.TypeOf((*tree.DropDatabase)(nil)): {fn: DropDatabase, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropOwnedBy)(nil)): {fn: DropOwnedBy, on: true, checks: isV222Active},
reflect.TypeOf((*tree.DropSchema)(nil)): {fn: DropSchema, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropSequence)(nil)): {fn: DropSequence, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropTable)(nil)): {fn: DropTable, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropType)(nil)): {fn: DropType, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropView)(nil)): {fn: DropView, on: true, checks: isV221Active},
reflect.TypeOf((*tree.CommentOnDatabase)(nil)): {fn: CommentOnDatabase, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnSchema)(nil)): {fn: CommentOnSchema, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnTable)(nil)): {fn: CommentOnTable, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnColumn)(nil)): {fn: CommentOnColumn, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnIndex)(nil)): {fn: CommentOnIndex, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnConstraint)(nil)): {fn: CommentOnConstraint, on: true, checks: isV222Active},
reflect.TypeOf((*tree.DropIndex)(nil)): {fn: DropIndex, on: true, checks: isV231Active},
reflect.TypeOf((*tree.DropFunction)(nil)): {fn: DropFunction, on: true, checks: isV231Active},
reflect.TypeOf((*tree.CreateRoutine)(nil)): {fn: CreateFunction, on: true, checks: isV231Active},
reflect.TypeOf((*tree.CreateSchema)(nil)): {fn: CreateSchema, on: true, checks: isV232Active},
reflect.TypeOf((*tree.CreateSequence)(nil)): {fn: CreateSequence, on: true, checks: isV232Active},
reflect.TypeOf((*tree.AlterTable)(nil)): {fn: AlterTable, statementTag: tree.AlterTableTag, on: true, checks: alterTableChecks},
reflect.TypeOf((*tree.CreateIndex)(nil)): {fn: CreateIndex, statementTag: tree.CreateIndexTag, on: true, checks: isV231Active},
reflect.TypeOf((*tree.DropDatabase)(nil)): {fn: DropDatabase, statementTag: tree.DropDatabaseTag, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropOwnedBy)(nil)): {fn: DropOwnedBy, statementTag: tree.DropOwnedByTag, on: true, checks: isV222Active},
reflect.TypeOf((*tree.DropSchema)(nil)): {fn: DropSchema, statementTag: tree.DropSchemaTag, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropSequence)(nil)): {fn: DropSequence, statementTag: tree.DropSequenceTag, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropTable)(nil)): {fn: DropTable, statementTag: tree.DropTableTag, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropType)(nil)): {fn: DropType, statementTag: tree.DropTypeTag, on: true, checks: isV221Active},
reflect.TypeOf((*tree.DropView)(nil)): {fn: DropView, statementTag: tree.DropViewTag, on: true, checks: isV221Active},
reflect.TypeOf((*tree.CommentOnConstraint)(nil)): {fn: CommentOnConstraint, statementTag: tree.CommentOnConstraintTag, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnDatabase)(nil)): {fn: CommentOnDatabase, statementTag: tree.CommentOnDatabaseTag, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnSchema)(nil)): {fn: CommentOnSchema, statementTag: tree.CommentOnSchemaTag, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnTable)(nil)): {fn: CommentOnTable, statementTag: tree.CommentOnTableTag, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnColumn)(nil)): {fn: CommentOnColumn, statementTag: tree.CommentOnColumnTag, on: true, checks: isV222Active},
reflect.TypeOf((*tree.CommentOnIndex)(nil)): {fn: CommentOnIndex, statementTag: tree.CommentOnIndexTag, on: true, checks: isV222Active},
reflect.TypeOf((*tree.DropIndex)(nil)): {fn: DropIndex, statementTag: tree.DropIndexTag, on: true, checks: isV231Active},
reflect.TypeOf((*tree.DropFunction)(nil)): {fn: DropFunction, statementTag: tree.DropFunctionTag, on: true, checks: isV231Active},
reflect.TypeOf((*tree.CreateRoutine)(nil)): {fn: CreateFunction, statementTag: tree.CreateRoutineTag, on: true, checks: isV231Active},
reflect.TypeOf((*tree.CreateSchema)(nil)): {fn: CreateSchema, statementTag: tree.CreateSchemaTag, on: false, checks: isV232Active},
reflect.TypeOf((*tree.CreateSequence)(nil)): {fn: CreateSequence, statementTag: tree.CreateSequenceTag, on: false, checks: isV232Active},
}

// supportedStatementTags tracks statement tags which are implemented
// by the declarative schema changer.
var supportedStatementTags = map[string]struct{}{}

func init() {
boolType := reflect.TypeOf((*bool)(nil)).Elem()
// Check function signatures inside the supportedStatements map.
Expand Down Expand Up @@ -99,6 +105,9 @@ func init() {
statementType, checks))
}
}
// Fetch the statement tag using the statement tag method on the type,
// we can use this as a blacklist of blocked schema changes.
supportedStatementTags[statementEntry.statementTag] = struct{}{}
}
}

Expand Down Expand Up @@ -157,9 +166,16 @@ func isFullySupportedWithFalsePositiveInternal(
// Process dispatches on the statement type to populate the BuilderState
// embedded in the BuildCtx. Any error will be panicked.
func Process(b BuildCtx, n tree.Statement) {
newSchemaChangerMode := b.EvalCtx().SessionData().NewSchemaChangerMode
// Check if the feature is either forced disabled or enabled,
// using a cluster setting.
disabledStatements := getSchemaChangerStatementControl(&b.ClusterSettings().SV)
if forcedEnabled := disabledStatements.CheckStatementControl(n); forcedEnabled {
newSchemaChangerMode = sessiondatapb.UseNewSchemaChangerUnsafe
}
// Run a few "quick checks" to see if the statement is not supported.
if !IsFullySupportedWithFalsePositive(n, b.EvalCtx().Settings.Version.ActiveVersion(b),
b.EvalCtx().SessionData().NewSchemaChangerMode) {
newSchemaChangerMode) {
panic(scerrors.NotImplementedError(n))
}

Expand Down
53 changes: 53 additions & 0 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
// Copyright 2023 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 scbuildstmt

import (
"reflect"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/stretchr/testify/require"
)

func TestSupportedStatements(t *testing.T) {
sv := &settings.Values{}
// Non-existent tags should error out.
require.Error(t, schemaChangerDisabledStatements.Validate(sv, "FAKE STATEMENT"))
// Generate the full set of statements
allTags := strings.Builder{}
noTags := strings.Builder{}
first := true
for typ, stmt := range supportedStatements {
require.Greaterf(t, len(stmt.statementTag), 0, "statement tag is missing %v %v", typ, stmt)
// Validate tags matches the statement tag
typTag, found := typ.MethodByName("StatementTag")
require.True(t, found, "unable to find stmt: %v %v", typ, stmt)
ret := typTag.Func.Call([]reflect.Value{reflect.New(typ.Elem())})
require.Equal(t, ret[0].String(), stmt.statementTag, "statement tag is different in AST")
// Validate all tags are supported.
require.NoError(t, schemaChangerDisabledStatements.Validate(sv, "+"+stmt.statementTag))
require.NoError(t, schemaChangerDisabledStatements.Validate(sv, "!"+stmt.statementTag))
// Validate all of them can be specified at once.
if !first {
allTags.WriteString(",")
noTags.WriteString(",")
}
first = false
allTags.WriteString("+")
allTags.WriteString(stmt.statementTag)
noTags.WriteString("!")
noTags.WriteString(stmt.statementTag)
}
require.NoError(t, schemaChangerDisabledStatements.Validate(sv, allTags.String()))
require.NoError(t, schemaChangerDisabledStatements.Validate(sv, noTags.String()))
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
// Copyright 2023 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 scbuildstmt

import (
"strings"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/errors"
)

// schemaStatementControl track if a statement tag is enabled or disabled
// forcefully by the user.
type schemaStatementControl map[string]bool

// schemaChangerDisabledStatements statements which are disabled
// for the declarative schema changer. Users can specify statement
// tags for each statement and a "!" symbol in front can have the opposite
// effect to force enable fully unimplemented features.
var schemaChangerDisabledStatements = func() *settings.StringSetting {
return settings.RegisterValidatedStringSetting(
settings.TenantWritable,
"sql.schema.force_declarative_statements",
"allows force enabling / disabling declarative schema changer for specific statements",
"",
func(values *settings.Values, s string) error {
if s == "" {
return nil
}
// First split the string into individual tags.
tags := strings.Split(s, ",")
for _, tag := range tags {
tag = strings.ToUpper(strings.TrimSpace(tag))
if len(tag) > 0 && (tag[0] == '+' || tag[0] == '!') {
tag = tag[1:]
} else {
return errors.Errorf("tag is not properly formatted, must start with '+' or '!' (%s)", tag)
}
if _, ok := supportedStatementTags[tag]; !ok {
return errors.Errorf("statement tag %q is not controlled by the declarative schema changer", tag)
}
}
return nil
})
}()

// CheckStatementControl if a statement is forced to disabled or enabled. If a
// statement is disabled then an not implemented error will be panicked. Otherwise,
// a flag is returned indicating if this statement has been *forced* to be enabled.
func (c schemaStatementControl) CheckStatementControl(n tree.Statement) (forceEnabled bool) {
// This map is only created *if* any force flags are set.
if c == nil {
return false
}
enabledOrDisabled, found := c[n.StatementTag()]
if !found {
return false
}
if !enabledOrDisabled {
panic(scerrors.NotImplementedErrorf(n,
"statement has been disabled via cluster setting"))
}
return enabledOrDisabled
}

// GetSchemaChangerStatementControl returns a map of statements that
// are explicitly disabled by administrators for the declarative schema
// changer.
func getSchemaChangerStatementControl(sv *settings.Values) schemaStatementControl {
statements := schemaChangerDisabledStatements.Get(sv)
var statementMap schemaStatementControl
for _, tag := range strings.Split(statements, ",") {
tag = strings.ToUpper(strings.TrimSpace(tag))
if len(tag) == 0 {
continue
}
enabledOrDisabled := true
tagStart := tag[0]
tag = tag[1:]
// If the tag starts with a ! its disabled.
if tagStart == '!' {
enabledOrDisabled = false
}
if statementMap == nil {
statementMap = make(schemaStatementControl)
}
statementMap[tag] = enabledOrDisabled
}
return statementMap
}
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/schemachanger_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -493,6 +493,7 @@ func TestSchemaChangeWaitsForOtherSchemaChanges(t *testing.T) {
defer s.Stopper().Stop(ctx)

tdb := sqlutils.MakeSQLRunner(sqlDB)
tdb.Exec(t, "SET CLUSTER SETTING sql.schema.force_declarative_statements='+CREATE SCHEMA'")
tdb.Exec(t, `CREATE DATABASE db`)
tdb.Exec(t, `CREATE SCHEMA db.s1`)
tdb.Exec(t, `CREATE SCHEMA db.s2`)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,8 +134,8 @@ upsert descriptor #107
+ columnIds:
+ - 2
+ id: 106
formatVersion: 3
id: 107
families:
- columnIds:
...
start: "1"
unexposedParentSchemaId: 105
Expand Down Expand Up @@ -289,8 +289,8 @@ upsert descriptor #107
+ columnIds:
+ - 2
+ id: 106
formatVersion: 3
id: 107
families:
- columnIds:
...
start: "1"
unexposedParentSchemaId: 105
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,8 @@ upsert descriptor #104
+ dependedOnBy:
+ - byId: true
+ id: 107
formatVersion: 3
id: 104
families:
- columnIds:
...
start: "1"
unexposedParentSchemaId: 101
Expand Down Expand Up @@ -120,8 +120,8 @@ upsert descriptor #104
+ dependedOnBy:
+ - byId: true
+ id: 107
formatVersion: 3
id: 104
families:
- columnIds:
...
start: "1"
unexposedParentSchemaId: 101
Expand Down
Loading

0 comments on commit 67e16cb

Please sign in to comment.