Skip to content

Commit

Permalink
Merge #120794
Browse files Browse the repository at this point in the history
120794: scplan: Fix deprules for dropping computed columns r=rimadeodhar a=rimadeodhar

This PR fixes the new schema changer deprules for
dropping virtual computed columns which are also
used for hash and expression indexes.
Currently, the optimizer allows for virtual, computed
columns to be evaluated even when under mutation.
However, this causes concurrent DML issues when the
schemachanger job is running as the column that the
virtual computed column depends on moves into WRITE_ONLY
stage prior to the computed column being dropped.
As a result, the optimizer is unable to access the column
for evaluating the compute expression.
This PR updates the dep rules to ensure the virtual,
computed column is dropped before the dependent column
moves to WRITE_ONLY ensuring that the compute expression
can be enforced correctly for concurrent DML during all
stages of the schema change.

Epic: none
Fixes: #111608
Fixes: #111619
Release note: None

------------------------------------------------------------------
**Note for reviewers:** This PR is stacked on top of #120792.

Co-authored-by: rimadeodhar <rima@cockroachlabs.com>
  • Loading branch information
craig[bot] and rimadeodhar committed May 2, 2024
2 parents e88bd17 + 47aa2d5 commit 00dd860
Show file tree
Hide file tree
Showing 50 changed files with 1,263 additions and 1,909 deletions.
2 changes: 0 additions & 2 deletions pkg/sql/schemachanger/dml_injection_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -345,7 +345,6 @@ func TestAlterTableDMLInjection(t *testing.T) {
"CREATE INDEX idx ON tbl (i) USING HASH",
},
schemaChange: "ALTER TABLE tbl DROP COLUMN i CASCADE",
skipIssue: 111619,
},
{
desc: "drop column with composite index + fk",
Expand Down Expand Up @@ -407,7 +406,6 @@ func TestAlterTableDMLInjection(t *testing.T) {
"CREATE INDEX idx ON tbl ((i + 1))",
},
schemaChange: "ALTER TABLE tbl DROP COLUMN i CASCADE",
skipIssue: 111608,
},
{
desc: "create materialized view from index",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func init() {
func init() {
registerDepRule(
"ensure columns are in increasing order",
scgraph.SameStagePrecedence,
scgraph.Precedence,
"later-column", "earlier-column",
func(from, to NodeVars) rel.Clauses {
status := rel.Var("status")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,51 @@ func init() {
},
)

// This rule ensures that a column is dropped only after any virtual, computed column dependent
// on the column is dropped i.e. if B is a virtual, computed column using column A
// in its compute expression, this rule ensures that the compute expression of B is dropped before
// A is dropped. The rules above ensure that the column B is dropped before the expression is dropped
// so this rule also implicitly implies that column B is dropped before column A.
// This is relevant for expression and hash indexes which create an internal, virtual column
// which computes the hash/expression key for the index.
//
// N.B. This rules has been intentionally made very specific to only virtual, computed columns as opposed
// to all computed columns. This is due an edge case within the optimizer which actually allows
// the compute expressions of virtual computed columns to be evaluated during an active schema change.
// Without this rule, the optimizer is unable to read the dependent column as the dependent column
// moves to the WRITE_ONLY stage before the computed column is fully dropped. As of now, we don't
// need to apply to all computed columns as the optimizer doesn't evaluate their expressions while
// dropping them so the above rule where the column type is dropped before the column is sufficient
// to enforce the dependency.
registerDepRuleForDrop(
"Virtual computed column expression is dropped before the column it depends on",
scgraph.Precedence,
"virtual-column-expr", "column",
scpb.Status_ABSENT, scpb.Status_WRITE_ONLY,
func(from, to NodeVars) rel.Clauses {
return rel.Clauses{
from.Type((*scpb.ColumnType)(nil)),
to.Type((*scpb.Column)(nil)),
JoinOnDescID(from, to, "table-id"),
FilterElements("computedColumnTypeReferencesColumn", from, to,
func(colType *scpb.ColumnType, column *scpb.Column) bool {
if !colType.IsVirtual {
return false
}
if colType.ComputeExpr == nil {
return false
}
for _, refColumns := range colType.ComputeExpr.ReferencedColumnIDs {
if refColumns == column.ColumnID {
return true
}
}
return false
}),
}
},
)

// Column constraint disappears in the same stage as the column
// becomes WRITE_ONLY.
//
Expand All @@ -138,6 +183,31 @@ func init() {
}
},
)

// This rule enforces that a new primary index moves to the public stage only after all columns stored
// within the old primary index move to WRITE_ONLY. Without this, the new primary index is at risk of not
// storing all public columns within the table (as the column being dropped is still considered public
// before it moves to WRITE_ONLY but the new primary index does not contain it since the schema changer
// knows it is transitioning to a target status of ABSENT).
registerDepRule(
"New primary index should go public only after columns being dropped move to WRITE_ONLY",
scgraph.Precedence,
"column", "new-primary-index",
func(from, to NodeVars) rel.Clauses {
ic := MkNodeVars("index-column")
relationID, columnID, indexID := rel.Var("table-id"), rel.Var("column-id"), rel.Var("index-id")
return rel.Clauses{
from.Type((*scpb.Column)(nil)),
to.Type((*scpb.PrimaryIndex)(nil)),
ColumnInSourcePrimaryIndex(ic, to, relationID, columnID, indexID),
JoinOnColumnID(ic, from, relationID, columnID),
from.TargetStatus(scpb.ToAbsent),
from.CurrentStatus(scpb.Status_WRITE_ONLY),
to.TargetStatus(scpb.ToPublic),
to.CurrentStatus(scpb.Status_PUBLIC),
}
},
)
}

// Special rules partial predicate expressions, which ensure that any columns
Expand Down Expand Up @@ -170,6 +240,7 @@ func init() {
}
},
)

registerDepRuleForDrop(
"secondary index partial no longer public before referenced column",
scgraph.Precedence,
Expand Down
150 changes: 148 additions & 2 deletions pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules
Original file line number Diff line number Diff line change
Expand Up @@ -628,6 +628,21 @@ deprules
- descriptorIsDataNotBeingAdded-24.1($descID)
- joinTargetNode($prev, $prev-Target, $prev-Node)
- joinTargetNode($next, $next-Target, $next-Node)
- name: New primary index should go public only after columns being dropped move to WRITE_ONLY
from: column-Node
kind: Precedence
to: new-primary-index-Node
query:
- $column[Type] = '*scpb.Column'
- $new-primary-index[Type] = '*scpb.PrimaryIndex'
- ColumnInSourcePrimaryIndex($index-column, $new-primary-index, $table-id, $column-id, $index-id)
- joinOnColumnID($index-column, $column, $table-id, $column-id)
- $column-Target[TargetStatus] = ABSENT
- $column-Node[CurrentStatus] = WRITE_ONLY
- $new-primary-index-Target[TargetStatus] = PUBLIC
- $new-primary-index-Node[CurrentStatus] = PUBLIC
- joinTargetNode($column, $column-Target, $column-Node)
- joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node)
- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY'
from: prev-Node
kind: PreviousTransactionPrecedence
Expand Down Expand Up @@ -2107,6 +2122,64 @@ deprules
- descriptorIsDataNotBeingAdded-24.1($descID)
- joinTargetNode($prev, $prev-Target, $prev-Node)
- joinTargetNode($next, $next-Target, $next-Node)
- name: Virtual computed column expression is dropped before the column it depends on
from: virtual-column-expr-Node
kind: Precedence
to: column-Node
query:
- $virtual-column-expr[Type] = '*scpb.ColumnType'
- $column[Type] = '*scpb.Column'
- joinOnDescID($virtual-column-expr, $column, $table-id)
- computedColumnTypeReferencesColumn(*scpb.ColumnType, *scpb.Column)($virtual-column-expr, $column)
- toAbsent($virtual-column-expr-Target, $column-Target)
- $virtual-column-expr-Node[CurrentStatus] = ABSENT
- $column-Node[CurrentStatus] = WRITE_ONLY
- joinTargetNode($virtual-column-expr, $virtual-column-expr-Target, $virtual-column-expr-Node)
- joinTargetNode($column, $column-Target, $column-Node)
- name: Virtual computed column expression is dropped before the column it depends on
from: virtual-column-expr-Node
kind: Precedence
to: column-Node
query:
- $virtual-column-expr[Type] = '*scpb.ColumnType'
- $column[Type] = '*scpb.Column'
- joinOnDescID($virtual-column-expr, $column, $table-id)
- computedColumnTypeReferencesColumn(*scpb.ColumnType, *scpb.Column)($virtual-column-expr, $column)
- transient($virtual-column-expr-Target, $column-Target)
- $virtual-column-expr-Node[CurrentStatus] = TRANSIENT_ABSENT
- $column-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY
- joinTargetNode($virtual-column-expr, $virtual-column-expr-Target, $virtual-column-expr-Node)
- joinTargetNode($column, $column-Target, $column-Node)
- name: Virtual computed column expression is dropped before the column it depends on
from: virtual-column-expr-Node
kind: Precedence
to: column-Node
query:
- $virtual-column-expr[Type] = '*scpb.ColumnType'
- $column[Type] = '*scpb.Column'
- joinOnDescID($virtual-column-expr, $column, $table-id)
- computedColumnTypeReferencesColumn(*scpb.ColumnType, *scpb.Column)($virtual-column-expr, $column)
- $virtual-column-expr-Target[TargetStatus] = TRANSIENT_ABSENT
- $virtual-column-expr-Node[CurrentStatus] = TRANSIENT_ABSENT
- $column-Target[TargetStatus] = ABSENT
- $column-Node[CurrentStatus] = WRITE_ONLY
- joinTargetNode($virtual-column-expr, $virtual-column-expr-Target, $virtual-column-expr-Node)
- joinTargetNode($column, $column-Target, $column-Node)
- name: Virtual computed column expression is dropped before the column it depends on
from: virtual-column-expr-Node
kind: Precedence
to: column-Node
query:
- $virtual-column-expr[Type] = '*scpb.ColumnType'
- $column[Type] = '*scpb.Column'
- joinOnDescID($virtual-column-expr, $column, $table-id)
- computedColumnTypeReferencesColumn(*scpb.ColumnType, *scpb.Column)($virtual-column-expr, $column)
- $virtual-column-expr-Target[TargetStatus] = ABSENT
- $virtual-column-expr-Node[CurrentStatus] = ABSENT
- $column-Target[TargetStatus] = TRANSIENT_ABSENT
- $column-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY
- joinTargetNode($virtual-column-expr, $virtual-column-expr-Target, $virtual-column-expr-Node)
- joinTargetNode($column, $column-Target, $column-Node)
- name: all adding indexes reached BACKFILL_ONLY before any of their columns disappear
from: index-Node
kind: Precedence
Expand Down Expand Up @@ -2959,7 +3032,7 @@ deprules
- joinTargetNode($data, $data-Target, $data-Node)
- name: ensure columns are in increasing order
from: later-column-Node
kind: SameStagePrecedence
kind: Precedence
to: earlier-column-Node
query:
- $later-column[Type] = '*scpb.Column'
Expand Down Expand Up @@ -4711,6 +4784,21 @@ deprules
- descriptorIsDataNotBeingAdded-24.1($descID)
- joinTargetNode($prev, $prev-Target, $prev-Node)
- joinTargetNode($next, $next-Target, $next-Node)
- name: New primary index should go public only after columns being dropped move to WRITE_ONLY
from: column-Node
kind: Precedence
to: new-primary-index-Node
query:
- $column[Type] = '*scpb.Column'
- $new-primary-index[Type] = '*scpb.PrimaryIndex'
- ColumnInSourcePrimaryIndex($index-column, $new-primary-index, $table-id, $column-id, $index-id)
- joinOnColumnID($index-column, $column, $table-id, $column-id)
- $column-Target[TargetStatus] = ABSENT
- $column-Node[CurrentStatus] = WRITE_ONLY
- $new-primary-index-Target[TargetStatus] = PUBLIC
- $new-primary-index-Node[CurrentStatus] = PUBLIC
- joinTargetNode($column, $column-Target, $column-Node)
- joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node)
- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY'
from: prev-Node
kind: PreviousTransactionPrecedence
Expand Down Expand Up @@ -6190,6 +6278,64 @@ deprules
- descriptorIsDataNotBeingAdded-24.1($descID)
- joinTargetNode($prev, $prev-Target, $prev-Node)
- joinTargetNode($next, $next-Target, $next-Node)
- name: Virtual computed column expression is dropped before the column it depends on
from: virtual-column-expr-Node
kind: Precedence
to: column-Node
query:
- $virtual-column-expr[Type] = '*scpb.ColumnType'
- $column[Type] = '*scpb.Column'
- joinOnDescID($virtual-column-expr, $column, $table-id)
- computedColumnTypeReferencesColumn(*scpb.ColumnType, *scpb.Column)($virtual-column-expr, $column)
- toAbsent($virtual-column-expr-Target, $column-Target)
- $virtual-column-expr-Node[CurrentStatus] = ABSENT
- $column-Node[CurrentStatus] = WRITE_ONLY
- joinTargetNode($virtual-column-expr, $virtual-column-expr-Target, $virtual-column-expr-Node)
- joinTargetNode($column, $column-Target, $column-Node)
- name: Virtual computed column expression is dropped before the column it depends on
from: virtual-column-expr-Node
kind: Precedence
to: column-Node
query:
- $virtual-column-expr[Type] = '*scpb.ColumnType'
- $column[Type] = '*scpb.Column'
- joinOnDescID($virtual-column-expr, $column, $table-id)
- computedColumnTypeReferencesColumn(*scpb.ColumnType, *scpb.Column)($virtual-column-expr, $column)
- transient($virtual-column-expr-Target, $column-Target)
- $virtual-column-expr-Node[CurrentStatus] = TRANSIENT_ABSENT
- $column-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY
- joinTargetNode($virtual-column-expr, $virtual-column-expr-Target, $virtual-column-expr-Node)
- joinTargetNode($column, $column-Target, $column-Node)
- name: Virtual computed column expression is dropped before the column it depends on
from: virtual-column-expr-Node
kind: Precedence
to: column-Node
query:
- $virtual-column-expr[Type] = '*scpb.ColumnType'
- $column[Type] = '*scpb.Column'
- joinOnDescID($virtual-column-expr, $column, $table-id)
- computedColumnTypeReferencesColumn(*scpb.ColumnType, *scpb.Column)($virtual-column-expr, $column)
- $virtual-column-expr-Target[TargetStatus] = TRANSIENT_ABSENT
- $virtual-column-expr-Node[CurrentStatus] = TRANSIENT_ABSENT
- $column-Target[TargetStatus] = ABSENT
- $column-Node[CurrentStatus] = WRITE_ONLY
- joinTargetNode($virtual-column-expr, $virtual-column-expr-Target, $virtual-column-expr-Node)
- joinTargetNode($column, $column-Target, $column-Node)
- name: Virtual computed column expression is dropped before the column it depends on
from: virtual-column-expr-Node
kind: Precedence
to: column-Node
query:
- $virtual-column-expr[Type] = '*scpb.ColumnType'
- $column[Type] = '*scpb.Column'
- joinOnDescID($virtual-column-expr, $column, $table-id)
- computedColumnTypeReferencesColumn(*scpb.ColumnType, *scpb.Column)($virtual-column-expr, $column)
- $virtual-column-expr-Target[TargetStatus] = ABSENT
- $virtual-column-expr-Node[CurrentStatus] = ABSENT
- $column-Target[TargetStatus] = TRANSIENT_ABSENT
- $column-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY
- joinTargetNode($virtual-column-expr, $virtual-column-expr-Target, $virtual-column-expr-Node)
- joinTargetNode($column, $column-Target, $column-Node)
- name: all adding indexes reached BACKFILL_ONLY before any of their columns disappear
from: index-Node
kind: Precedence
Expand Down Expand Up @@ -7042,7 +7188,7 @@ deprules
- joinTargetNode($data, $data-Target, $data-Node)
- name: ensure columns are in increasing order
from: later-column-Node
kind: SameStagePrecedence
kind: Precedence
to: earlier-column-Node
query:
- $later-column[Type] = '*scpb.Column'
Expand Down
21 changes: 21 additions & 0 deletions pkg/sql/schemachanger/scplan/internal/rules/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,12 @@ func ColumnInSwappedInPrimaryIndex(
return columnInSwappedInPrimaryIndexUntyped(indexColumn.El, index.El, relationIDVar, columnIDVar, indexIDVar)
}

func ColumnInSourcePrimaryIndex(
indexColumn, index NodeVars, relationIDVar, columnIDVar, indexIDVar rel.Var,
) rel.Clause {
return columnInSourcePrimaryIndex(indexColumn.El, index.El, relationIDVar, columnIDVar, indexIDVar)
}

// IsPotentialSecondaryIndexSwap determines if a secondary index recreate is
// occurring because of a primary key alter.
func IsPotentialSecondaryIndexSwap(indexIdVar rel.Var, tableIDVar rel.Var) rel.Clauses {
Expand Down Expand Up @@ -292,6 +298,21 @@ var (
}
})

columnInSourcePrimaryIndex = screl.Schema.Def5(
"ColumnInSourcePrimaryIndex",
"index-column", "index", "table-id", "column-id", "index-id", func(
indexColumn, index, tableID, columnID, indexID rel.Var,
) rel.Clauses {
return rel.Clauses{
indexColumn.Type((*scpb.IndexColumn)(nil)),
indexColumn.AttrEqVar(screl.DescID, tableID),
indexColumn.AttrEqVar(screl.ColumnID, columnID),
indexColumn.AttrEqVar(screl.IndexID, indexID),
index.AttrEqVar(screl.SourceIndexID, indexID),
JoinOnDescIDUntyped(index, indexColumn, tableID),
}
})

// IsNotPotentialSecondaryIndexSwap determines if no secondary index recreation
// is happening because of a primary key alter.
IsNotPotentialSecondaryIndexSwap = screl.Schema.DefNotJoin2("no secondary index swap is on going",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -635,6 +635,10 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k);
to: [IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT]
kind: Precedence
rule: column no longer public before dependents
- from: [Column:{DescID: 104, ColumnID: 3}, WRITE_ONLY]
to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, PUBLIC]
kind: Precedence
rule: New primary index should go public only after columns being dropped move to WRITE_ONLY
- from: [ColumnDefaultExpression:{DescID: 104, ColumnID: 3, Expr: unique_rowid()}, ABSENT]
to: [Column:{DescID: 104, ColumnID: 3}, ABSENT]
kind: Precedence
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column
Original file line number Diff line number Diff line change
Expand Up @@ -788,6 +788,10 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE;
to: [IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 2}, ABSENT]
kind: Precedence
rule: column no longer public before dependents
- from: [Column:{DescID: 107, ColumnID: 2}, WRITE_ONLY]
to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC]
kind: Precedence
rule: New primary index should go public only after columns being dropped move to WRITE_ONLY
- from: [Column:{DescID: 108, ColumnID: 1}, ABSENT]
to: [View:{DescID: 108}, ABSENT]
kind: Precedence
Expand Down Expand Up @@ -2146,6 +2150,10 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE;
to: [IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 2}, ABSENT]
kind: Precedence
rule: column no longer public before dependents
- from: [Column:{DescID: 107, ColumnID: 3}, WRITE_ONLY]
to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC]
kind: Precedence
rule: New primary index should go public only after columns being dropped move to WRITE_ONLY
- from: [Column:{DescID: 108, ColumnID: 1}, ABSENT]
to: [View:{DescID: 108}, ABSENT]
kind: Precedence
Expand Down Expand Up @@ -3157,6 +3165,10 @@ ALTER TABLE defaultdb.foo DROP COLUMN udfcol;
to: [IndexColumn:{DescID: 107, ColumnID: 4, IndexID: 1}, ABSENT]
kind: Precedence
rule: column no longer public before dependents
- from: [Column:{DescID: 107, ColumnID: 4}, WRITE_ONLY]
to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC]
kind: Precedence
rule: New primary index should go public only after columns being dropped move to WRITE_ONLY
- from: [ColumnDefaultExpression:{DescID: 107, ColumnID: 4, Expr: [FUNCTION 100109](), ReferencedFunctionIDs: [109]}, ABSENT]
to: [Column:{DescID: 107, ColumnID: 4}, ABSENT]
kind: Precedence
Expand Down
Loading

0 comments on commit 00dd860

Please sign in to comment.