diff --git a/pkg/sql/schemachanger/scgraphviz/graphviz.go b/pkg/sql/schemachanger/scgraphviz/graphviz.go index ab9de8098e16..e3c1fd2ae4bd 100644 --- a/pkg/sql/schemachanger/scgraphviz/graphviz.go +++ b/pkg/sql/schemachanger/scgraphviz/graphviz.go @@ -161,11 +161,11 @@ func htmlLabel(o interface{}) dot.HTML { return dot.HTML(buf.String()) } -// toMap converts a struct to a map, field by field. If at any point a protobuf +// ToMap converts a struct to a map, field by field. If at any point a protobuf // message is encountered, it is converted to a map using jsonpb to marshal it // to json and then marshaling it back to a map. This approach allows zero // values to be effectively omitted. -func toMap(v interface{}) (interface{}, error) { +func ToMap(v interface{}) (interface{}, error) { if v == nil { return nil, nil } @@ -202,7 +202,7 @@ func toMap(v interface{}) (interface{}, error) { continue } var err error - if m[vt.Field(i).Name], err = toMap(vvf.Interface()); err != nil { + if m[vt.Field(i).Name], err = ToMap(vvf.Interface()); err != nil { return nil, err } } @@ -231,7 +231,7 @@ var objectTemplate = template.Must(template.New("obj").Funcs(template.FuncMap{ "isStruct": func(v interface{}) bool { return reflect.ValueOf(v).Kind() == reflect.Struct }, - "toMap": toMap, + "ToMap": ToMap, }).Parse(` {{- define "key" -}} @@ -271,7 +271,7 @@ var objectTemplate = template.Must(template.New("obj").Funcs(template.FuncMap{ {{- typeOf . -}} -{{- template "mapVal" (toMap $v) -}} +{{- template "mapVal" (ToMap $v) -}} {{- else -}} -{{- template "val" $v -}} @@ -291,7 +291,7 @@ var objectTemplate = template.Must(template.New("obj").Funcs(template.FuncMap{ {{- template "header" . -}}
-{{- template "mapVal" (toMap .) -}} +{{- template "mapVal" (ToMap .) -}}
`)) diff --git a/pkg/sql/schemachanger/scplan/main_test.go b/pkg/sql/schemachanger/scplan/main_test.go new file mode 100644 index 000000000000..0fcc82ddb345 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/main_test.go @@ -0,0 +1,31 @@ +// 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 scplan_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go index 2b4814478c83..655ce82c9633 100644 --- a/pkg/sql/schemachanger/scplan/plan.go +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -56,6 +56,9 @@ type Params struct { // // This doesn't do anything right now. CreatedDescriptorIDs catalog.DescriptorIDSet + // DisableOpRandomization disables randomization for the final set of + // operations. + DisableOpRandomization bool } // A Plan is a schema change plan, primarily containing ops to be executed that @@ -110,7 +113,7 @@ func MakePlan(initialStates []*scpb.Node, params Params) (_ Plan, err error) { }); err != nil { return Plan{}, err } - stages := buildStages(initialStates, g) + stages := buildStages(initialStates, g, params) return Plan{ Params: params, InitialNodes: initialStates, @@ -119,7 +122,7 @@ func MakePlan(initialStates []*scpb.Node, params Params) (_ Plan, err error) { }, nil } -func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage { +func buildStages(init []*scpb.Node, g *scgraph.Graph, params Params) []Stage { // TODO(ajwerner): deal with the case where the target state was // fulfilled by something that preceded the initial state. cur := init @@ -236,12 +239,15 @@ func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage { // be order independent, however we will // try to execute non-failing ones first. opsSlice := s.Ops.Slice() - rand.Seed(timeutil.Now().UnixNano()) - rand.Shuffle(len(opsSlice), func(i, j int) { - tmp := opsSlice[i] - opsSlice[i] = opsSlice[j] - opsSlice[j] = tmp - }) + if !params.DisableOpRandomization { + + rand.Seed(timeutil.Now().UnixNano()) + rand.Shuffle(len(opsSlice), func(i, j int) { + tmp := opsSlice[i] + opsSlice[i] = opsSlice[j] + opsSlice[j] = tmp + }) + } // Place non-revertible operations at the end sort.SliceStable(opsSlice, func(i, j int) bool { if opsSlice[i].Revertible() == opsSlice[j].Revertible() { @@ -251,6 +257,7 @@ func buildStages(init []*scpb.Node, g *scgraph.Graph) []Stage { }) stages = append(stages, s) cur = s.After + } return stages } diff --git a/pkg/sql/schemachanger/scplan/plan_test.go b/pkg/sql/schemachanger/scplan/plan_test.go new file mode 100644 index 000000000000..e0a155d9e86b --- /dev/null +++ b/pkg/sql/schemachanger/scplan/plan_test.go @@ -0,0 +1,213 @@ +// 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 scplan_test + +import ( + "context" + "fmt" + "path/filepath" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraphviz" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" + "gopkg.in/yaml.v2" +) + +func TestPlanAlterTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + datadriven.Walk(t, filepath.Join("testdata"), func(t *testing.T, path string) { + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + tdb := sqlutils.MakeSQLRunner(sqlDB) + run := func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "create-view": + fallthrough + case "create-sequence": + fallthrough + case "create-table": + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + require.Len(t, stmts, 1) + tableName := "" + switch node := stmts[0].AST.(type) { + case *tree.CreateTable: + tableName = node.Table.String() + case *tree.CreateSequence: + tableName = node.Name.String() + case *tree.CreateView: + tableName = node.Name.String() + default: + t.Fatal("not a CREATE TABLE/SEQUENCE/VIEW statement") + } + tdb.Exec(t, d.Input) + + var tableID descpb.ID + tdb.QueryRow(t, `SELECT $1::regclass::int`, tableName).Scan(&tableID) + if tableID == 0 { + t.Fatalf("failed to read ID of new table %s", tableName) + } + t.Logf("created relation with id %d", tableID) + + return "" + case "deps": + fallthrough + case "ops": + deps, cleanup := newTestingPlanDeps(s) + defer cleanup() + + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + var outputNodes []*scpb.Node + for i := range stmts { + outputNodes, err = scbuild.Build(ctx, stmts[i].AST, *deps, outputNodes) + require.NoError(t, err) + } + + plan, err := scplan.MakePlan(outputNodes, + scplan.Params{ + ExecutionPhase: scplan.PostCommitPhase, + DisableOpRandomization: true, + }) + require.NoError(t, err) + + if d.Cmd == "ops" { + return marshalOps(t, &plan) + } + return marshalDeps(t, &plan) + case "unimplemented": + deps, cleanup := newTestingPlanDeps(s) + defer cleanup() + + stmts, err := parser.Parse(d.Input) + require.NoError(t, err) + require.Len(t, stmts, 1) + + stmt := stmts[0] + alter, ok := stmt.AST.(*tree.AlterTable) + require.Truef(t, ok, "not an ALTER TABLE statement: %s", stmt.SQL) + _, err = scbuild.Build(ctx, alter, *deps, nil) + require.Truef(t, scbuild.HasNotImplemented(err), "expected unimplemented, got %v", err) + return "" + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + } + datadriven.RunTest(t, path, run) + }) +} + +// indentText indents text for formatting out marshalled data. +func indentText(input string, tab string) (final string) { + split := strings.Split(input, "\n") + for idx, line := range split { + if len(line) == 0 { + continue + } + final += tab + line + if idx != len(split)-1 { + final = final + "\n" + } + } + return final +} + +// marshalDeps marshals dependencies in scplan.Plan to a string. +func marshalDeps(t *testing.T, plan *scplan.Plan) string { + stages := "" + convertNode := func(node *scpb.Node) string { + depMap, err := scgraphviz.ToMap(node) + require.NoError(t, err) + data, err := yaml.Marshal(depMap) + require.NoError(t, err) + return string(data) + } + + plan.Graph.ForEachNode(func(n *scpb.Node) error { + return plan.Graph.ForEachDepEdgeFrom(n, func(de *scgraph.DepEdge) error { + stages += fmt.Sprintf("Dependency From:\n%sTo:\n%s", + indentText(convertNode(de.To()), " "), + indentText(convertNode(de.From()), " ")) + return nil + }) + }) + return stages +} + +// marshalOps marshals operations in scplan.Plan to a string. +func marshalOps(t *testing.T, plan *scplan.Plan) string { + stages := "" + for stageIdx, stage := range plan.Stages { + stages += fmt.Sprintf("Stage %d\n", stageIdx) + stageOps := "" + for _, op := range stage.Ops.Slice() { + opMap, err := scgraphviz.ToMap(op) + require.NoError(t, err) + data, err := yaml.Marshal(opMap) + require.NoError(t, err) + stageOps += fmt.Sprintf("%T\n%s", op, indentText(string(data), " ")) + } + stages += indentText(stageOps, " ") + } + return stages +} + +func newTestingPlanDeps(s serverutils.TestServerInterface) (*scbuild.Dependencies, func()) { + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + ip, cleanup := sql.NewInternalPlanner( + "test", + kv.NewTxn(context.Background(), s.DB(), s.NodeID()), + security.RootUserName(), + &sql.MemoryMetrics{}, + &execCfg, + // Setting the database on the session data to "defaultdb" in the obvious + // way doesn't seem to do what we want. + sessiondatapb.SessionData{}, + ) + planner := ip.(interface { + resolver.SchemaResolver + SemaCtx() *tree.SemaContext + EvalContext() *tree.EvalContext + Descriptors() *descs.Collection + }) + buildDeps := scbuild.Dependencies{ + Res: planner, + SemaCtx: planner.SemaCtx(), + EvalCtx: planner.EvalContext(), + Descs: planner.Descriptors(), + } + return &buildDeps, cleanup +} diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table b/pkg/sql/schemachanger/scplan/testdata/alter_table new file mode 100644 index 000000000000..1f4cb2cce9f5 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table @@ -0,0 +1,559 @@ +create-table +CREATE TABLE defaultdb.foo (i INT PRIMARY KEY) +---- + +ops +ALTER TABLE defaultdb.foo ADD COLUMN j INT +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + ID: 2 + Name: j + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - j + Unique: true + Version: 3 + TableID: 52 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + +ops +ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123 +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + DefaultExpr: 123:::INT8 + ID: 2 + Name: j + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - j + Unique: true + Version: 3 + TableID: 52 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + +ops +ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123; +ALTER TABLE defaultdb.foo ADD COLUMN k INT DEFAULT 456; +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + DefaultExpr: 123:::INT8 + ID: 2 + Name: j + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + - 3 + StoreColumnNames: + - j + - k + Unique: true + Version: 3 + TableID: 52 + *scop.MakeAddedColumnDeleteOnly + Column: + DefaultExpr: 456:::INT8 + ID: 3 + Name: k + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 3 + TableID: 52 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 + *scop.MakeColumnPublic + ColumnID: 3 + TableID: 52 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + +ops +ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + ComputeExpr: i + 1:::INT8 + ID: 2 + Name: a + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - a + Unique: true + Version: 3 + TableID: 52 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + + +create-table +CREATE TABLE defaultdb.bar (j INT); +---- + +ops +ALTER TABLE defaultdb.foo ADD COLUMN a INT; +ALTER TABLE defaultdb.bar ADD COLUMN b INT; +---- +Stage 0 + *scop.MakeAddedColumnDeleteOnly + Column: + ID: 2 + Name: a + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - a + Unique: true + Version: 3 + TableID: 52 + *scop.MakeAddedColumnDeleteOnly + Column: + ID: 3 + Name: b + Nullable: true + Type: + family: IntFamily + oid: 20 + width: 64 + FamilyName: primary + TableID: 53 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: new_primary_key + StoreColumnIDs: + - 1 + - 3 + StoreColumnNames: + - j + - b + Unique: true + TableID: 53 +Stage 1 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 2 + TableID: 52 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 52 + *scop.MakeAddedColumnDeleteAndWriteOnly + ColumnID: 3 + TableID: 53 + *scop.MakeAddedIndexDeleteAndWriteOnly + IndexID: 2 + TableID: 53 +Stage 2 + scop.BackfillIndex + IndexID: 2 + TableID: 52 + scop.BackfillIndex + IndexID: 2 + TableID: 53 +Stage 3 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 52 + scop.ValidateUniqueIndex + IndexID: 2 + PrimaryIndexID: 1 + TableID: 53 +Stage 4 + *scop.MakeColumnPublic + ColumnID: 2 + TableID: 52 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + Unique: true + Version: 3 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: primary + Unique: true + Version: 3 + TableID: 52 + *scop.MakeColumnPublic + ColumnID: 3 + TableID: 53 + *scop.MakeAddedPrimaryIndexPublic + Index: + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: new_primary_key + Unique: true + TableID: 53 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: primary + StoreColumnIDs: + - 1 + StoreColumnNames: + - j + Unique: true + TableID: 53 +Stage 5 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 52 + *scop.MakeDroppedIndexDeleteOnly + IndexID: 1 + TableID: 53 +Stage 6 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 52 + *scop.MakeIndexAbsent + IndexID: 1 + TableID: 53 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_sequence b/pkg/sql/schemachanger/scplan/testdata/drop_sequence new file mode 100644 index 000000000000..ffc8a5072b76 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/testdata/drop_sequence @@ -0,0 +1,88 @@ +create-sequence +CREATE SEQUENCE defaultdb.SQ1 +---- + +ops +DROP SEQUENCE defaultdb.SQ1 CASCADE +---- +Stage 0 + *scop.CreateGcJobForDescriptor + DescID: 52 + *scop.MarkDescriptorAsDropped + TableID: 52 +Stage 1 + *scop.CreateGcJobForDescriptor + DescID: 52 + *scop.DrainDescriptorName + TableID: 52 + +create-table +CREATE TABLE defaultdb.blog_posts (id INT PRIMARY KEY, val int DEFAULT nextval('defaultdb.sq1'), title text) +---- + +create-table +CREATE TABLE defaultdb.blog_posts2 (id INT PRIMARY KEY, val int DEFAULT nextval('defaultdb.sq1'), title text) +---- + +ops +DROP SEQUENCE defaultdb.SQ1 CASCADE +---- +Stage 0 + *scop.RemoveColumnDefaultExpression + ColumnID: 2 + TableID: 53 + *scop.UpdateRelationDeps + TableID: 53 + *scop.RemoveColumnDefaultExpression + ColumnID: 2 + TableID: 54 + *scop.UpdateRelationDeps + TableID: 54 + *scop.CreateGcJobForDescriptor + DescID: 52 + *scop.MarkDescriptorAsDropped + TableID: 52 +Stage 1 + *scop.CreateGcJobForDescriptor + DescID: 52 + *scop.DrainDescriptorName + TableID: 52 + + +deps +DROP SEQUENCE defaultdb.SQ1 CASCADE +---- +Dependency From: + State: 1 + Target: + direction: DROP + elementProto: + defaultExpression: + columnId: 2 + tableId: 53 + usesSequenceIDs: + - 52 +To: + State: 2 + Target: + direction: DROP + elementProto: + table: + tableId: 52 +Dependency From: + State: 1 + Target: + direction: DROP + elementProto: + defaultExpression: + columnId: 2 + tableId: 54 + usesSequenceIDs: + - 52 +To: + State: 2 + Target: + direction: DROP + elementProto: + table: + tableId: 52