From 35e66c17a700471153bf358bb9ad4ac11a0a104b Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 17:54:10 -0500 Subject: [PATCH 1/6] sql/schemachanger/scgraph: refactor dependency edge iteration Release note: None --- pkg/sql/schemachanger/scgraph/BUILD.bazel | 10 +- .../schemachanger/scgraph/dep_edge_tree.go | 102 ++++++++++++++ .../scgraph/dep_edge_tree_test.go | 129 ++++++++++++++++++ pkg/sql/schemachanger/scgraph/graph.go | 91 +++--------- pkg/sql/schemachanger/scgraph/iteration.go | 27 +--- .../schemachanger/scplan/deprules/registry.go | 7 +- 6 files changed, 264 insertions(+), 102 deletions(-) create mode 100644 pkg/sql/schemachanger/scgraph/dep_edge_tree.go create mode 100644 pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel index c5e3c91993cd..e229c3cee857 100644 --- a/pkg/sql/schemachanger/scgraph/BUILD.bazel +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "scgraph", srcs = [ + "dep_edge_tree.go", "graph.go", "iteration.go", ], @@ -21,12 +22,17 @@ go_library( go_test( name = "scgraph_test", - srcs = ["graph_test.go"], + srcs = [ + "dep_edge_tree_test.go", + "graph_test.go", + ], + embed = [":scgraph"], deps = [ - ":scgraph", "//pkg/sql/catalog/descpb", "//pkg/sql/schemachanger/scop", "//pkg/sql/schemachanger/scpb", + "//pkg/util/iterutil", + "//pkg/util/leaktest", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/sql/schemachanger/scgraph/dep_edge_tree.go b/pkg/sql/schemachanger/scgraph/dep_edge_tree.go new file mode 100644 index 000000000000..dcc3354574d5 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/dep_edge_tree.go @@ -0,0 +1,102 @@ +// 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 scgraph + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/google/btree" +) + +type depEdgeTree struct { + t *btree.BTree + order edgeTreeOrder + cmp nodeCmpFn +} + +type nodeCmpFn func(a, b *scpb.Node) (less, eq bool) + +func newDepEdgeTree(order edgeTreeOrder, cmp nodeCmpFn) *depEdgeTree { + const degree = 8 // arbitrary + return &depEdgeTree{ + t: btree.New(degree), + order: order, + cmp: cmp, + } +} + +// edgeTreeOrder order in which the edge tree is sorted, +// either based on from/to node indexes. +type edgeTreeOrder bool + +func (o edgeTreeOrder) first(e Edge) *scpb.Node { + if o == fromTo { + return e.From() + } + return e.To() +} + +func (o edgeTreeOrder) second(e Edge) *scpb.Node { + if o == toFrom { + return e.From() + } + return e.To() +} + +const ( + fromTo edgeTreeOrder = true + toFrom edgeTreeOrder = false +) + +// edgeTreeEntry BTree items for tracking edges +// in an ordered manner. +type edgeTreeEntry struct { + t *depEdgeTree + edge *DepEdge +} + +func (et *depEdgeTree) insert(e *DepEdge) { + et.t.ReplaceOrInsert(&edgeTreeEntry{ + t: et, + edge: e, + }) +} + +func (et *depEdgeTree) iterateSourceNode(n *scpb.Node, it DepEdgeIterator) (err error) { + e := &edgeTreeEntry{t: et, edge: &DepEdge{}} + if et.order == fromTo { + e.edge.from = n + } else { + e.edge.to = n + } + et.t.AscendGreaterOrEqual(e, func(i btree.Item) (wantMore bool) { + e := i.(*edgeTreeEntry) + if et.order.first(e.edge) != n { + return false + } + err = it(e.edge) + return err == nil + }) + if iterutil.Done(err) { + err = nil + } + return err +} + +// Less implements btree.Item. +func (e *edgeTreeEntry) Less(otherItem btree.Item) bool { + o := otherItem.(*edgeTreeEntry) + if less, eq := e.t.cmp(e.t.order.first(e.edge), e.t.order.first(o.edge)); !eq { + return less + } + less, _ := e.t.cmp(e.t.order.second(e.edge), e.t.order.second(o.edge)) + return less +} diff --git a/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go new file mode 100644 index 000000000000..0b25a7aeef25 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go @@ -0,0 +1,129 @@ +// 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 scgraph + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +// TestDepEdgeTree exercises the depEdgeTree data structure to ensure it works +// as expected. +func TestDepEdgeTree(t *testing.T) { + defer leaktest.AfterTest(t)() + + type nodeID int + type edge [2]nodeID + // queryCase runs a query to iterate all edges sources at the node with id q. + type queryCase struct { + q nodeID + take int // if > 1, indicates a desire to stop early + res []edge // expected results + } + // testCase describes the edges to be added and the queries to run. + type testCase struct { + order edgeTreeOrder + edges []edge + queries []queryCase + } + testCases := []testCase{ + { + order: fromTo, + edges: []edge{ + {2, 4}, {2, 3}, {4, 5}, {1, 2}, + }, + queries: []queryCase{ + {q: 1, res: []edge{{1, 2}}}, + {q: 2, res: []edge{{2, 3}, {2, 4}}}, + {q: 2, take: 1, res: []edge{{2, 3}}}, + }, + }, + { + order: toFrom, + edges: []edge{ + {2, 4}, {2, 3}, {4, 5}, {1, 2}, {2, 5}, {1, 5}, + }, + queries: []queryCase{ + {q: 1, res: nil}, + {q: 2, res: []edge{{1, 2}}}, + {q: 5, res: []edge{{1, 5}, {2, 5}, {4, 5}}}, + {q: 5, take: 1, res: []edge{{1, 5}}}, + }, + }, + } + + // testCaseState is used for each queryCase in a testCase. + type testCaseState struct { + tree *depEdgeTree + nodes []*scpb.Node // nodes with lower indexes sort lower + nodesToID map[*scpb.Node]nodeID + } + makeTestCaseState := func(tc testCase) testCaseState { + tcs := testCaseState{ + nodesToID: make(map[*scpb.Node]nodeID), + } + target := scpb.Target{} + getNode := func(i nodeID) *scpb.Node { + if i > nodeID(len(tcs.nodes)-1) { + for j := nodeID(len(tcs.nodes)); j <= i; j++ { + tcs.nodes = append(tcs.nodes, &scpb.Node{ + Target: &target, + Status: scpb.Status(j), + }) + tcs.nodesToID[tcs.nodes[j]] = j + } + } + return tcs.nodes[i] + } + tcs.tree = newDepEdgeTree(tc.order, func(a, b *scpb.Node) (less, eq bool) { + ai, bi := tcs.nodesToID[a], tcs.nodesToID[b] + return ai < bi, ai == bi + }) + for _, e := range tc.edges { + tcs.tree.insert(&DepEdge{ + from: getNode(e[0]), + to: getNode(e[1]), + }) + } + return tcs + } + runQueryCase := func(t *testing.T, tcs testCaseState, qc queryCase) { + i := 0 + var res []edge + require.NoError(t, tcs.tree.iterateSourceNode(tcs.nodes[qc.q], func(de *DepEdge) error { + if i++; qc.take > 0 && i > qc.take { + return iterutil.StopIteration() + } + res = append(res, edge{ + tcs.nodesToID[de.From()], + tcs.nodesToID[de.To()], + }) + return nil + })) + require.Equal(t, qc.res, res) + } + + for _, tc := range testCases { + t.Run(fmt.Sprintf("%v,%v", tc.order, tc.edges), func(t *testing.T) { + tcs := makeTestCaseState(tc) + for _, qc := range tc.queries { + t.Run(fmt.Sprintf("%d,%d", qc.q, qc.take), func(t *testing.T) { + runQueryCase(t, tcs, qc) + }) + } + }) + } +} diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 0e5f9843a4fa..20f88fb8f28a 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" "github.com/cockroachdb/errors" - "github.com/google/btree" ) // Graph is a graph whose nodes are *scpb.Nodes. Graphs are constructed during @@ -49,12 +48,7 @@ type Graph struct { // nodeDepEdgesFrom maps a Node from its dependencies. // A Node dependency is another target node which must be // reached before or concurrently with this node. - nodeDepEdgesFrom *btree.BTree - - // nodeDepEdgesTo maps a Node to its dependencies. - // A Node dependency is another target node which must be - // reached before or concurrently with this node. - nodeDepEdgesTo *btree.BTree + nodeDepEdgesFrom *depEdgeTree // opToNode maps from an operation back to the // opEdge that generated it as an index. @@ -90,14 +84,13 @@ func New(initial scpb.State) (*Graph, error) { g := Graph{ targetIdxMap: map[*scpb.Target]int{}, nodeOpEdgesFrom: map[*scpb.Node]*OpEdge{}, - nodeDepEdgesFrom: btree.New(2), - nodeDepEdgesTo: btree.New(2), optimizedOutOpEdges: map[*OpEdge]bool{}, opToNode: map[scop.Op]*scpb.Node{}, entities: db, statements: initial.Statements, authorization: initial.Authorization, } + g.nodeDepEdgesFrom = newDepEdgeTree(fromTo, g.compareNodes) for _, n := range initial.Nodes { if existing, ok := g.targetIdxMap[n.Target]; ok { return nil, errors.Errorf("invalid initial state contains duplicate target: %v and %v", n, initial.Nodes[existing]) @@ -127,7 +120,6 @@ func (g *Graph) ShallowClone() *Graph { targetIdxMap: g.targetIdxMap, nodeOpEdgesFrom: g.nodeOpEdgesFrom, nodeDepEdgesFrom: g.nodeDepEdgesFrom, - nodeDepEdgesTo: g.nodeDepEdgesTo, opToNode: g.opToNode, edges: g.edges, entities: g.entities, @@ -251,16 +243,7 @@ func (g *Graph) AddDepEdge( return err } g.edges = append(g.edges, de) - g.nodeDepEdgesFrom.ReplaceOrInsert(&edgeTreeEntry{ - g: g, - edge: de, - order: fromTo, - }) - g.nodeDepEdgesTo.ReplaceOrInsert(&edgeTreeEntry{ - g: g, - edge: de, - order: toFrom, - }) + g.nodeDepEdgesFrom.insert(de) return nil } @@ -344,6 +327,23 @@ func (de *DepEdge) To() *scpb.Node { return de.to } // Name returns the name of the rule which generated this edge. func (de *DepEdge) Name() string { return de.rule } +// compareNodes compares two nodes in a graph. A nil nodes is the minimum value. +func (g *Graph) compareNodes(a, b *scpb.Node) (less, eq bool) { + switch { + case a == b: + return false, true + case a == nil: + return true, false + case b == nil: + return false, false + case a.Target == b.Target: + return a.Status < b.Status, a.Status == b.Status + default: + aIdx, bIdx := g.targetIdxMap[a.Target], g.targetIdxMap[b.Target] + return aIdx < bIdx, aIdx == bIdx + } +} + // GetNodeRanks fetches ranks of nodes in topological order. func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { defer func() { @@ -399,54 +399,3 @@ func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { } return rank, nil } - -// edgeTreeOrder order in which the edge tree is sorted, -// either based on from/to node indexes. -type edgeTreeOrder bool - -const ( - fromTo edgeTreeOrder = true - toFrom edgeTreeOrder = false -) - -// edgeTreeEntry BTree items for tracking edges -// in an ordered manner. -type edgeTreeEntry struct { - g *Graph - edge Edge - order edgeTreeOrder -} - -// Less implements btree.Item -func (e *edgeTreeEntry) Less(other btree.Item) bool { - o := other.(*edgeTreeEntry) - var a1, b1, a2, b2 *scpb.Node - switch e.order { - case fromTo: - a1, b1, a2, b2 = e.edge.From(), o.edge.From(), e.edge.To(), o.edge.To() - case toFrom: - a1, b1, a2, b2 = e.edge.To(), o.edge.To(), e.edge.From(), o.edge.From() - } - less, eq := compareNodes(e.g, a1, b1) - if eq { - less, _ = compareNodes(e.g, a2, b2) - } - return less -} - -// compareNodes compares two nodes in a graph. A nil nodes is the minimum value. -func compareNodes(g *Graph, a, b *scpb.Node) (less, eq bool) { - switch { - case a == b: - return false, true - case a == nil: - return true, false - case b == nil: - return false, false - case a.Target == b.Target: - return a.Status < b.Status, a.Status == b.Status - default: - aIdx, bIdx := g.targetIdxMap[a.Target], g.targetIdxMap[b.Target] - return aIdx < bIdx, aIdx == bIdx - } -} diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go index 74d8749a362f..70346216ea97 100644 --- a/pkg/sql/schemachanger/scgraph/iteration.go +++ b/pkg/sql/schemachanger/scgraph/iteration.go @@ -13,7 +13,6 @@ package scgraph import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/util/iterutil" - "github.com/google/btree" ) // NodeIterator is used to iterate nodes. Return iterutil.StopIteration to @@ -60,29 +59,5 @@ type DepEdgeIterator func(de *DepEdge) error // ForEachDepEdgeFrom iterates the dep edges in the graph. func (g *Graph) ForEachDepEdgeFrom(n *scpb.Node, it DepEdgeIterator) (err error) { - g.nodeDepEdgesFrom.AscendGreaterOrEqual(&edgeTreeEntry{ - g: g, - edge: &DepEdge{ - from: n, - to: nil, - rule: "", - }, - order: fromTo, - }, - func(i btree.Item) bool { - e := i.(*edgeTreeEntry) - // End the iteration once the from nodes - // stop matching. - if e.edge.From() != n { - return false - } - if err = it(e.edge.(*DepEdge)); err != nil { - if iterutil.Done(err) { - err = nil - } - return false - } - return true - }) - return err + return g.nodeDepEdgesFrom.iterateSourceNode(n, it) } diff --git a/pkg/sql/schemachanger/scplan/deprules/registry.go b/pkg/sql/schemachanger/scplan/deprules/registry.go index dc5998dd6c78..ed5fb7d90c59 100644 --- a/pkg/sql/schemachanger/scplan/deprules/registry.go +++ b/pkg/sql/schemachanger/scplan/deprules/registry.go @@ -39,9 +39,10 @@ func Apply(g *scgraph.Graph) error { var depRules []rule type rule struct { - name string - from, to rel.Var - q *rel.Query + name string + from, to rel.Var + q *rel.Query + sameStage bool } func register(ruleName string, from, to rel.Var, query *rel.Query) { From b5e7b2767caf5bb3ee343db1a914ea0ca4941065 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 18:03:18 -0500 Subject: [PATCH 2/6] sql/schemachanger/scgraph: refactor rank ordering test Just pulling symbols into the test and making the table a bit more explicit. Release note: None --- pkg/sql/schemachanger/scgraph/graph_test.go | 252 ++++++++++---------- 1 file changed, 128 insertions(+), 124 deletions(-) diff --git a/pkg/sql/schemachanger/scgraph/graph_test.go b/pkg/sql/schemachanger/scgraph/graph_test.go index 424a3ce6dca8..48df7e930c2b 100644 --- a/pkg/sql/schemachanger/scgraph/graph_test.go +++ b/pkg/sql/schemachanger/scgraph/graph_test.go @@ -22,147 +22,151 @@ import ( "github.com/stretchr/testify/require" ) -type depEdge struct { - from, to int -} +// TestPlanGraphSort sanity checks sorting of the graph. +func TestGraphRanks(t *testing.T) { -func runRankTestForGraph( - t *testing.T, addNode []bool, depEdges []depEdge, expectedOrder []int, expectedRankErr string, -) { - // Setup a state based on if it is a add or drop. - state := scpb.State{ - Nodes: make([]*scpb.Node, 0, len(addNode)), - } - for idx := range addNode { - if addNode[idx] { - state.Nodes = append(state.Nodes, &scpb.Node{ - Target: scpb.NewTarget(scpb.Target_ADD, - &scpb.Table{ - TableID: descpb.ID(idx), - }, - nil /* metadata */), - Status: scpb.Status_ABSENT, - }) - } else { - state.Nodes = append(state.Nodes, &scpb.Node{ - Target: scpb.NewTarget(scpb.Target_DROP, - &scpb.Table{ - TableID: descpb.ID(idx), - }, - nil /* metadata */), - Status: scpb.Status_PUBLIC, - }) - } - } - // Setup the nodes first. - graph, err := scgraph.New(state) - require.NoError(t, err) - // Setup op edges for all the nodes. - for idx := range addNode { - if addNode[idx] { - require.NoError(t, graph.AddOpEdges(state.Nodes[idx].Target, - scpb.Status_ABSENT, - scpb.Status_PUBLIC, - true, - &scop.MakeColumnAbsent{})) - } else { - require.NoError(t, graph.AddOpEdges(state.Nodes[idx].Target, - scpb.Status_PUBLIC, - scpb.Status_ABSENT, - true, - &scop.MakeColumnAbsent{})) - } - } - // Add the dep edges next. - for _, edge := range depEdges { - require.NoError(t, graph.AddDepEdge( - fmt.Sprintf("%d to %d", edge.from, edge.to), - state.Nodes[edge.from].Target, - scpb.Status_PUBLIC, - state.Nodes[edge.to].Target, - scpb.Status_PUBLIC, - )) + type depEdge struct { + from, to int } - // Validates the rank order for nodes. - validateNodeRanks := func(graph *scgraph.Graph, expectedOrder []int) { - rank, err := graph.GetNodeRanks() - if expectedRankErr != "" { - require.Errorf(t, err, expectedRankErr) - return // Nothing else to validate - } else { - require.NoError(t, err) - } - unsortedNodes := make([]*scpb.Node, 0, len(state.Nodes)) - for _, node := range state.Nodes { - publicNode, ok := graph.GetNode(node.Target, scpb.Status_PUBLIC) - require.Truef(t, ok, "public node doesn't exist") - unsortedNodes = append(unsortedNodes, publicNode) - } - sort.SliceStable(unsortedNodes, func(i, j int) bool { - return rank[unsortedNodes[i]] > rank[unsortedNodes[j]] - }) - sortedOrder := make([]int, 0, len(unsortedNodes)) - for _, node := range unsortedNodes { - sortedOrder = append(sortedOrder, int(node.Table.TableID)) - } - require.EqualValues(t, expectedOrder, sortedOrder, "ranks are not in expected order") + type testCase struct { + name string + addNode []bool + depEdges []depEdge + expectedOrder []int + expectedRankErr string } - validateNodeRanks(graph, expectedOrder) -} -// TestPlanGraphSort sanity checks sorting of the graph. -func TestGraphRanks(t *testing.T) { - // We will set up the dependency graph for basic ordering, so that: - // 1) 0 depends on 1 - // 2) 3 depends on 0 - // 3) 2 depends on nothing - t.Run("simple dependency graph", func(t *testing.T) { - runRankTestForGraph(t, - []bool{true, true, true, true}, - []depEdge{ + testCases := []testCase{ + + // We will set up the dependency graph for basic ordering, so that + // 2 depends on nothing. + { + name: "simple dependency graph", + addNode: []bool{true, true, true, true}, + depEdges: []depEdge{ {0, 1}, {3, 0}, }, - []int{1, 0, 2, 3}, - "", - ) - }) + expectedOrder: []int{1, 0, 2, 3}, + }, - // We will set up the dependency graph, so that its - // intentionally cyclic, which should panic: - // 1) 0 depends on 1 - // 2) 3 depends on 0 - // 3) 1 depends on 3 - // 4) 3 depends on 1 - t.Run("cyclic graph", func(t *testing.T) { - runRankTestForGraph(t, - []bool{true, true, true, true}, - []depEdge{ + // We will set up the dependency graph, so that its intentionally cyclic, + // which should result in an error. + { + name: "cyclic graph", + addNode: []bool{true, true, true, true}, + depEdges: []depEdge{ {0, 1}, {3, 0}, {1, 3}, {3, 1}, }, - nil, // Not expecting this to run. - "graph is not a dag", - ) - }) + expectedRankErr: "graph is not a dag", + }, - // We will set up the dependency graph to have a swap - // 1) 0 (adding) depends on 1 (dropping) - // 2) 1 (dropping) depends on 0 (adding) - // 3) 2 (adding) depends on 0 (adding) - t.Run("dependency graph with a swap", func(t *testing.T) { - runRankTestForGraph(t, - []bool{true, false, true}, - []depEdge{ + // We will set up the dependency graph to have a swap: + // 1) 0 (adding) depends on 1 (dropping) + // 2) 1 (dropping) depends on 0 (adding) + // 3) 2 (adding) depends on 0 (adding) + { + + name: "dependency graph with a swap", + addNode: []bool{true, false, true}, + depEdges: []depEdge{ {0, 1}, {1, 0}, {2, 0}, }, - []int{1, 0, 2}, // We expect the drop to be ordered first. - "", - ) - }) + expectedOrder: []int{1, 0, 2}, // We expect the drop to be ordered first. + }, + } + + run := func( + t *testing.T, tc testCase, + ) { + // Setup a state based on if it is a add or drop. + state := scpb.State{ + Nodes: make([]*scpb.Node, 0, len(tc.addNode)), + } + for idx := range tc.addNode { + if tc.addNode[idx] { + state.Nodes = append(state.Nodes, &scpb.Node{ + Target: scpb.NewTarget(scpb.Target_ADD, + &scpb.Table{ + TableID: descpb.ID(idx), + }, + nil /* metadata */), + Status: scpb.Status_ABSENT, + }) + } else { + state.Nodes = append(state.Nodes, &scpb.Node{ + Target: scpb.NewTarget(scpb.Target_DROP, + &scpb.Table{ + TableID: descpb.ID(idx), + }, + nil /* metadata */), + Status: scpb.Status_PUBLIC, + }) + } + } + // Setup the nodes first. + graph, err := scgraph.New(state) + require.NoError(t, err) + // Setup op edges for all the nodes. + for idx := range tc.addNode { + if tc.addNode[idx] { + require.NoError(t, graph.AddOpEdges(state.Nodes[idx].Target, + scpb.Status_ABSENT, + scpb.Status_PUBLIC, + true, + &scop.MakeColumnAbsent{})) + } else { + require.NoError(t, graph.AddOpEdges(state.Nodes[idx].Target, + scpb.Status_PUBLIC, + scpb.Status_ABSENT, + true, + &scop.MakeColumnAbsent{})) + } + } + // Add the dep edges next. + for _, edge := range tc.depEdges { + require.NoError(t, graph.AddDepEdge( + fmt.Sprintf("%d to %d", edge.from, edge.to), + state.Nodes[edge.from].Target, + scpb.Status_PUBLIC, + state.Nodes[edge.to].Target, + scpb.Status_PUBLIC, + )) + } + + // Validates the rank order for nodes. + validateNodeRanks := func(graph *scgraph.Graph, expectedOrder []int) { + rank, err := graph.GetNodeRanks() + if tc.expectedRankErr != "" { + require.Regexp(t, tc.expectedRankErr, err) + return // Nothing else to validate + } else { + require.NoError(t, err) + } + unsortedNodes := make([]*scpb.Node, 0, len(state.Nodes)) + for _, node := range state.Nodes { + publicNode, ok := graph.GetNode(node.Target, scpb.Status_PUBLIC) + require.Truef(t, ok, "public node doesn't exist") + unsortedNodes = append(unsortedNodes, publicNode) + } + sort.SliceStable(unsortedNodes, func(i, j int) bool { + return rank[unsortedNodes[i]] > rank[unsortedNodes[j]] + }) + sortedOrder := make([]int, 0, len(unsortedNodes)) + for _, node := range unsortedNodes { + sortedOrder = append(sortedOrder, int(node.Table.TableID)) + } + require.EqualValues(t, expectedOrder, sortedOrder, "ranks are not in expected order") + } + validateNodeRanks(graph, tc.expectedOrder) + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { run(t, tc) }) + } } From 7d8f9fc4cbc962ed1f684be219f317750439abf2 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 18:19:08 -0500 Subject: [PATCH 3/6] sql/schemachanger/scgraph: add a simple test for (*Graph).compareNodes() Release note: None --- .../scgraph/dep_edge_tree_test.go | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go index 0b25a7aeef25..bcb34bce4090 100644 --- a/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go +++ b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go @@ -127,3 +127,55 @@ func TestDepEdgeTree(t *testing.T) { }) } } + +// TestGraphCompareNodes ensures the semantics of (*Graph).compareNodes is sane. +func TestGraphCompareNodes(t *testing.T) { + defer leaktest.AfterTest(t)() + t1 := scpb.NewTarget(scpb.Target_ADD, &scpb.Table{TableID: 1}, nil) + t2 := scpb.NewTarget(scpb.Target_DROP, &scpb.Table{TableID: 2}, nil) + mkNode := func(t *scpb.Target, s scpb.Status) *scpb.Node { + return &scpb.Node{Target: t, Status: s} + } + t1ABSENT := mkNode(t1, scpb.Status_ABSENT) + t2PUBLIC := mkNode(t2, scpb.Status_PUBLIC) + g, err := New(scpb.State{ + Nodes: []*scpb.Node{t1ABSENT, t2PUBLIC}, + }) + targetStr := func(target *scpb.Target) string { + switch target { + case t1: + return "t1" + case t2: + return "t2" + default: + panic("unexpected target") + } + } + nodeStr := func(n *scpb.Node) string { + if n == nil { + return "nil" + } + return fmt.Sprintf("%s:%s", targetStr(n.Target), n.Status.String()) + } + + require.NoError(t, err) + for _, tc := range []struct { + a, b *scpb.Node + less, eq bool + }{ + {a: nil, b: nil, less: false, eq: true}, + {a: t1ABSENT, b: nil, less: false, eq: false}, + {a: nil, b: t1ABSENT, less: true, eq: false}, + {a: t1ABSENT, b: t1ABSENT, less: false, eq: true}, + {a: t2PUBLIC, b: t1ABSENT, less: false, eq: false}, + {a: t1ABSENT, b: t2PUBLIC, less: true, eq: false}, + {a: t1ABSENT, b: mkNode(t1, scpb.Status_PUBLIC), less: true, eq: false}, + {a: mkNode(t1, scpb.Status_PUBLIC), b: t1ABSENT, less: false, eq: false}, + } { + t.Run(fmt.Sprintf("cmp(%s,%s)", nodeStr(tc.a), nodeStr(tc.b)), func(t *testing.T) { + less, eq := g.compareNodes(tc.a, tc.b) + require.Equal(t, tc.less, less, "less") + require.Equal(t, tc.eq, eq, "eq") + }) + } +} From 8e1a3135b5f8a90e3c0daf388e708f19d982f832 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 18:25:25 -0500 Subject: [PATCH 4/6] sql/schemachanger/scgraph: move edge defitions to a file Release note: None --- pkg/sql/schemachanger/scgraph/BUILD.bazel | 1 + pkg/sql/schemachanger/scgraph/edge.go | 70 ++++++++++++++++++ pkg/sql/schemachanger/scgraph/graph.go | 88 +++++------------------ 3 files changed, 88 insertions(+), 71 deletions(-) create mode 100644 pkg/sql/schemachanger/scgraph/edge.go diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel index e229c3cee857..ccb05e325211 100644 --- a/pkg/sql/schemachanger/scgraph/BUILD.bazel +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -4,6 +4,7 @@ go_library( name = "scgraph", srcs = [ "dep_edge_tree.go", + "edge.go", "graph.go", "iteration.go", ], diff --git a/pkg/sql/schemachanger/scgraph/edge.go b/pkg/sql/schemachanger/scgraph/edge.go new file mode 100644 index 000000000000..5c11b1b2b018 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/edge.go @@ -0,0 +1,70 @@ +// 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 scgraph + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" +) + +// Edge represents a relationship between two Nodes. +// +// TODO(ajwerner): Consider hiding Node pointers behind an interface to clarify +// mutability. +type Edge interface { + From() *scpb.Node + To() *scpb.Node +} + +// OpEdge represents an edge changing the state of a target with an op. +type OpEdge struct { + from, to *scpb.Node + op []scop.Op + typ scop.Type + revertible bool +} + +// From implements the Edge interface. +func (oe *OpEdge) From() *scpb.Node { return oe.from } + +// To implements the Edge interface. +func (oe *OpEdge) To() *scpb.Node { return oe.to } + +// Op returns the scop.Op for execution that is associated with the op edge. +func (oe *OpEdge) Op() []scop.Op { return oe.op } + +// Revertible returns if the dependency edge is revertible +func (oe *OpEdge) Revertible() bool { return oe.revertible } + +// Type returns the types of operations associated with this edge. +func (oe *OpEdge) Type() scop.Type { + return oe.typ +} + +// DepEdge represents a dependency between two nodes. A dependency +// implies that the To() node cannot be reached before the From() node. It +// can be reached concurrently. +type DepEdge struct { + from, to *scpb.Node + + // TODO(ajwerner): Deal with the possibility that multiple rules could + // generate the same edge. + rule string +} + +// From implements the Edge interface. +func (de *DepEdge) From() *scpb.Node { return de.from } + +// To implements the Edge interface. +func (de *DepEdge) To() *scpb.Node { return de.to } + +// Name returns the name of the rule which generated this edge. +func (de *DepEdge) Name() string { return de.rule } diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 20f88fb8f28a..0708291ec908 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -273,77 +273,6 @@ func (g *Graph) GetMetadataFromTarget(target *scpb.Target) scpb.ElementMetadata } } -// Edge represents a relationship between two Nodes. -// -// TODO(ajwerner): Consider hiding Node pointers behind an interface to clarify -// mutability. -type Edge interface { - From() *scpb.Node - To() *scpb.Node -} - -// OpEdge represents an edge changing the state of a target with an op. -type OpEdge struct { - from, to *scpb.Node - op []scop.Op - typ scop.Type - revertible bool -} - -// From implements the Edge interface. -func (oe *OpEdge) From() *scpb.Node { return oe.from } - -// To implements the Edge interface. -func (oe *OpEdge) To() *scpb.Node { return oe.to } - -// Op returns the scop.Op for execution that is associated with the op edge. -func (oe *OpEdge) Op() []scop.Op { return oe.op } - -// Revertible returns if the dependency edge is revertible -func (oe *OpEdge) Revertible() bool { return oe.revertible } - -// Type returns the types of operations associated with this edge. -func (oe *OpEdge) Type() scop.Type { - return oe.typ -} - -// DepEdge represents a dependency between two nodes. A dependency -// implies that the To() node cannot be reached before the From() node. It -// can be reached concurrently. -type DepEdge struct { - from, to *scpb.Node - - // TODO(ajwerner): Deal with the possibility that multiple rules could - // generate the same edge. - rule string -} - -// From implements the Edge interface. -func (de *DepEdge) From() *scpb.Node { return de.from } - -// To implements the Edge interface. -func (de *DepEdge) To() *scpb.Node { return de.to } - -// Name returns the name of the rule which generated this edge. -func (de *DepEdge) Name() string { return de.rule } - -// compareNodes compares two nodes in a graph. A nil nodes is the minimum value. -func (g *Graph) compareNodes(a, b *scpb.Node) (less, eq bool) { - switch { - case a == b: - return false, true - case a == nil: - return true, false - case b == nil: - return false, false - case a.Target == b.Target: - return a.Status < b.Status, a.Status == b.Status - default: - aIdx, bIdx := g.targetIdxMap[a.Target], g.targetIdxMap[b.Target] - return aIdx < bIdx, aIdx == bIdx - } -} - // GetNodeRanks fetches ranks of nodes in topological order. func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { defer func() { @@ -399,3 +328,20 @@ func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { } return rank, nil } + +// compareNodes compares two nodes in a graph. A nil nodes is the minimum value. +func (g *Graph) compareNodes(a, b *scpb.Node) (less, eq bool) { + switch { + case a == b: + return false, true + case a == nil: + return true, false + case b == nil: + return false, false + case a.Target == b.Target: + return a.Status < b.Status, a.Status == b.Status + default: + aIdx, bIdx := g.targetIdxMap[a.Target], g.targetIdxMap[b.Target] + return aIdx < bIdx, aIdx == bIdx + } +} From 2c7d4ab479d81f3e5cf4a50ba3d152fa468ca650 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 18:51:24 -0500 Subject: [PATCH 5/6] sql/schemachanger/scgraph: rename some fields Release note: None --- pkg/sql/schemachanger/scgraph/graph.go | 24 +++++++++++----------- pkg/sql/schemachanger/scgraph/iteration.go | 2 +- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 0708291ec908..86767c604eed 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -41,14 +41,14 @@ type Graph struct { // Maps a target to its index in targetNodes. targetIdxMap map[*scpb.Target]int - // nodeOpEdgesFrom maps a Node to an opEdge that proceeds + // opEdgesFrom maps a Node to an opEdge that proceeds // from it. A Node may have at most one opEdge from it. - nodeOpEdgesFrom map[*scpb.Node]*OpEdge + opEdgesFrom map[*scpb.Node]*OpEdge - // nodeDepEdgesFrom maps a Node from its dependencies. + // depEdgesFrom maps a Node from its dependencies. // A Node dependency is another target node which must be // reached before or concurrently with this node. - nodeDepEdgesFrom *depEdgeTree + depEdgesFrom *depEdgeTree // opToNode maps from an operation back to the // opEdge that generated it as an index. @@ -83,14 +83,14 @@ func New(initial scpb.State) (*Graph, error) { } g := Graph{ targetIdxMap: map[*scpb.Target]int{}, - nodeOpEdgesFrom: map[*scpb.Node]*OpEdge{}, + opEdgesFrom: map[*scpb.Node]*OpEdge{}, optimizedOutOpEdges: map[*OpEdge]bool{}, opToNode: map[scop.Op]*scpb.Node{}, entities: db, statements: initial.Statements, authorization: initial.Authorization, } - g.nodeDepEdgesFrom = newDepEdgeTree(fromTo, g.compareNodes) + g.depEdgesFrom = newDepEdgeTree(fromTo, g.compareNodes) for _, n := range initial.Nodes { if existing, ok := g.targetIdxMap[n.Target]; ok { return nil, errors.Errorf("invalid initial state contains duplicate target: %v and %v", n, initial.Nodes[existing]) @@ -118,8 +118,8 @@ func (g *Graph) ShallowClone() *Graph { authorization: g.authorization, targetNodes: g.targetNodes, targetIdxMap: g.targetIdxMap, - nodeOpEdgesFrom: g.nodeOpEdgesFrom, - nodeDepEdgesFrom: g.nodeDepEdgesFrom, + opEdgesFrom: g.opEdgesFrom, + depEdgesFrom: g.depEdgesFrom, opToNode: g.opToNode, edges: g.edges, entities: g.entities, @@ -177,7 +177,7 @@ var _ = (*Graph)(nil).containsTarget // GetOpEdgeFrom returns the unique outgoing op edge from the specified node, // if one exists. func (g *Graph) GetOpEdgeFrom(n *scpb.Node) (*OpEdge, bool) { - oe, ok := g.nodeOpEdgesFrom[n] + oe, ok := g.opEdgesFrom[n] return oe, ok } @@ -196,7 +196,7 @@ func (g *Graph) AddOpEdges( if oe.to, err = g.getOrCreateNode(t, to); err != nil { return err } - if existing, exists := g.nodeOpEdgesFrom[oe.from]; exists { + if existing, exists := g.opEdgesFrom[oe.from]; exists { return errors.Errorf("duplicate outbound op edge %v and %v", oe, existing) } @@ -211,7 +211,7 @@ func (g *Graph) AddOpEdges( } } oe.typ = typ - g.nodeOpEdgesFrom[oe.from] = oe + g.opEdgesFrom[oe.from] = oe // Store mapping from op to Edge for _, op := range ops { g.opToNode[op] = oe.To() @@ -243,7 +243,7 @@ func (g *Graph) AddDepEdge( return err } g.edges = append(g.edges, de) - g.nodeDepEdgesFrom.insert(de) + g.depEdgesFrom.insert(de) return nil } diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go index 70346216ea97..cd15c4676599 100644 --- a/pkg/sql/schemachanger/scgraph/iteration.go +++ b/pkg/sql/schemachanger/scgraph/iteration.go @@ -59,5 +59,5 @@ type DepEdgeIterator func(de *DepEdge) error // ForEachDepEdgeFrom iterates the dep edges in the graph. func (g *Graph) ForEachDepEdgeFrom(n *scpb.Node, it DepEdgeIterator) (err error) { - return g.nodeDepEdgesFrom.iterateSourceNode(n, it) + return g.depEdgesFrom.iterateSourceNode(n, it) } From 05080aa35aca1ba4e105abd1c373997a5a9be773 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 20:00:18 -0500 Subject: [PATCH 6/6] sql/schemachanger: remove cycle DepEdge rules, add SameStage kind This commit seeks to rectify an early mistake in the architecture of the declarative schema changer. In the original design, we knew we wanted certain transitions to happen in the same stage. In order to deal with that, we created rules that allowed for special types of cycles in dependencies to exist. This was a mistake. Instead, we replace this by a `Kind` property of `DepEdge`s which indicates whether the target pointed to merely needs to `HappenBefore` or whether it also needs to happen in the `SameStage`. This allows us to express exactly what we meant. This change also uncovered some broken cycles which never were intended to exist. The resultant plans generally look better. Release note: None --- pkg/sql/schemachanger/scgraph/BUILD.bazel | 8 + .../scgraph/depedgekind_string.go | 25 ++ pkg/sql/schemachanger/scgraph/edge.go | 23 ++ pkg/sql/schemachanger/scgraph/graph.go | 28 +-- pkg/sql/schemachanger/scgraph/graph_test.go | 10 +- pkg/sql/schemachanger/scgraph/iteration.go | 6 + pkg/sql/schemachanger/scplan/BUILD.bazel | 1 + .../schemachanger/scplan/deprules/registry.go | 13 +- .../schemachanger/scplan/deprules/rules.go | 76 ++---- .../scplan/deprules/testdata/rules | 81 ------- pkg/sql/schemachanger/scplan/plan.go | 44 +++- pkg/sql/schemachanger/scplan/plan_test.go | 2 + .../scplan/testdata/create_index | 14 ++ .../scplan/testdata/drop_database | 202 ++++++++++------ .../schemachanger/scplan/testdata/drop_schema | 170 +++++++++----- .../scplan/testdata/drop_sequence | 32 +-- .../schemachanger/scplan/testdata/drop_table | 46 ++-- .../schemachanger/scplan/testdata/drop_view | 112 +++++---- pkg/sql/schemachanger/testdata/drop | 217 +++++++----------- 19 files changed, 588 insertions(+), 522 deletions(-) create mode 100644 pkg/sql/schemachanger/scgraph/depedgekind_string.go diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel index ccb05e325211..e2fdedb30990 100644 --- a/pkg/sql/schemachanger/scgraph/BUILD.bazel +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//build:STRINGER.bzl", "stringer") go_library( name = "scgraph", @@ -7,6 +8,7 @@ go_library( "edge.go", "graph.go", "iteration.go", + ":gen-depedgekind-stringer", # keep ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph", visibility = ["//visibility:public"], @@ -37,3 +39,9 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +stringer( + name = "gen-depedgekind-stringer", + src = "edge.go", + typ = "DepEdgeKind", +) diff --git a/pkg/sql/schemachanger/scgraph/depedgekind_string.go b/pkg/sql/schemachanger/scgraph/depedgekind_string.go new file mode 100644 index 000000000000..ecabe22091d5 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/depedgekind_string.go @@ -0,0 +1,25 @@ +// Code generated by "stringer"; DO NOT EDIT. + +package scgraph + +import "strconv" + +func _() { + // An "invalid array index" compiler error signifies that the constant values have changed. + // Re-run the stringer command to generate them again. + var x [1]struct{} + _ = x[HappensAfter-1] + _ = x[SameStage-2] +} + +const _DepEdgeKind_name = "HappensAfterSameStage" + +var _DepEdgeKind_index = [...]uint8{0, 12, 21} + +func (i DepEdgeKind) String() string { + i -= 1 + if i < 0 || i >= DepEdgeKind(len(_DepEdgeKind_index)-1) { + return "DepEdgeKind(" + strconv.FormatInt(int64(i+1), 10) + ")" + } + return _DepEdgeKind_name[_DepEdgeKind_index[i]:_DepEdgeKind_index[i+1]] +} diff --git a/pkg/sql/schemachanger/scgraph/edge.go b/pkg/sql/schemachanger/scgraph/edge.go index 5c11b1b2b018..5b53cf2c408c 100644 --- a/pkg/sql/schemachanger/scgraph/edge.go +++ b/pkg/sql/schemachanger/scgraph/edge.go @@ -49,11 +49,31 @@ func (oe *OpEdge) Type() scop.Type { return oe.typ } +// DepEdgeKind indicates the kind of constraint enforced by the edge. +type DepEdgeKind int + +//go:generate stringer -type DepEdgeKind + +const ( + _ DepEdgeKind = iota + + // HappensAfter indicates that the source (from) of the edge must not be + // entered until after the destination (to) has entered the state. It could + // be in the same stage, or it could be in a subsequent stage. + HappensAfter + + // SameStage indicates that the source (from) of the edge must + // not be entered until after the destination (to) has entered the state and + // that both nodes must enter the state in the same stage. + SameStage +) + // DepEdge represents a dependency between two nodes. A dependency // implies that the To() node cannot be reached before the From() node. It // can be reached concurrently. type DepEdge struct { from, to *scpb.Node + kind DepEdgeKind // TODO(ajwerner): Deal with the possibility that multiple rules could // generate the same edge. @@ -68,3 +88,6 @@ func (de *DepEdge) To() *scpb.Node { return de.to } // Name returns the name of the rule which generated this edge. func (de *DepEdge) Name() string { return de.rule } + +// Kind returns the kind of the DepEdge. +func (de *DepEdge) Kind() DepEdgeKind { return de.kind } diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 86767c604eed..f6a105b98ee5 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -50,6 +50,10 @@ type Graph struct { // reached before or concurrently with this node. depEdgesFrom *depEdgeTree + // sameStageDepEdgesTo maps a Node to the DepEdges with the + // SameStage kind incident upon the indexed node. + sameStageDepEdgesTo *depEdgeTree + // opToNode maps from an operation back to the // opEdge that generated it as an index. opToNode map[scop.Op]*scpb.Node @@ -91,6 +95,7 @@ func New(initial scpb.State) (*Graph, error) { authorization: initial.Authorization, } g.depEdgesFrom = newDepEdgeTree(fromTo, g.compareNodes) + g.sameStageDepEdgesTo = newDepEdgeTree(toFrom, g.compareNodes) for _, n := range initial.Nodes { if existing, ok := g.targetIdxMap[n.Target]; ok { return nil, errors.Errorf("invalid initial state contains duplicate target: %v and %v", n, initial.Nodes[existing]) @@ -120,6 +125,7 @@ func (g *Graph) ShallowClone() *Graph { targetIdxMap: g.targetIdxMap, opEdgesFrom: g.opEdgesFrom, depEdgesFrom: g.depEdgesFrom, + sameStageDepEdgesTo: g.sameStageDepEdgesTo, opToNode: g.opToNode, edges: g.edges, entities: g.entities, @@ -230,12 +236,13 @@ var _ = (*Graph)(nil).GetNodeFromOp // and statuses). func (g *Graph) AddDepEdge( rule string, + kind DepEdgeKind, fromTarget *scpb.Target, fromStatus scpb.Status, toTarget *scpb.Target, toStatus scpb.Status, ) (err error) { - de := &DepEdge{rule: rule} + de := &DepEdge{rule: rule, kind: kind} if de.from, err = g.getOrCreateNode(fromTarget, fromStatus); err != nil { return err } @@ -244,6 +251,9 @@ func (g *Graph) AddDepEdge( } g.edges = append(g.edges, de) g.depEdgesFrom.insert(de) + if de.Kind() == SameStage { + g.sameStageDepEdgesTo.insert(de) + } return nil } @@ -284,14 +294,6 @@ func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { err = rAsErr } }() - backCycleExists := func(n *scpb.Node, de *DepEdge) bool { - var foundBack bool - _ = g.ForEachDepEdgeFrom(de.To(), func(maybeBack *DepEdge) error { - foundBack = foundBack || maybeBack.To() == n - return nil - }) - return foundBack - } l := list.New() marks := make(map[*scpb.Node]bool) var visit func(n *scpb.Node) @@ -305,18 +307,12 @@ func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { } marks[n] = false _ = g.ForEachDepEdgeFrom(n, func(de *DepEdge) error { - // We want to eliminate cycles caused by swaps. In that - // case, we want to pretend that there is no edge from the - // add to the drop, and, in that way, the drop is ordered first. - if n.Direction == scpb.Target_ADD || !backCycleExists(n, de) { - visit(de.To()) - } + visit(de.To()) return nil }) marks[n] = true l.PushFront(n) } - _ = g.ForEachNode(func(n *scpb.Node) error { visit(n) return nil diff --git a/pkg/sql/schemachanger/scgraph/graph_test.go b/pkg/sql/schemachanger/scgraph/graph_test.go index 48df7e930c2b..2770fc27faa3 100644 --- a/pkg/sql/schemachanger/scgraph/graph_test.go +++ b/pkg/sql/schemachanger/scgraph/graph_test.go @@ -65,12 +65,9 @@ func TestGraphRanks(t *testing.T) { expectedRankErr: "graph is not a dag", }, - // We will set up the dependency graph to have a swap: - // 1) 0 (adding) depends on 1 (dropping) - // 2) 1 (dropping) depends on 0 (adding) - // 3) 2 (adding) depends on 0 (adding) + // We will set up the dependency graph to have a swap, which won't affect + // the fact that there's still a cycle. { - name: "dependency graph with a swap", addNode: []bool{true, false, true}, depEdges: []depEdge{ @@ -78,7 +75,7 @@ func TestGraphRanks(t *testing.T) { {1, 0}, {2, 0}, }, - expectedOrder: []int{1, 0, 2}, // We expect the drop to be ordered first. + expectedRankErr: "graph is not a dag", }, } @@ -133,6 +130,7 @@ func TestGraphRanks(t *testing.T) { for _, edge := range tc.depEdges { require.NoError(t, graph.AddDepEdge( fmt.Sprintf("%d to %d", edge.from, edge.to), + scgraph.HappensAfter, state.Nodes[edge.from].Target, scpb.Status_PUBLIC, state.Nodes[edge.to].Target, diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go index cd15c4676599..19c9378ddc22 100644 --- a/pkg/sql/schemachanger/scgraph/iteration.go +++ b/pkg/sql/schemachanger/scgraph/iteration.go @@ -61,3 +61,9 @@ type DepEdgeIterator func(de *DepEdge) error func (g *Graph) ForEachDepEdgeFrom(n *scpb.Node, it DepEdgeIterator) (err error) { return g.depEdgesFrom.iterateSourceNode(n, it) } + +// ForEachSameStageDepEdgeTo iterates the dep edges in the graph of kind +// SameStage which point to the provided node. +func (g *Graph) ForEachSameStageDepEdgeTo(n *scpb.Node, it DepEdgeIterator) (err error) { + return g.sameStageDepEdgesTo.iterateSourceNode(n, it) +} diff --git a/pkg/sql/schemachanger/scplan/BUILD.bazel b/pkg/sql/schemachanger/scplan/BUILD.bazel index e626a7c3cc31..e70404c23568 100644 --- a/pkg/sql/schemachanger/scplan/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/sql/schemachanger/scplan/deprules", "//pkg/sql/schemachanger/scplan/opgen", "//pkg/sql/schemachanger/scplan/scopt", + "//pkg/sql/schemachanger/screl", "//pkg/util/iterutil", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/sql/schemachanger/scplan/deprules/registry.go b/pkg/sql/schemachanger/scplan/deprules/registry.go index ed5fb7d90c59..99366c66eac1 100644 --- a/pkg/sql/schemachanger/scplan/deprules/registry.go +++ b/pkg/sql/schemachanger/scplan/deprules/registry.go @@ -26,7 +26,7 @@ func Apply(g *scgraph.Graph) error { from := r.Var(dr.from).(*scpb.Node) to := r.Var(dr.to).(*scpb.Node) return g.AddDepEdge( - dr.name, from.Target, from.Status, to.Target, to.Status, + dr.name, dr.kind, from.Target, from.Status, to.Target, to.Status, ) }); err != nil { return err @@ -39,15 +39,16 @@ func Apply(g *scgraph.Graph) error { var depRules []rule type rule struct { - name string - from, to rel.Var - q *rel.Query - sameStage bool + name string + from, to rel.Var + q *rel.Query + kind scgraph.DepEdgeKind } -func register(ruleName string, from, to rel.Var, query *rel.Query) { +func register(ruleName string, edgeKind scgraph.DepEdgeKind, from, to rel.Var, query *rel.Query) { depRules = append(depRules, rule{ name: ruleName, + kind: edgeKind, from: from, to: to, q: query, diff --git a/pkg/sql/schemachanger/scplan/deprules/rules.go b/pkg/sql/schemachanger/scplan/deprules/rules.go index fe63c69511fe..40f2f928d33f 100644 --- a/pkg/sql/schemachanger/scplan/deprules/rules.go +++ b/pkg/sql/schemachanger/scplan/deprules/rules.go @@ -13,6 +13,7 @@ package deprules import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" "github.com/cockroachdb/errors" @@ -52,9 +53,9 @@ func init() { // are dropped before any children are dealt with. register( "parent dependencies", + scgraph.HappensAfter, parentNode, otherNode, screl.MustQuery( - parent.Type((*scpb.Database)(nil), (*scpb.Schema)(nil)), other.Type( (*scpb.Type)(nil), (*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil), @@ -102,9 +103,9 @@ func init() { var id, status, direction rel.Var = "id", "index-status", "direction" register( "column depends on indexes", + scgraph.HappensAfter, columnNode, indexNode, screl.MustQuery( - status.In(deleteAndWriteOnly, public), direction.Eq(add), @@ -145,9 +146,9 @@ func init() { register( "index depends on column", + scgraph.HappensAfter, indexNode, columnNode, screl.MustQuery( - column.Type((*scpb.Column)(nil)), index.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), @@ -204,14 +205,10 @@ func init() { register( "primary index add depends on drop", + scgraph.SameStage, addNode, dropNode, primaryIndexReferenceEachOther, ) - register( - "primary index drop depends on add", - dropNode, addNode, - primaryIndexReferenceEachOther, - ) } func init() { @@ -222,6 +219,7 @@ func init() { register( "partitioning information needs the basic index as created", + scgraph.HappensAfter, partitioningNode, addNode, screl.MustQuery( addIdx.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), @@ -244,6 +242,7 @@ func init() { register( "index needs partitioning information to be filled", + scgraph.HappensAfter, partitioningNode, addNode, screl.MustQuery( addIdx.Type((*scpb.PrimaryIndex)(nil)), @@ -268,6 +267,7 @@ func init() { var id rel.Var = "id" register( "dependency needs relation/type as non-synthetically dropped", + scgraph.SameStage, depNode, relationNode, screl.MustQuery( @@ -293,52 +293,6 @@ func init() { screl.ReferencedDescID) } -func init() { - relationNeedsDepToBeRemoved := func(ruleName string, depTypes []interface{}, depDescIDMatch rel.Attr, swapped bool) { - // Before any parts of a relation can be dropped, the relation - // should exit the synthetic drop state. - relation, relationTarget, relationNode := targetNodeVars("relation") - dep, depTarget, depNode := targetNodeVars("dep") - var id rel.Var = "id" - firstNode, secondNode := relationNode, depNode - if swapped { - firstNode, secondNode = depNode, relationNode - } - register( - "relation/type needs dependency as dropped", - firstNode, secondNode, - screl.MustQuery( - - relation.Type((*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil), (*scpb.Type)(nil)), - dep.Type(depTypes[0], depTypes[1:]...), - - id.Entities(screl.DescID, relation, dep), - - joinTargetNode(relation, relationTarget, relationNode, drop, absent), - joinTargetNode(dep, depTarget, depNode, drop, absent), - ), - ) - } - relationNeedsDepToBeRemoved("relation/type needs dependency as dropped", - []interface{}{(*scpb.DefaultExpression)(nil), (*scpb.RelationDependedOnBy)(nil), - (*scpb.SequenceOwnedBy)(nil), (*scpb.ForeignKey)(nil)}, - screl.DescID, - false /*swapped*/) - - relationNeedsDepToBeRemoved("relation/type (ref desc) needs dependency as dropped", - []interface{}{(*scpb.ForeignKeyBackReference)(nil), - (*scpb.ViewDependsOnType)(nil), (*scpb.DefaultExprTypeReference)(nil), - (*scpb.OnUpdateExprTypeReference)(nil), (*scpb.ComputedExprTypeReference)(nil), - (*scpb.ColumnTypeReference)(nil)}, - screl.ReferencedDescID, - false /*swapped*/) - - relationNeedsDepToBeRemoved("relation dependency clean up needs dependent relation to be dropped first", - []interface{}{(*scpb.RelationDependedOnBy)(nil)}, - screl.ReferencedDescID, - true /*swapped*/) -} - func init() { // Ensures that the name is drained first, only when // the descriptor is cleaned up. @@ -347,9 +301,9 @@ func init() { tabID := rel.Var("desc-id") register( "namespace needs descriptor to be dropped", + scgraph.HappensAfter, nsNode, depNode, screl.MustQuery( - ns.Type((*scpb.Namespace)(nil)), dep.Type((*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil), (*scpb.Database)(nil), (*scpb.Schema)(nil)), @@ -365,9 +319,9 @@ func init() { // dropped. register( "descriptor can only be cleaned up once the name is drained", + scgraph.HappensAfter, depNode, nsNode, screl.MustQuery( - ns.Type((*scpb.Namespace)(nil)), dep.Type((*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil), (*scpb.Database)(nil), (*scpb.Schema)(nil)), @@ -388,6 +342,7 @@ func init() { register( "column name is assigned once the column is created", + scgraph.HappensAfter, columnNameNode, columnNode, screl.MustQuery( @@ -404,6 +359,7 @@ func init() { register( "column needs a name to be assigned", + scgraph.HappensAfter, columnNode, columnNameNode, screl.MustQuery( @@ -427,9 +383,9 @@ func init() { register( "index name is assigned once the index is created", + scgraph.HappensAfter, indexNameNode, indexNode, screl.MustQuery( - indexName.Type((*scpb.IndexName)(nil)), index.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), @@ -443,9 +399,9 @@ func init() { register( "index needs a name to be assigned", + scgraph.HappensAfter, indexNode, indexNameNode, screl.MustQuery( - indexName.Type((*scpb.IndexName)(nil)), index.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), @@ -466,7 +422,9 @@ func init() { typeID := rel.Var("type-id") tableID := rel.Var("table-id") - register("type ref drop is no-op if ref is being added", + register( + "type ref drop is no-op if ref is being added", + scgraph.HappensAfter, typeRefDropNode, typeRefDropNode, screl.MustQuery( typeRefDrop.Type((*scpb.DefaultExprTypeReference)(nil), (*scpb.ColumnTypeReference)(nil), diff --git a/pkg/sql/schemachanger/scplan/deprules/testdata/rules b/pkg/sql/schemachanger/scplan/deprules/testdata/rules index cc09123a72ed..fd61857d68c0 100644 --- a/pkg/sql/schemachanger/scplan/deprules/testdata/rules +++ b/pkg/sql/schemachanger/scplan/deprules/testdata/rules @@ -84,27 +84,6 @@ rules - $drop-idx-node[Target] = $drop-idx-target - $drop-idx-target[Direction] = DROP - $drop-idx-node[Status] = DELETE_AND_WRITE_ONLY -- name: primary index drop depends on add - from: drop-idx-node - to: add-idx-node - query: - - $add-idx[Type] = '*scpb.PrimaryIndex' - - $drop-idx[Type] = '*scpb.PrimaryIndex' - - $add-idx[DescID] = $id - - $drop-idx[DescID] = $id - - referenceEachOther(*scpb.PrimaryIndex, *scpb.PrimaryIndex)($add-idx, $drop-idx) - - $add-idx-target[Type] = '*scpb.Target' - - $add-idx-target[Element] = $add-idx - - $add-idx-node[Type] = '*scpb.Node' - - $add-idx-node[Target] = $add-idx-target - - $add-idx-target[Direction] = ADD - - $add-idx-node[Status] = PUBLIC - - $drop-idx-target[Type] = '*scpb.Target' - - $drop-idx-target[Element] = $drop-idx - - $drop-idx-node[Type] = '*scpb.Node' - - $drop-idx-node[Target] = $drop-idx-target - - $drop-idx-target[Direction] = DROP - - $drop-idx-node[Status] = DELETE_AND_WRITE_ONLY - name: partitioning information needs the basic index as created from: partitioning-node to: add-idx-node @@ -189,66 +168,6 @@ rules - $dep-node[Target] = $dep-target - $dep-target[Direction] = DROP - $dep-node[Status] = ABSENT -- name: relation/type needs dependency as dropped - from: relation-node - to: dep-node - query: - - $relation[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence', '*scpb.Type'] - - $dep[Type] IN ['*scpb.DefaultExpression', '*scpb.RelationDependedOnBy', '*scpb.SequenceOwnedBy', '*scpb.ForeignKey'] - - $relation[DescID] = $id - - $dep[DescID] = $id - - $relation-target[Type] = '*scpb.Target' - - $relation-target[Element] = $relation - - $relation-node[Type] = '*scpb.Node' - - $relation-node[Target] = $relation-target - - $relation-target[Direction] = DROP - - $relation-node[Status] = ABSENT - - $dep-target[Type] = '*scpb.Target' - - $dep-target[Element] = $dep - - $dep-node[Type] = '*scpb.Node' - - $dep-node[Target] = $dep-target - - $dep-target[Direction] = DROP - - $dep-node[Status] = ABSENT -- name: relation/type needs dependency as dropped - from: relation-node - to: dep-node - query: - - $relation[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence', '*scpb.Type'] - - $dep[Type] IN ['*scpb.ForeignKeyBackReference', '*scpb.ViewDependsOnType', '*scpb.DefaultExprTypeReference', '*scpb.OnUpdateExprTypeReference', '*scpb.ComputedExprTypeReference', '*scpb.ColumnTypeReference'] - - $relation[DescID] = $id - - $dep[DescID] = $id - - $relation-target[Type] = '*scpb.Target' - - $relation-target[Element] = $relation - - $relation-node[Type] = '*scpb.Node' - - $relation-node[Target] = $relation-target - - $relation-target[Direction] = DROP - - $relation-node[Status] = ABSENT - - $dep-target[Type] = '*scpb.Target' - - $dep-target[Element] = $dep - - $dep-node[Type] = '*scpb.Node' - - $dep-node[Target] = $dep-target - - $dep-target[Direction] = DROP - - $dep-node[Status] = ABSENT -- name: relation/type needs dependency as dropped - from: dep-node - to: relation-node - query: - - $relation[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence', '*scpb.Type'] - - $dep[Type] = '*scpb.RelationDependedOnBy' - - $relation[DescID] = $id - - $dep[DescID] = $id - - $relation-target[Type] = '*scpb.Target' - - $relation-target[Element] = $relation - - $relation-node[Type] = '*scpb.Node' - - $relation-node[Target] = $relation-target - - $relation-target[Direction] = DROP - - $relation-node[Status] = ABSENT - - $dep-target[Type] = '*scpb.Target' - - $dep-target[Element] = $dep - - $dep-node[Type] = '*scpb.Node' - - $dep-node[Target] = $dep-target - - $dep-target[Direction] = DROP - - $dep-node[Status] = ABSENT - name: namespace needs descriptor to be dropped from: namespace-node to: dep-node diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go index 6996972708a2..d1251d7ee804 100644 --- a/pkg/sql/schemachanger/scplan/plan.go +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/deprules" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/opgen" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/scopt" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/errors" ) @@ -111,24 +112,59 @@ func buildStages(init scpb.State, g *scgraph.Graph, params Params) []Stage { cur := init fulfilled := map[*scpb.Node]struct{}{} filterUnsatisfiedEdgesStep := func(edges []*scgraph.OpEdge) ([]*scgraph.OpEdge, bool) { - candidates := make(map[*scpb.Node]struct{}) + candidates := make(map[*scpb.Node]*scgraph.OpEdge) for _, e := range edges { - candidates[e.To()] = struct{}{} + candidates[e.To()] = e } // Check to see if the current set of edges will have their dependencies met // if they are all run. Any which will not must be pruned. This greedy // algorithm works, but a justification is in order. failed := map[*scgraph.OpEdge]struct{}{} for _, e := range edges { - _ = g.ForEachDepEdgeFrom(e.To(), func(de *scgraph.DepEdge) error { + if err := g.ForEachDepEdgeFrom(e.To(), func(de *scgraph.DepEdge) error { _, isFulfilled := fulfilled[de.To()] _, isCandidate := candidates[de.To()] + if de.Kind() == scgraph.SameStage && isFulfilled { + // This is bad, we have a happens-after relationship, and it has + // already happened. + return errors.AssertionFailedf("failed to satisfy %v->%v (%s) dependency", + screl.NodeString(de.From()), screl.NodeString(de.To()), de.Name()) + } if isFulfilled || isCandidate { return nil } failed[e] = struct{}{} return iterutil.StopIteration() - }) + }); err != nil { + panic(err) + } + } + // Ensure that all SameStage DepEdges are met appropriately. + for _, e := range edges { + if err := g.ForEachSameStageDepEdgeTo(e.To(), func(de *scgraph.DepEdge) error { + if _, isFulfilled := fulfilled[de.From()]; isFulfilled { + // This is bad, we have a happens-after relationship, and it has + // already happened. + return errors.AssertionFailedf("failed to satisfy %v->%v (%s) dependency", + screl.NodeString(de.From()), screl.NodeString(de.To()), de.Name()) + } + fromCandidate, fromIsCandidate := candidates[de.From()] + if !fromIsCandidate { + failed[e] = struct{}{} + return iterutil.StopIteration() + } + _, fromIsFailed := failed[fromCandidate] + if fromIsFailed { + failed[e] = struct{}{} + return iterutil.StopIteration() + } + if _, eIsFailed := failed[e]; eIsFailed { + failed[fromCandidate] = struct{}{} + } + return nil + }); err != nil { + panic(err) + } } if len(failed) == 0 { return edges, true diff --git a/pkg/sql/schemachanger/scplan/plan_test.go b/pkg/sql/schemachanger/scplan/plan_test.go index 5faadcec6406..2a173c5f677a 100644 --- a/pkg/sql/schemachanger/scplan/plan_test.go +++ b/pkg/sql/schemachanger/scplan/plan_test.go @@ -156,6 +156,8 @@ func marshalDeps(t *testing.T, plan *scplan.Plan) string { screl.ElementString(de.From().Element()), de.From().Status) fmt.Fprintf(&deps, " to: [%s, %s]\n", screl.ElementString(de.To().Element()), de.To().Status) + fmt.Fprintf(&deps, " kind: %s\n", de.Kind()) + fmt.Fprintf(&deps, " rule: %s\n", de.Name()) sortedDeps = append(sortedDeps, deps.String()) return nil }) diff --git a/pkg/sql/schemachanger/scplan/testdata/create_index b/pkg/sql/schemachanger/scplan/testdata/create_index index bb3330f00415..8dfcdd3dffb3 100644 --- a/pkg/sql/schemachanger/scplan/testdata/create_index +++ b/pkg/sql/schemachanger/scplan/testdata/create_index @@ -60,8 +60,12 @@ CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) ---- - from: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] to: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_ONLY] + kind: HappensAfter + rule: index name is assigned once the index is created - from: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_AND_WRITE_ONLY] to: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] + kind: HappensAfter + rule: index needs a name to be assigned ops CREATE INVERTED INDEX concurrently id1 on defaultdb.t1(id, name) storing (money) @@ -123,8 +127,12 @@ CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) ---- - from: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] to: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_ONLY] + kind: HappensAfter + rule: index name is assigned once the index is created - from: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_AND_WRITE_ONLY] to: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] + kind: HappensAfter + rule: index needs a name to be assigned ops CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) PARTITION BY LIST (id) ( @@ -199,7 +207,13 @@ CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) PARTITION BY LIST (i ---- - from: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] to: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_ONLY] + kind: HappensAfter + rule: index name is assigned once the index is created - from: [Partitioning:{DescID: 52, IndexID: 2}, PUBLIC] to: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_ONLY] + kind: HappensAfter + rule: partitioning information needs the basic index as created - from: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_AND_WRITE_ONLY] to: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] + kind: HappensAfter + rule: index needs a name to be assigned diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_database b/pkg/sql/schemachanger/scplan/testdata/drop_database index 83d121ba897d..8f0b809a6e20 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_database +++ b/pkg/sql/schemachanger/scplan/testdata/drop_database @@ -133,6 +133,7 @@ Stage 1 Stage 2 (non-revertible) transitions: [Sequence:{DescID: 54}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 57}, TXN_DROPPED, DROP] -> DROPPED [Column:{DescID: 57, ColumnID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 57, ColumnID: 2}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY @@ -140,16 +141,23 @@ Stage 2 (non-revertible) [DefaultExpression:{DescID: 57, ColumnID: 3}, PUBLIC, DROP] -> ABSENT [PrimaryIndex:{DescID: 57, IndexID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Sequence:{DescID: 55}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 56}, TXN_DROPPED, DROP] -> DROPPED [Column:{DescID: 56, ColumnID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 56, ColumnID: 2}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 56, ColumnID: 3}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [DefaultExpression:{DescID: 56, ColumnID: 3}, PUBLIC, DROP] -> ABSENT [PrimaryIndex:{DescID: 56, IndexID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY + [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT [View:{DescID: 58}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT [View:{DescID: 59}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, PUBLIC, DROP] -> ABSENT [View:{DescID: 60}, TXN_DROPPED, DROP] -> DROPPED [View:{DescID: 61}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, PUBLIC, DROP] -> ABSENT [View:{DescID: 64}, TXN_DROPPED, DROP] -> DROPPED [ViewDependsOnType:{DescID: 64, ReferencedDescID: 62}, PUBLIC, DROP] -> ABSENT [ViewDependsOnType:{DescID: 64, ReferencedDescID: 63}, PUBLIC, DROP] -> ABSENT @@ -162,6 +170,9 @@ Stage 2 (non-revertible) DescID: 54 *scop.MarkDescriptorAsDropped DescID: 57 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 54 *scop.RemoveColumnDefaultExpression ColumnID: 3 TableID: 57 @@ -171,6 +182,9 @@ Stage 2 (non-revertible) DescID: 55 *scop.MarkDescriptorAsDropped DescID: 56 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 56 + TableID: 55 *scop.RemoveColumnDefaultExpression ColumnID: 3 TableID: 56 @@ -178,14 +192,32 @@ Stage 2 (non-revertible) TableID: 56 *scop.MarkDescriptorAsDropped DescID: 58 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 58 + TableID: 56 *scop.MarkDescriptorAsDropped DescID: 59 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 59 + TableID: 58 *scop.MarkDescriptorAsDropped DescID: 60 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 60 + TableID: 58 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 60 + TableID: 59 *scop.MarkDescriptorAsDropped DescID: 61 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 61 + TableID: 59 *scop.MarkDescriptorAsDropped DescID: 64 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 64 + TableID: 61 *scop.MarkDescriptorAsDropped DescID: 62 *scop.RemoveTypeBackRef @@ -203,29 +235,21 @@ Stage 2 (non-revertible) Stage 3 (non-revertible) transitions: [Sequence:{DescID: 54}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 57}, DROPPED, DROP] -> ABSENT [Column:{DescID: 57, ColumnID: 1}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 57, ColumnID: 2}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 57, ColumnID: 3}, DELETE_ONLY, DROP] -> ABSENT [PrimaryIndex:{DescID: 57, IndexID: 1}, DELETE_ONLY, DROP] -> ABSENT [Sequence:{DescID: 55}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 56}, DROPPED, DROP] -> ABSENT [Column:{DescID: 56, ColumnID: 1}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 56, ColumnID: 2}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 56, ColumnID: 3}, DELETE_ONLY, DROP] -> ABSENT [PrimaryIndex:{DescID: 56, IndexID: 1}, DELETE_ONLY, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT [View:{DescID: 58}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT [View:{DescID: 59}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, PUBLIC, DROP] -> ABSENT [View:{DescID: 60}, DROPPED, DROP] -> ABSENT [View:{DescID: 61}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, PUBLIC, DROP] -> ABSENT [View:{DescID: 64}, DROPPED, DROP] -> ABSENT [Type:{DescID: 62}, DROPPED, DROP] -> ABSENT [Type:{DescID: 63}, DROPPED, DROP] -> ABSENT @@ -246,9 +270,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 54 *scop.LogEvent DescID: 57 Direction: 2 @@ -277,9 +298,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 56 - TableID: 55 *scop.LogEvent DescID: 56 Direction: 2 @@ -294,9 +312,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 56 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 58 - TableID: 56 *scop.LogEvent DescID: 58 Direction: 2 @@ -311,12 +326,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 58 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 59 - TableID: 58 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 60 - TableID: 58 *scop.LogEvent DescID: 59 Direction: 2 @@ -331,12 +340,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 59 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 60 - TableID: 59 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 61 - TableID: 59 *scop.LogEvent DescID: 60 Direction: 2 @@ -365,9 +368,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 61 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 64 - TableID: 61 *scop.LogEvent DescID: 64 Direction: 2 @@ -434,139 +434,197 @@ DROP DATABASE db1 CASCADE ---- - from: [Database:{DescID: 52}, ABSENT] to: [Sequence:{DescID: 54}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Database:{DescID: 52}, ABSENT] to: [Table:{DescID: 57}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [DefaultExpression:{DescID: 56, ColumnID: 3}, ABSENT] to: [Table:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [DefaultExpression:{DescID: 57, ColumnID: 3}, ABSENT] to: [Table:{DescID: 57}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Namespace:{DescID: 54, Name: sq1}, ABSENT] to: [Sequence:{DescID: 54}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 55, Name: sq1}, ABSENT] to: [Sequence:{DescID: 55}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 56, Name: t1}, ABSENT] to: [Table:{DescID: 56}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 57, Name: t1}, ABSENT] to: [Table:{DescID: 57}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 58, Name: v1}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 59, Name: v2}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 60, Name: v3}, ABSENT] to: [View:{DescID: 60}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 61, Name: v4}, ABSENT] to: [View:{DescID: 61}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 64, Name: v5}, ABSENT] to: [View:{DescID: 64}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, ABSENT] - to: [Sequence:{DescID: 54}, ABSENT] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, ABSENT] to: [Sequence:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, ABSENT] to: [Table:{DescID: 57}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] - to: [Sequence:{DescID: 55}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] to: [Sequence:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] to: [Table:{DescID: 56}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] - to: [Table:{DescID: 56}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] to: [Table:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] to: [View:{DescID: 58}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, ABSENT] - to: [View:{DescID: 58}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, ABSENT] to: [View:{DescID: 59}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, ABSENT] - to: [View:{DescID: 58}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, ABSENT] to: [View:{DescID: 60}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, ABSENT] - to: [View:{DescID: 59}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, ABSENT] to: [View:{DescID: 60}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, ABSENT] - to: [View:{DescID: 59}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, ABSENT] to: [View:{DescID: 61}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, ABSENT] - to: [View:{DescID: 61}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, ABSENT] to: [View:{DescID: 61}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, ABSENT] to: [View:{DescID: 64}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Schema:{DescID: 53}, ABSENT] to: [Sequence:{DescID: 55}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [Table:{DescID: 56}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [Type:{DescID: 62}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [Type:{DescID: 63}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 58}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 59}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 60}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 61}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 64}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Sequence:{DescID: 54}, ABSENT] to: [Namespace:{DescID: 54, Name: sq1}, ABSENT] -- from: [Sequence:{DescID: 54}, ABSENT] - to: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [Sequence:{DescID: 55}, ABSENT] to: [Namespace:{DescID: 55, Name: sq1}, ABSENT] -- from: [Sequence:{DescID: 55}, ABSENT] - to: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] -- from: [Table:{DescID: 56}, ABSENT] - to: [DefaultExpression:{DescID: 56, ColumnID: 3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [Table:{DescID: 56}, ABSENT] to: [Namespace:{DescID: 56, Name: t1}, ABSENT] -- from: [Table:{DescID: 56}, ABSENT] - to: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] -- from: [Table:{DescID: 57}, ABSENT] - to: [DefaultExpression:{DescID: 57, ColumnID: 3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [Table:{DescID: 57}, ABSENT] to: [Namespace:{DescID: 57, Name: t1}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 58}, ABSENT] to: [Namespace:{DescID: 58, Name: v1}, ABSENT] -- from: [View:{DescID: 58}, ABSENT] - to: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, ABSENT] -- from: [View:{DescID: 58}, ABSENT] - to: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 59}, ABSENT] to: [Namespace:{DescID: 59, Name: v2}, ABSENT] -- from: [View:{DescID: 59}, ABSENT] - to: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, ABSENT] -- from: [View:{DescID: 59}, ABSENT] - to: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 60}, ABSENT] to: [Namespace:{DescID: 60, Name: v3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 61}, ABSENT] to: [Namespace:{DescID: 61, Name: v4}, ABSENT] -- from: [View:{DescID: 61}, ABSENT] - to: [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 64}, ABSENT] to: [Namespace:{DescID: 64, Name: v5}, ABSENT] -- from: [View:{DescID: 64}, ABSENT] - to: [ViewDependsOnType:{DescID: 64, ReferencedDescID: 62}, ABSENT] -- from: [View:{DescID: 64}, ABSENT] - to: [ViewDependsOnType:{DescID: 64, ReferencedDescID: 63}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [ViewDependsOnType:{DescID: 64, ReferencedDescID: 62}, ABSENT] to: [Type:{DescID: 62}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [ViewDependsOnType:{DescID: 64, ReferencedDescID: 63}, ABSENT] to: [Type:{DescID: 63}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_schema b/pkg/sql/schemachanger/scplan/testdata/drop_schema index a46cfd5bf1d8..714bda07cb90 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_schema +++ b/pkg/sql/schemachanger/scplan/testdata/drop_schema @@ -39,118 +39,164 @@ DROP SCHEMA defaultdb.SC1 CASCADE ---- - from: [DefaultExpression:{DescID: 54, ColumnID: 3}, ABSENT] to: [Table:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Namespace:{DescID: 53, Name: sq1}, ABSENT] to: [Sequence:{DescID: 53}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 54, Name: t1}, ABSENT] to: [Table:{DescID: 54}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 55, Name: v1}, ABSENT] to: [View:{DescID: 55}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 56, Name: v2}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 57, Name: v3}, ABSENT] to: [View:{DescID: 57}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 58, Name: v4}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 61, Name: v5}, ABSENT] to: [View:{DescID: 61}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] - to: [Sequence:{DescID: 53}, ABSENT] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] to: [Sequence:{DescID: 53}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] to: [Table:{DescID: 54}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] - to: [Table:{DescID: 54}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [Table:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 55}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] - to: [View:{DescID: 55}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] to: [View:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] to: [View:{DescID: 56}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] - to: [View:{DescID: 55}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 57}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, ABSENT] - to: [View:{DescID: 56}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 57}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] - to: [View:{DescID: 56}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] to: [View:{DescID: 58}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, ABSENT] - to: [View:{DescID: 58}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, ABSENT] to: [View:{DescID: 61}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Schema:{DescID: 52}, ABSENT] to: [Sequence:{DescID: 53}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [Table:{DescID: 54}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [Type:{DescID: 59}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [Type:{DescID: 60}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 55}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 56}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 57}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 58}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 61}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Sequence:{DescID: 53}, ABSENT] to: [Namespace:{DescID: 53, Name: sq1}, ABSENT] -- from: [Sequence:{DescID: 53}, ABSENT] - to: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] -- from: [Table:{DescID: 54}, ABSENT] - to: [DefaultExpression:{DescID: 54, ColumnID: 3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [Table:{DescID: 54}, ABSENT] to: [Namespace:{DescID: 54, Name: t1}, ABSENT] -- from: [Table:{DescID: 54}, ABSENT] - to: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 55}, ABSENT] to: [Namespace:{DescID: 55, Name: v1}, ABSENT] -- from: [View:{DescID: 55}, ABSENT] - to: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] -- from: [View:{DescID: 55}, ABSENT] - to: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 56}, ABSENT] to: [Namespace:{DescID: 56, Name: v2}, ABSENT] -- from: [View:{DescID: 56}, ABSENT] - to: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, ABSENT] -- from: [View:{DescID: 56}, ABSENT] - to: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 57}, ABSENT] to: [Namespace:{DescID: 57, Name: v3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 58}, ABSENT] to: [Namespace:{DescID: 58, Name: v4}, ABSENT] -- from: [View:{DescID: 58}, ABSENT] - to: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 61}, ABSENT] to: [Namespace:{DescID: 61, Name: v5}, ABSENT] -- from: [View:{DescID: 61}, ABSENT] - to: [ViewDependsOnType:{DescID: 61, ReferencedDescID: 59}, ABSENT] -- from: [View:{DescID: 61}, ABSENT] - to: [ViewDependsOnType:{DescID: 61, ReferencedDescID: 60}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [ViewDependsOnType:{DescID: 61, ReferencedDescID: 59}, ABSENT] to: [Type:{DescID: 59}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [ViewDependsOnType:{DescID: 61, ReferencedDescID: 60}, ABSENT] to: [Type:{DescID: 60}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped ops DROP SCHEMA defaultdb.SC1 CASCADE @@ -219,16 +265,23 @@ Stage 1 Stage 2 (non-revertible) transitions: [Sequence:{DescID: 53}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 54}, TXN_DROPPED, DROP] -> DROPPED [Column:{DescID: 54, ColumnID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 54, ColumnID: 2}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 54, ColumnID: 3}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [DefaultExpression:{DescID: 54, ColumnID: 3}, PUBLIC, DROP] -> ABSENT [PrimaryIndex:{DescID: 54, IndexID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY + [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT [View:{DescID: 55}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [View:{DescID: 56}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT [View:{DescID: 57}, TXN_DROPPED, DROP] -> DROPPED [View:{DescID: 58}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, PUBLIC, DROP] -> ABSENT [View:{DescID: 61}, TXN_DROPPED, DROP] -> DROPPED [ViewDependsOnType:{DescID: 61, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT [ViewDependsOnType:{DescID: 61, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT @@ -240,6 +293,9 @@ Stage 2 (non-revertible) DescID: 53 *scop.MarkDescriptorAsDropped DescID: 54 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 54 + TableID: 53 *scop.RemoveColumnDefaultExpression ColumnID: 3 TableID: 54 @@ -247,14 +303,32 @@ Stage 2 (non-revertible) TableID: 54 *scop.MarkDescriptorAsDropped DescID: 55 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 55 + TableID: 54 *scop.MarkDescriptorAsDropped DescID: 56 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 56 + TableID: 55 *scop.MarkDescriptorAsDropped DescID: 57 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 55 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 56 *scop.MarkDescriptorAsDropped DescID: 58 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 58 + TableID: 56 *scop.MarkDescriptorAsDropped DescID: 61 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 61 + TableID: 58 *scop.MarkDescriptorAsDropped DescID: 59 *scop.RemoveTypeBackRef @@ -270,22 +344,15 @@ Stage 2 (non-revertible) Stage 3 (non-revertible) transitions: [Sequence:{DescID: 53}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 54}, DROPPED, DROP] -> ABSENT [Column:{DescID: 54, ColumnID: 1}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 54, ColumnID: 2}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 54, ColumnID: 3}, DELETE_ONLY, DROP] -> ABSENT [PrimaryIndex:{DescID: 54, IndexID: 1}, DELETE_ONLY, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT [View:{DescID: 55}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [View:{DescID: 56}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT [View:{DescID: 57}, DROPPED, DROP] -> ABSENT [View:{DescID: 58}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, PUBLIC, DROP] -> ABSENT [View:{DescID: 61}, DROPPED, DROP] -> ABSENT [Type:{DescID: 59}, DROPPED, DROP] -> ABSENT [Type:{DescID: 60}, DROPPED, DROP] -> ABSENT @@ -305,9 +372,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 53 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 54 - TableID: 53 *scop.LogEvent DescID: 54 Direction: 2 @@ -322,9 +386,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 55 - TableID: 54 *scop.LogEvent DescID: 55 Direction: 2 @@ -339,12 +400,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 56 - TableID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 55 *scop.LogEvent DescID: 56 Direction: 2 @@ -359,12 +414,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 56 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 56 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 58 - TableID: 56 *scop.LogEvent DescID: 57 Direction: 2 @@ -393,9 +442,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 58 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 61 - TableID: 58 *scop.LogEvent DescID: 61 Direction: 2 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_sequence b/pkg/sql/schemachanger/scplan/testdata/drop_sequence index ef795c98acce..5eefacf67c85 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scplan/testdata/drop_sequence @@ -68,14 +68,20 @@ Stage 1 Stage 2 (non-revertible) transitions: [Sequence:{DescID: 52}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT ops: *scop.MarkDescriptorAsDropped DescID: 52 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 53 + TableID: 52 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 54 + TableID: 52 Stage 3 (non-revertible) transitions: [Sequence:{DescID: 52}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT ops: *scop.LogEvent DescID: 52 @@ -91,12 +97,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 52 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 53 - TableID: 52 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 54 - TableID: 52 deps @@ -104,17 +104,17 @@ DROP SEQUENCE defaultdb.SQ1 CASCADE ---- - from: [Namespace:{DescID: 52, Name: sq1}, ABSENT] to: [Sequence:{DescID: 52}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] - to: [Sequence:{DescID: 52}, ABSENT] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] to: [Sequence:{DescID: 52}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, ABSENT] - to: [Sequence:{DescID: 52}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, ABSENT] to: [Sequence:{DescID: 52}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Sequence:{DescID: 52}, ABSENT] to: [Namespace:{DescID: 52, Name: sq1}, ABSENT] -- from: [Sequence:{DescID: 52}, ABSENT] - to: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] -- from: [Sequence:{DescID: 52}, ABSENT] - to: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index d7ed733b27fd..406f56980162 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -77,6 +77,7 @@ Stage 2 (non-revertible) [PrimaryIndex:{DescID: 55, IndexID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [ForeignKey:{DescID: 55, ReferencedDescID: 52, Name: fk_customers}, PUBLIC, DROP] -> ABSENT [ForeignKey:{DescID: 55, ReferencedDescID: 53, Name: fk_orders}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [View:{DescID: 57}, TXN_DROPPED, DROP] -> DROPPED ops: *scop.MarkDescriptorAsDropped @@ -86,18 +87,21 @@ Stage 2 (non-revertible) TableID: 55 *scop.UpdateRelationDeps TableID: 55 + *scop.RemoveSequenceOwnedBy + TableID: 56 *scop.RemoveColumnDefaultExpression ColumnID: 5 TableID: 55 *scop.UpdateRelationDeps TableID: 55 - *scop.RemoveSequenceOwnedBy - TableID: 56 *scop.RemoveRelationDependedOnBy DependedOnBy: 55 TableID: 54 *scop.MarkDescriptorAsDropped DescID: 57 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 55 Stage 3 (non-revertible) transitions: [Table:{DescID: 55}, DROPPED, DROP] -> ABSENT @@ -107,7 +111,6 @@ Stage 3 (non-revertible) [Column:{DescID: 55, ColumnID: 4}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 55, ColumnID: 5}, DELETE_ONLY, DROP] -> ABSENT [PrimaryIndex:{DescID: 55, IndexID: 1}, DELETE_ONLY, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [View:{DescID: 57}, DROPPED, DROP] -> ABSENT ops: *scop.LogEvent @@ -124,9 +127,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 55 *scop.LogEvent DescID: 57 Direction: 2 @@ -147,35 +147,45 @@ DROP TABLE defaultdb.shipments CASCADE; ---- - from: [DefaultExpression:{DescID: 55, ColumnID: 1}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [DefaultExpression:{DescID: 55, ColumnID: 5}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [ForeignKey:{DescID: 55, ReferencedDescID: 52, Name: fk_customers}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [ForeignKey:{DescID: 55, ReferencedDescID: 53, Name: fk_orders}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Namespace:{DescID: 55, Name: shipments}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 57, Name: v1}, ABSENT] to: [View:{DescID: 57}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [Table:{DescID: 55}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] - to: [Table:{DescID: 55}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 57}, DROPPED] -- from: [Table:{DescID: 55}, ABSENT] - to: [DefaultExpression:{DescID: 55, ColumnID: 1}, ABSENT] -- from: [Table:{DescID: 55}, ABSENT] - to: [DefaultExpression:{DescID: 55, ColumnID: 5}, ABSENT] -- from: [Table:{DescID: 55}, ABSENT] - to: [ForeignKey:{DescID: 55, ReferencedDescID: 52, Name: fk_customers}, ABSENT] -- from: [Table:{DescID: 55}, ABSENT] - to: [ForeignKey:{DescID: 55, ReferencedDescID: 53, Name: fk_orders}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Table:{DescID: 55}, ABSENT] to: [Namespace:{DescID: 55, Name: shipments}, ABSENT] -- from: [Table:{DescID: 55}, ABSENT] - to: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 57}, ABSENT] to: [Namespace:{DescID: 57, Name: v1}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_view b/pkg/sql/schemachanger/scplan/testdata/drop_view index 7763706efd3c..8ff043a25b52 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_view +++ b/pkg/sql/schemachanger/scplan/testdata/drop_view @@ -55,10 +55,16 @@ DROP VIEW defaultdb.v1 ---- - from: [Namespace:{DescID: 53, Name: v1}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [View:{DescID: 53}, ABSENT] to: [Namespace:{DescID: 53, Name: v1}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained create-view CREATE VIEW defaultdb.v2 AS (SELECT name AS n1, name AS n2 FROM v1) @@ -123,9 +129,14 @@ Stage 2 (non-revertible) transitions: [View:{DescID: 53}, TXN_DROPPED, DROP] -> DROPPED [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT [View:{DescID: 54}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT [View:{DescID: 55}, TXN_DROPPED, DROP] -> DROPPED [View:{DescID: 56}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT [View:{DescID: 59}, TXN_DROPPED, DROP] -> DROPPED [ViewDependsOnType:{DescID: 59, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [ViewDependsOnType:{DescID: 59, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT @@ -137,12 +148,27 @@ Stage 2 (non-revertible) TableID: 52 *scop.MarkDescriptorAsDropped DescID: 54 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 54 + TableID: 53 *scop.MarkDescriptorAsDropped DescID: 55 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 55 + TableID: 53 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 55 + TableID: 54 *scop.MarkDescriptorAsDropped DescID: 56 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 56 + TableID: 54 *scop.MarkDescriptorAsDropped DescID: 59 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 59 + TableID: 56 *scop.RemoveTypeBackRef DescID: 59 TypeID: 57 @@ -152,14 +178,9 @@ Stage 2 (non-revertible) Stage 3 (non-revertible) transitions: [View:{DescID: 53}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT [View:{DescID: 54}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT [View:{DescID: 55}, DROPPED, DROP] -> ABSENT [View:{DescID: 56}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT [View:{DescID: 59}, DROPPED, DROP] -> ABSENT ops: *scop.LogEvent @@ -176,12 +197,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 53 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 54 - TableID: 53 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 55 - TableID: 53 *scop.LogEvent DescID: 54 Direction: 2 @@ -196,12 +211,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 55 - TableID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 56 - TableID: 54 *scop.LogEvent DescID: 55 Direction: 2 @@ -230,9 +239,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 56 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 59 - TableID: 56 *scop.LogEvent DescID: 59 Direction: 2 @@ -253,67 +259,85 @@ DROP VIEW defaultdb.v1 CASCADE ---- - from: [Namespace:{DescID: 53, Name: v1}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 54, Name: v2}, ABSENT] to: [View:{DescID: 54}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 55, Name: v3}, ABSENT] to: [View:{DescID: 55}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 56, Name: v4}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 59, Name: v5}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] to: [View:{DescID: 53}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] - to: [View:{DescID: 53}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] to: [View:{DescID: 54}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, ABSENT] - to: [View:{DescID: 53}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 55}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] - to: [View:{DescID: 54}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 55}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, ABSENT] - to: [View:{DescID: 54}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, ABSENT] to: [View:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, ABSENT] to: [View:{DescID: 56}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, ABSENT] - to: [View:{DescID: 56}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [View:{DescID: 53}, ABSENT] to: [Namespace:{DescID: 53, Name: v1}, ABSENT] -- from: [View:{DescID: 53}, ABSENT] - to: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] -- from: [View:{DescID: 53}, ABSENT] - to: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 54}, ABSENT] to: [Namespace:{DescID: 54, Name: v2}, ABSENT] -- from: [View:{DescID: 54}, ABSENT] - to: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] -- from: [View:{DescID: 54}, ABSENT] - to: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 55}, ABSENT] to: [Namespace:{DescID: 55, Name: v3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 56}, ABSENT] to: [Namespace:{DescID: 56, Name: v4}, ABSENT] -- from: [View:{DescID: 56}, ABSENT] - to: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 59}, ABSENT] to: [Namespace:{DescID: 59, Name: v5}, ABSENT] -- from: [View:{DescID: 59}, ABSENT] - to: [ViewDependsOnType:{DescID: 59, ReferencedDescID: 57}, ABSENT] -- from: [View:{DescID: 59}, ABSENT] - to: [ViewDependsOnType:{DescID: 59, ReferencedDescID: 58}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained diff --git a/pkg/sql/schemachanger/testdata/drop b/pkg/sql/schemachanger/testdata/drop index 8c9d19a0519a..ce99df34fdfa 100644 --- a/pkg/sql/schemachanger/testdata/drop +++ b/pkg/sql/schemachanger/testdata/drop @@ -213,7 +213,7 @@ delete object namespace entry {58 59 v2} -> 65 delete object namespace entry {58 59 v3} -> 66 delete object namespace entry {58 59 v4} -> 67 delete object namespace entry {58 59 v5} -> 70 -## stage 2 in PreCommitPhase: 19 MutationType ops +## stage 2 in PreCommitPhase: 27 MutationType ops upsert descriptor #58 ... sc1: @@ -229,6 +229,16 @@ upsert descriptor #59 + state: DROP + version: "2" upsert descriptor #60 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - byId: true + - columnIds: + - - 3 + - id: 63 + families: + - columnIds: ... sequenceOwner: {} start: "1" @@ -237,6 +247,16 @@ upsert descriptor #60 - version: "2" + version: "3" upsert descriptor #61 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - byId: true + - columnIds: + - - 3 + - id: 62 + families: + - columnIds: ... sequenceOwner: {} start: "1" @@ -301,6 +321,20 @@ upsert descriptor #63 - version: "1" + version: "2" upsert descriptor #64 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - id: 65 + - - columnIds: + - - 1 + - id: 66 + - dependsOn: + - - 62 + formatVersion: 3 + id: 64 ... replacementOf: time: {} @@ -310,6 +344,22 @@ upsert descriptor #64 + version: "4" viewQuery: (SELECT name FROM db1.sc1.t1) upsert descriptor #65 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - - 2 + - id: 66 + - - columnIds: + - - 1 + - - 2 + - id: 67 + - dependsOn: + - - 64 + formatVersion: 3 + id: 65 ... replacementOf: time: {} @@ -319,6 +369,14 @@ upsert descriptor #65 + version: "4" viewQuery: (SELECT name AS n1, name AS n2 FROM db1.sc1.v1) upsert descriptor #66 + ... + createAsOfTime: + wallTime: "1" + - dependsOn: + - - 64 + - - 65 + formatVersion: 3 + id: 66 ... replacementOf: time: {} @@ -328,6 +386,18 @@ upsert descriptor #66 + version: "2" viewQuery: (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2) upsert descriptor #67 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - - 2 + - id: 70 + - dependsOn: + - - 65 + formatVersion: 3 + id: 67 ... replacementOf: time: {} @@ -357,6 +427,13 @@ upsert descriptor #69 + state: DROP + version: "3" upsert descriptor #70 + ... + createAsOfTime: + wallTime: "1" + - dependsOn: + - - 67 + dependsOnTypes: + - 68 ... replacementOf: time: {} @@ -446,147 +523,11 @@ upsert descriptor #70 commit transaction #1 # begin PostCommitPhase begin transaction #2 -## stage 1 in PostCommitPhase: 30 MutationType ops +## stage 1 in PostCommitPhase: 22 MutationType ops create job #2: "GC for Dropping descriptors 60 63 61 62 64 65 66 67 70 " descriptor IDs: [60 63 61 62 64 65 66 67 70] delete database namespace entry {0 0 db1} -> 58 delete schema namespace entry {58 0 sc1} -> 59 -upsert descriptor #60 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - byId: true - - columnIds: - - - 3 - - id: 63 - families: - - columnIds: - ... - state: DROP - unexposedParentSchemaId: 29 - - version: "3" - + version: "4" -upsert descriptor #61 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - byId: true - - columnIds: - - - 3 - - id: 62 - families: - - columnIds: - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "3" - + version: "4" -upsert descriptor #62 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "3" - + version: "4" -upsert descriptor #63 - ... - state: DROP - unexposedParentSchemaId: 29 - - version: "2" - + version: "3" -upsert descriptor #64 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - id: 65 - - - columnIds: - - - 1 - - id: 66 - - dependsOn: - - - 62 - formatVersion: 3 - id: 64 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "4" - + version: "5" - viewQuery: (SELECT name FROM db1.sc1.t1) -upsert descriptor #65 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - - 2 - - id: 66 - - - columnIds: - - - 1 - - - 2 - - id: 67 - - dependsOn: - - - 64 - formatVersion: 3 - id: 65 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "4" - + version: "5" - viewQuery: (SELECT name AS n1, name AS n2 FROM db1.sc1.v1) -upsert descriptor #66 - ... - createAsOfTime: - wallTime: "1" - - dependsOn: - - - 64 - - - 65 - formatVersion: 3 - id: 66 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "2" - + version: "3" - viewQuery: (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2) -upsert descriptor #67 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - - 2 - - id: 70 - - dependsOn: - - - 65 - formatVersion: 3 - id: 67 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "3" - + version: "4" - viewQuery: (SELECT n2, n1 FROM db1.sc1.v2) -upsert descriptor #70 - ... - createAsOfTime: - wallTime: "1" - - dependsOn: - - - 67 - dependsOnTypes: - - 68 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "2" - + version: "3" - viewQuery: (SELECT 'a':::sc1.typ::STRING AS k, n2, n1 FROM db1.sc1.v4) update progress of schema change job #1 commit transaction #2 begin transaction #3