Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
57149: opt: don't hold on to evalCtx from detached Memo r=RaduBerinde a=RaduBerinde

This change adds more "cleanup" code when detaching a Memo (a detached
memo is stored in the query cache and is reused later in a "read-only"
fashion). In particular, we clear the EvalContext in
logicalPropsBuilder which can lead to inadvertently holding on to a
lot of memory.

Fixes #57059.

Release note: None

57153: optbuilder: fix ambiguous column references for FK cascades r=RaduBerinde a=mgartner

This commit fixes an issue in optbuilder that caused "ambiguous column
reference" errors. This error would be produced during cascading updates
if a child table's reference column name was equal to the parent column
name concatenated with `_new`, and the child table had a check
constraint, computed column, or partial index predicate that referenced
the column.

For example, the following `UPDATE` statement would produce an error.
The expected behavior is a successful `UPDATE`. Notice that `p_new` of
the child table references `p` of the parent table.

    CREATE TABLE parent (p INT PRIMARY KEY)

    CREATE TABLE child (
      c INT PRIMARY KEY,
      p_new INT REFERENCES parent(p) ON UPDATE CASCADE,
      CHECK (p_new > 0)
    )

    UPDATE parent SET p = p * 10 WHERE p > 1

This issue was the result of incorrect scoping while building foreign
key cascading update expressions. A column with the same name and column
ID was added to the update expression's input scope. Because the
`mutationBuilder.disambiguateColumns` function is unable to disambiguate
columns with the same name and column ID, building any expression that
referenced the duplicated column would result in an error.

This commit fixes the issue by no longer duplicating columns in the
update expression's input scope. `mutationBuilder.addUpdateCols` now
detects the special case when the update expression is a `*scopeColumn`
and avoids duplicating it in the generated projection scope.

Fixes #57148

Release note (bug fix): A bug has been fix that caused an "ambiguous
column reference" error during foreign key cascading updates. This error
was incorrectly produced when the child table's reference column name
was equal to the concatenation of the parent's reference column name and
"_new", and when the child table had a CHECK constraint, computed
column, or partial index predicate expression that referenced the
column. This bug was introduce in version 20.2.

57242: kvserver: avoid serving an unsafe string to log.Fatalf r=irfansharif a=knz

A linter change in #57134 made me discover this bug.

Release note: None

57246: rowenc: de-flake TestEncodeContainingArrayInvertedIndexSpans r=rytaft a=mgartner

`TestEncodeContainingArrayInvertedIndexSpans` was failing sporadically
because of randomized test cases that were incorrectly determining the
expected value for the `unique` return value from
`EncodeContainingInvertedIndexSpans`. The test was using the
`reflect.DeepEqual` function to check for `Datum` equality, which does
not return true in all cases where `Datum.Compare` returns `0`.

Fixes #57237

Release note: None

Co-authored-by: Radu Berinde <radu@cockroachlabs.com>
Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
  • Loading branch information
4 people committed Nov 30, 2020
5 parents 75d99fa + a94f1d9 + cefe554 + 1fe7794 + 32b2ea4 commit d5222dc
Show file tree
Hide file tree
Showing 7 changed files with 138 additions and 19 deletions.
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -262,7 +262,7 @@ func (r *Replica) computeChecksumPostApply(ctx context.Context, cc kvserverpb.Co
_ = r.store.engine.MkdirAll(auxDir)
path := base.PreventedStartupFile(auxDir)

preventStartupMsg := fmt.Sprintf(`ATTENTION:
const attentionFmt = `ATTENTION:
this node is terminating because a replica inconsistency was detected between %s
and its other replicas. Please check your cluster-wide log files for more
Expand All @@ -274,8 +274,8 @@ A checkpoints directory to aid (expert) debugging should be present in:
A file preventing this node from restarting was placed at:
%s
`, r, auxDir, path)

`
preventStartupMsg := fmt.Sprintf(attentionFmt, r, auxDir, path)
if err := fs.WriteFile(r.store.engine, path, []byte(preventStartupMsg)); err != nil {
log.Warningf(ctx, "%v", err)
}
Expand All @@ -284,7 +284,7 @@ A file preventing this node from restarting was placed at:
p(*r.store.Ident)
} else {
time.Sleep(10 * time.Second)
log.Fatalf(r.AnnotateCtx(context.Background()), preventStartupMsg)
log.Fatalf(r.AnnotateCtx(context.Background()), attentionFmt, r, auxDir, path)
}
}

Expand Down
34 changes: 23 additions & 11 deletions pkg/sql/opt/memo/memo.go
Original file line number Diff line number Diff line change
Expand Up @@ -383,17 +383,29 @@ func (m *Memo) NextWithID() opt.WithID {
return m.curWithID
}

// ClearColStats clears all column statistics from every relational expression
// in the memo. This is used to free up the potentially large amount of memory
// used by histograms.
func (m *Memo) ClearColStats(parent opt.Expr) {
for i, n := 0, parent.ChildCount(); i < n; i++ {
child := parent.Child(i)
m.ClearColStats(child)
}
// Detach is used when we detach a memo that is to be reused later (either for
// execbuilding or with AssignPlaceholders). New expressions should no longer be
// constructed in this memo.
func (m *Memo) Detach() {
m.interner = interner{}
// It is important to not hold on to the EvalCtx in the logicalPropsBuilder
// (#57059).
m.logPropsBuilder = logicalPropsBuilder{}

// Clear all column statistics from every relational expression in the memo.
// This is used to free up the potentially large amount of memory used by
// histograms.
var clearColStats func(parent opt.Expr)
clearColStats = func(parent opt.Expr) {
for i, n := 0, parent.ChildCount(); i < n; i++ {
child := parent.Child(i)
clearColStats(child)
}

switch t := parent.(type) {
case RelExpr:
t.Relational().Stats.ColStats = props.ColStatsMap{}
switch t := parent.(type) {
case RelExpr:
t.Relational().Stats.ColStats = props.ColStatsMap{}
}
}
clearColStats(m.RootExpr())
}
6 changes: 3 additions & 3 deletions pkg/sql/opt/norm/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,11 +124,11 @@ func (f *Factory) FoldingControl() *FoldingControl {
// placeholders are assigned. If there are no placeholders, there is no need
// for column statistics, since the memo is already fully optimized.
func (f *Factory) DetachMemo() *memo.Memo {
f.mem.ClearColStats(f.mem.RootExpr())
detach := f.mem
m := f.mem
f.mem = nil
m.Detach()
f.Init(f.evalCtx, nil /* catalog */)
return detach
return m
}

// DisableOptimizations disables all transformation rules. The unaltered input
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/opt/optbuilder/fk_cascade.go
Original file line number Diff line number Diff line change
Expand Up @@ -654,6 +654,10 @@ func (cb *onUpdateCascadeBuilder) Build(
updateExprs[i] = &tree.UpdateExpr{}
switch cb.action {
case tree.Cascade:
// TODO(radu): This requires special code in addUpdateCols to
// prevent this scopeColumn from being duplicated in mb.outScope
// (see the addCol anonymous function in addUpdateCols). Find a
// cleaner way to handle this.
updateExprs[i].Expr = &newValScopeCols[i]
case tree.SetNull:
updateExprs[i].Expr = tree.DNull
Expand Down
88 changes: 88 additions & 0 deletions pkg/sql/opt/optbuilder/testdata/fk-on-update-cascade
Original file line number Diff line number Diff line change
Expand Up @@ -730,3 +730,91 @@ root
└── filters
├── c:43 = child_multi.c:45
└── column3:44 = child_multi.q:47

# Regression test for #57148. A check constraint or computed column in a child
# table that references a column with the same name as the parent's synthesized
# update column should not result in an ambiguous column reference error. In
# this test the synthesized update column for the parent is "p" + "_new" =
# "p_new", which is the name of the FK column in the child.
exec-ddl
CREATE TABLE parent_check_ambig (p INT PRIMARY KEY)
----

exec-ddl
CREATE TABLE child_check_ambig (
c INT PRIMARY KEY,
p_new INT REFERENCES parent_check_ambig(p) ON UPDATE CASCADE,
i INT AS (p_new * 2) STORED,
CHECK (p_new > 0)
)
----

build-cascades
UPDATE parent_check_ambig SET p = p * 10 WHERE p > 1
----
root
├── update parent_check_ambig
│ ├── columns: <none>
│ ├── fetch columns: p:3
│ ├── update-mapping:
│ │ └── p_new:5 => p:1
│ ├── input binding: &1
│ ├── cascades
│ │ └── fk_p_new_ref_parent_check_ambig
│ └── project
│ ├── columns: p_new:5!null p:3!null crdb_internal_mvcc_timestamp:4
│ ├── select
│ │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4
│ │ ├── scan parent_check_ambig
│ │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4
│ │ └── filters
│ │ └── p:3 > 1
│ └── projections
│ └── p:3 * 10 [as=p_new:5]
└── cascade
└── update child_check_ambig
├── columns: <none>
├── fetch columns: c:10 child_check_ambig.p_new:11 i:12
├── update-mapping:
│ ├── p_new:15 => child_check_ambig.p_new:7
│ └── column16:16 => i:8
├── check columns: check1:17
├── input binding: &2
├── project
│ ├── columns: check1:17!null c:10!null child_check_ambig.p_new:11!null i:12 p:14!null p_new:15!null column16:16!null
│ ├── project
│ │ ├── columns: column16:16!null c:10!null child_check_ambig.p_new:11!null i:12 p:14!null p_new:15!null
│ │ ├── inner-join (hash)
│ │ │ ├── columns: c:10!null child_check_ambig.p_new:11!null i:12 p:14!null p_new:15!null
│ │ │ ├── scan child_check_ambig
│ │ │ │ ├── columns: c:10!null child_check_ambig.p_new:11 i:12
│ │ │ │ └── computed column expressions
│ │ │ │ └── i:12
│ │ │ │ └── child_check_ambig.p_new:11 * 2
│ │ │ ├── select
│ │ │ │ ├── columns: p:14!null p_new:15!null
│ │ │ │ ├── with-scan &1
│ │ │ │ │ ├── columns: p:14!null p_new:15!null
│ │ │ │ │ └── mapping:
│ │ │ │ │ ├── parent_check_ambig.p:3 => p:14
│ │ │ │ │ └── p_new:5 => p_new:15
│ │ │ │ └── filters
│ │ │ │ └── p:14 IS DISTINCT FROM p_new:15
│ │ │ └── filters
│ │ │ └── child_check_ambig.p_new:11 = p:14
│ │ └── projections
│ │ └── p_new:15 * 2 [as=column16:16]
│ └── projections
│ └── p_new:15 > 0 [as=check1:17]
└── f-k-checks
└── f-k-checks-item: child_check_ambig(p_new) -> parent_check_ambig(p)
└── anti-join (hash)
├── columns: p_new:18!null
├── with-scan &2
│ ├── columns: p_new:18!null
│ └── mapping:
│ └── p_new:15 => p_new:18
├── scan parent_check_ambig
│ └── columns: parent_check_ambig.p:19!null
└── filters
└── p_new:18 = parent_check_ambig.p:19
15 changes: 15 additions & 0 deletions pkg/sql/opt/optbuilder/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,21 @@ func (mb *mutationBuilder) addUpdateCols(exprs tree.UpdateExprs) {
}

addCol := func(expr tree.Expr, targetColID opt.ColumnID) {
// If the expression is already a scopeColumn, we can skip creating a
// new scopeColumn and proceed with type checking and adding the column
// to the list of source columns to update. The expression can be a
// scopeColumn when addUpdateCols is called from the
// onUpdateCascadeBuilder while building foreign key cascading updates.
//
// The input scopeColumn is a pointer to a column in mb.outScope. It was
// copied by value to projectionsScope. The checkCol function mutates
// the name of projected columns, so we must lookup the column in
// projectionsScope so that the correct scopeColumn is renamed.
if scopeCol, ok := expr.(*scopeColumn); ok {
checkCol(projectionsScope.getColumn(scopeCol.id), targetColID)
return
}

// Allow right side of SET to be DEFAULT.
if _, ok := expr.(tree.DefaultVal); ok {
expr = mb.parseDefaultOrComputedExpr(targetColID)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/rowenc/index_encoding_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -533,7 +533,7 @@ func TestEncodeContainingArrayInvertedIndexSpans(t *testing.T) {
arr := right.(*tree.DArray).Array
expectUnique := len(arr) > 0
for i := range arr {
if i > 0 && !reflect.DeepEqual(arr[i], arr[0]) {
if i > 0 && arr[i].Compare(&evalCtx, arr[0]) != 0 {
expectUnique = false
break
}
Expand Down

0 comments on commit d5222dc

Please sign in to comment.