From 3275eae66f95ecac64f7dfed555f96d7ff4f7887 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Fri, 15 Feb 2019 19:34:17 -0500 Subject: [PATCH 1/3] opt: test reproducing detached memo race Test that reproduces #34904. Writing the test exposed a problem in the `CopyAndReplace` API: there was no way to recursively copy a subtree when creating an internal node. The API was reverted to the first iteration Andy had in his PR. Release note: None --- pkg/sql/opt/memo/memo_test.go | 36 +------ pkg/sql/opt/norm/factory.go | 30 +++--- pkg/sql/opt/norm/factory_test.go | 27 ++---- pkg/sql/opt/optgen/cmd/optgen/factory_gen.go | 17 +--- .../opt/optgen/cmd/optgen/testdata/factory | 16 +--- pkg/sql/opt/testutils/build.go | 47 ++++++++++ pkg/sql/opt/xform/optimizer_test.go | 94 ++++++++++++------- 7 files changed, 145 insertions(+), 122 deletions(-) create mode 100644 pkg/sql/opt/testutils/build.go diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go index 5a272a466a21..179b5d402960 100644 --- a/pkg/sql/opt/memo/memo_test.go +++ b/pkg/sql/opt/memo/memo_test.go @@ -21,11 +21,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" - "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder" + opttestutils "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/opttester" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" "github.com/cockroachdb/cockroach/pkg/sql/opt/xform" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -61,24 +60,10 @@ func TestMemoInit(t *testing.T) { t.Fatal(err) } - stmt, err := parser.ParseOne("SELECT * FROM abc WHERE $1=10") - if err != nil { - t.Fatal(err) - } - - ctx := context.Background() - semaCtx := tree.MakeSemaContext() - if err := semaCtx.Placeholders.Init(stmt.NumPlaceholders, nil /* typeHints */); err != nil { - t.Fatal(err) - } evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) var o xform.Optimizer - o.Init(&evalCtx) - err = optbuilder.New(ctx, &semaCtx, &evalCtx, catalog, o.Factory(), stmt.AST).Build() - if err != nil { - t.Fatal(err) - } + opttestutils.BuildQuery(t, &o, catalog, &evalCtx, "SELECT * FROM abc WHERE $1=10") o.Init(&evalCtx) if !o.Memo().IsEmpty() { @@ -110,27 +95,16 @@ func TestMemoIsStale(t *testing.T) { // access via the view. catalog.Table(tree.NewTableName("t", "abc")).Revoked = true - ctx := context.Background() - semaCtx := tree.MakeSemaContext() - evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) - // Initialize context with starting values. + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) evalCtx.SessionData.Database = "t" - stmt, err := parser.ParseOne("SELECT a, b+1 FROM abcview WHERE c='foo'") - if err != nil { - t.Fatal(err) - } - var o xform.Optimizer - o.Init(&evalCtx) - err = optbuilder.New(ctx, &semaCtx, &evalCtx, catalog, o.Factory(), stmt.AST).Build() - if err != nil { - t.Fatal(err) - } + opttestutils.BuildQuery(t, &o, catalog, &evalCtx, "SELECT a, b+1 FROM abcview WHERE c='foo'") o.Memo().Metadata().AddSchemaDependency(catalog.Schema().Name(), catalog.Schema(), privilege.CREATE) o.Memo().Metadata().AddSchema(catalog.Schema()) + ctx := context.Background() stale := func() { t.Helper() if isStale, err := o.Memo().IsStale(ctx, &evalCtx, catalog); err != nil { diff --git a/pkg/sql/opt/norm/factory.go b/pkg/sql/opt/norm/factory.go index ddd426584f91..b3b7621c4652 100644 --- a/pkg/sql/opt/norm/factory.go +++ b/pkg/sql/opt/norm/factory.go @@ -165,19 +165,25 @@ func (f *Factory) CustomFuncs() *CustomFuncs { // The "replace" callback function allows the caller to override the default // traversal and cloning behavior with custom logic. It is called for each node // in the "from" subtree, and has the choice of constructing an arbitrary -// replacement node, or else delegating to the default behavior by returning the -// unchanged "e" expression. The default behavior simply constructs a copy of -// the source operator using children returned by recursive calls to the replace -// callback. Here is example usage: +// replacement node, or delegating to the default behavior by calling +// CopyAndReplaceDefault, which constructs a copy of the source operator using +// children returned by recursive calls to the replace callback. Note that if a +// non-leaf replacement node is constructed, its inputs must be copied using +// CopyAndReplaceDefault. // -// f.CopyAndReplace(from, fromProps, func(e opt.Expr) opt.Expr { +// Sample usage: +// +// var replaceFn ReplaceFunc +// replaceFn = func(e opt.Expr) opt.Expr { // if e.Op() == opt.PlaceholderOp { // return f.ConstructConst(evalPlaceholder(e)) // } // -// // Return unchanged "e" expression to get default behavior. -// return e -// }) +// // Copy e, calling replaceFn on its inputs recursively. +// return f.CopyAndReplaceDefault(e, replaceFn) +// } +// +// f.CopyAndReplace(from, fromProps, replaceFn) // // NOTE: Callers must take care to always create brand new copies of non- // singleton source nodes rather than referencing existing nodes. The source @@ -222,7 +228,8 @@ func (f *Factory) AssignPlaceholders(from *memo.Memo) (err error) { // Copy the "from" memo to this memo, replacing any Placeholder operators as // the copy proceeds. - f.CopyAndReplace(from.RootExpr().(memo.RelExpr), from.RootProps(), func(e opt.Expr) opt.Expr { + var replaceFn ReplaceFunc + replaceFn = func(e opt.Expr) opt.Expr { if placeholder, ok := e.(*memo.PlaceholderExpr); ok { d, err := e.(*memo.PlaceholderExpr).Value.Eval(f.evalCtx) if err != nil { @@ -230,8 +237,9 @@ func (f *Factory) AssignPlaceholders(from *memo.Memo) (err error) { } return f.ConstructConstVal(d, placeholder.DataType()) } - return e - }) + return f.CopyAndReplaceDefault(e, replaceFn) + } + f.CopyAndReplace(from.RootExpr().(memo.RelExpr), from.RootProps(), replaceFn) return nil } diff --git a/pkg/sql/opt/norm/factory_test.go b/pkg/sql/opt/norm/factory_test.go index 670c2d42649e..4e7764b545db 100644 --- a/pkg/sql/opt/norm/factory_test.go +++ b/pkg/sql/opt/norm/factory_test.go @@ -15,17 +15,15 @@ package norm_test import ( - "context" "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/norm" - "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder" + "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" "github.com/cockroachdb/cockroach/pkg/sql/opt/xform" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" ) @@ -83,22 +81,9 @@ func TestCopyAndReplace(t *testing.T) { } evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) - semaCtx := tree.MakeSemaContext() var o xform.Optimizer - o.Init(&evalCtx) - - stmt, err := parser.ParseOne("SELECT * FROM ab INNER JOIN cde ON a=c AND d=$1") - if err != nil { - t.Fatal(err) - } - if err := semaCtx.Placeholders.Init(stmt.NumPlaceholders, nil /* typeHints */); err != nil { - t.Fatal(err) - } - b := optbuilder.New(context.Background(), &semaCtx, &evalCtx, cat, o.Factory(), stmt.AST) - if err := b.Build(); err != nil { - t.Fatal(err) - } + testutils.BuildQuery(t, &o, cat, &evalCtx, "SELECT * FROM ab INNER JOIN cde ON a=c AND d=$1") if e := o.Optimize(); e.Op() != opt.MergeJoinOp { t.Errorf("expected optimizer to choose merge-join, not %v", e.Op()) @@ -107,12 +92,14 @@ func TestCopyAndReplace(t *testing.T) { m := o.Factory().DetachMemo() o.Init(&evalCtx) - o.Factory().CopyAndReplace(m.RootExpr().(memo.RelExpr), m.RootProps(), func(e opt.Expr) opt.Expr { + var replaceFn norm.ReplaceFunc + replaceFn = func(e opt.Expr) opt.Expr { if e.Op() == opt.PlaceholderOp { return o.Factory().ConstructConstVal(tree.NewDInt(1), types.Int) } - return e - }) + return o.Factory().CopyAndReplaceDefault(e, replaceFn) + } + o.Factory().CopyAndReplace(m.RootExpr().(memo.RelExpr), m.RootProps(), replaceFn) if e := o.Optimize(); e.Op() != opt.LookupJoinOp { t.Errorf("expected optimizer to choose lookup-join, not %v", e.Op()) diff --git a/pkg/sql/opt/optgen/cmd/optgen/factory_gen.go b/pkg/sql/opt/optgen/cmd/optgen/factory_gen.go index d5e5dd456808..852986520403 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/factory_gen.go +++ b/pkg/sql/opt/optgen/cmd/optgen/factory_gen.go @@ -284,11 +284,11 @@ func (g *factoryGen) genReplace() { // default traversal and cloning behavior for the factory's CopyAndReplace // method. func (g *factoryGen) genCopyAndReplaceDefault() { - g.w.writeIndent("// copyAndReplaceDefault performs the default traversal and cloning behavior\n") + g.w.writeIndent("// CopyAndReplaceDefault performs the default traversal and cloning behavior\n") g.w.writeIndent("// for the CopyAndReplace method. It constructs a copy of the given source\n") g.w.writeIndent("// operator using children copied (and potentially remapped) by the given replace\n") g.w.writeIndent("// function. See comments for CopyAndReplace for more details.\n") - g.w.nestIndent("func (f *Factory) copyAndReplaceDefault(src opt.Expr, replace ReplaceFunc) (dst opt.Expr)") + g.w.nestIndent("func (f *Factory) CopyAndReplaceDefault(src opt.Expr, replace ReplaceFunc) (dst opt.Expr)") g.w.nest("{\n") g.w.writeIndent("switch t := src.(type) {\n") @@ -376,23 +376,14 @@ func (g *factoryGen) genCopyAndReplaceDefault() { g.w.unnest("}\n\n") } - g.w.writeIndent("// invokeReplace wraps the user-provided replace function. If replace returns\n") - g.w.writeIndent("// its input unchanged, then invokeReplace automatically calls\n") - g.w.writeIndent("// copyAndReplaceDefault to get default replace behavior. See comments for\n") + g.w.writeIndent("// invokeReplace wraps the user-provided replace function. See comments for\n") g.w.writeIndent("// CopyAndReplace for more details.\n") g.w.nestIndent("func (f *Factory) invokeReplace(src opt.Expr, replace ReplaceFunc) (dst opt.Expr)") g.w.nest("{\n") g.w.nest("if rel, ok := src.(memo.RelExpr); ok {\n") g.w.writeIndent("src = rel.FirstExpr()\n") - g.w.writeIndent("dst = replace(src)\n") - g.w.unnest("}") - g.w.nest(" else {\n") - g.w.writeIndent("dst = replace(src)\n") g.w.unnest("}\n") - g.w.nest("if src == dst {\n") - g.w.writeIndent("return f.copyAndReplaceDefault(src, replace)\n") - g.w.unnest("}\n") - g.w.writeIndent("return dst\n") + g.w.nest("return replace(src)\n") g.w.unnest("}\n\n") } diff --git a/pkg/sql/opt/optgen/cmd/optgen/testdata/factory b/pkg/sql/opt/optgen/cmd/optgen/testdata/factory index 43e1cc89144c..ddeae3835f21 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/testdata/factory +++ b/pkg/sql/opt/optgen/cmd/optgen/testdata/factory @@ -218,11 +218,11 @@ func (f *Factory) replaceFiltersExpr(list memo.FiltersExpr, replace ReplaceFunc) return newList, true } -// copyAndReplaceDefault performs the default traversal and cloning behavior +// CopyAndReplaceDefault performs the default traversal and cloning behavior // for the CopyAndReplace method. It constructs a copy of the given source // operator using children copied (and potentially remapped) by the given replace // function. See comments for CopyAndReplace for more details. -func (f *Factory) copyAndReplaceDefault(src opt.Expr, replace ReplaceFunc) (dst opt.Expr) { +func (f *Factory) CopyAndReplaceDefault(src opt.Expr, replace ReplaceFunc) (dst opt.Expr) { switch t := src.(type) { case *memo.SelectExpr: return f.ConstructSelect( @@ -256,21 +256,13 @@ func (f *Factory) copyAndReplaceDefaultFiltersExpr(src memo.FiltersExpr, replace return dst } -// invokeReplace wraps the user-provided replace function. If replace returns -// its input unchanged, then invokeReplace automatically calls -// copyAndReplaceDefault to get default replace behavior. See comments for +// invokeReplace wraps the user-provided replace function. See comments for // CopyAndReplace for more details. func (f *Factory) invokeReplace(src opt.Expr, replace ReplaceFunc) (dst opt.Expr) { if rel, ok := src.(memo.RelExpr); ok { src = rel.FirstExpr() - dst = replace(src) - } else { - dst = replace(src) } - if src == dst { - return f.copyAndReplaceDefault(src, replace) - } - return dst + return replace(src) } func (f *Factory) DynamicConstruct(op opt.Operator, args ...interface{}) opt.Expr { diff --git a/pkg/sql/opt/testutils/build.go b/pkg/sql/opt/testutils/build.go new file mode 100644 index 000000000000..627558b1f4b7 --- /dev/null +++ b/pkg/sql/opt/testutils/build.go @@ -0,0 +1,47 @@ +// Copyright 2019 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package testutils + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" + "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder" + "github.com/cockroachdb/cockroach/pkg/sql/opt/xform" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +// BuildQuery initializes an optimizer and builds the given sql statement. +func BuildQuery( + t *testing.T, o *xform.Optimizer, catalog cat.Catalog, evalCtx *tree.EvalContext, sql string, +) { + stmt, err := parser.ParseOne(sql) + if err != nil { + t.Fatal(err) + } + + ctx := context.Background() + semaCtx := tree.MakeSemaContext() + if err := semaCtx.Placeholders.Init(stmt.NumPlaceholders, nil /* typeHints */); err != nil { + t.Fatal(err) + } + o.Init(evalCtx) + err = optbuilder.New(ctx, &semaCtx, evalCtx, catalog, o.Factory(), stmt.AST).Build() + if err != nil { + t.Fatal(err) + } +} diff --git a/pkg/sql/opt/xform/optimizer_test.go b/pkg/sql/opt/xform/optimizer_test.go index c65e5688a0f0..e26fdee2353b 100644 --- a/pkg/sql/opt/xform/optimizer_test.go +++ b/pkg/sql/opt/xform/optimizer_test.go @@ -15,18 +15,19 @@ package xform_test import ( - "context" "flag" "strings" + "sync" "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" - "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder" + "github.com/cockroachdb/cockroach/pkg/sql/opt/norm" + "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/opttester" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" "github.com/cockroachdb/cockroach/pkg/sql/opt/xform" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/testutils/datadriven" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -39,23 +40,9 @@ func TestDetachMemo(t *testing.T) { t.Fatal(err) } - stmt, err := parser.ParseOne("SELECT * FROM abc WHERE c=$1") - if err != nil { - t.Fatal(err) - } - - ctx := context.Background() - semaCtx := tree.MakeSemaContext() - if err := semaCtx.Placeholders.Init(stmt.NumPlaceholders, nil /* typeHints */); err != nil { - t.Fatal(err) - } - evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) - var o xform.Optimizer - o.Init(&evalCtx) - if err := optbuilder.New(ctx, &semaCtx, &evalCtx, catalog, o.Factory(), stmt.AST).Build(); err != nil { - t.Fatal(err) - } + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) + testutils.BuildQuery(t, &o, catalog, &evalCtx, "SELECT * FROM abc WHERE c=$1") before := o.DetachMemo() @@ -63,22 +50,7 @@ func TestDetachMemo(t *testing.T) { t.Error("memo expression should be reinitialized by DetachMemo") } - if err := semaCtx.Placeholders.Init(1 /* numPlaceholders */, nil /* typeHints */); err != nil { - t.Fatal(err) - } - o.Init(&evalCtx) - - stmt2, err := parser.ParseOne("SELECT a=$1 FROM abc") - if err != nil { - t.Fatal(err) - } - - if err := semaCtx.Placeholders.Init(stmt2.NumPlaceholders, nil /* typeHints */); err != nil { - t.Fatal(err) - } - if err := optbuilder.New(ctx, &semaCtx, &evalCtx, catalog, o.Factory(), stmt2.AST).Build(); err != nil { - t.Fatal(err) - } + testutils.BuildQuery(t, &o, catalog, &evalCtx, "SELECT a=$1 FROM abc") after := o.Memo() if after == before { @@ -106,6 +78,58 @@ func TestDetachMemo(t *testing.T) { } } +// TestDetachMemoRace reproduces the condition in #34904: a detached memo still +// aliases table annotations in the metadata. The problematic annotation is a +// statistics object. Construction of new expression can trigger calculation of +// new statistics. +func TestDetachMemoRace(t *testing.T) { + defer leaktest.AfterTest(t)() + + catalog := testcat.New() + + _, err := catalog.ExecuteDDL("CREATE TABLE abc (a INT, b INT, c INT, d INT)") + if err != nil { + t.Fatal(err) + } + var o xform.Optimizer + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) + testutils.BuildQuery(t, &o, catalog, &evalCtx, "SELECT * FROM abc WHERE a = $1") + mem := o.DetachMemo() + + var wg sync.WaitGroup + for i := 0; i < 4; i++ { + col := opt.ColumnID(i + 1) + wg.Add(1) + go func() { + var o xform.Optimizer + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) + o.Init(&evalCtx) + f := o.Factory() + var replaceFn norm.ReplaceFunc + replaceFn = func(e opt.Expr) opt.Expr { + if sel, ok := e.(*memo.SelectExpr); ok { + return f.ConstructSelect( + f.CopyAndReplaceDefault(sel.Input, replaceFn).(memo.RelExpr), + memo.FiltersExpr{{ + Condition: f.ConstructEq( + f.ConstructVariable(col), + f.ConstructConst(tree.NewDInt(10)), + ), + }}, + ) + } + return f.CopyAndReplaceDefault(e, replaceFn) + } + // Rewrite the filter to use a different column, which will trigger creation + // of new table statistics. If the statistics object is aliased, this will + // be racy. + f.CopyAndReplace(mem.RootExpr().(memo.RelExpr), mem.RootProps(), replaceFn) + wg.Done() + }() + } + wg.Wait() +} + // TestCoster files can be run separately like this: // make test PKG=./pkg/sql/opt/xform TESTS="TestCoster/sort" // make test PKG=./pkg/sql/opt/xform TESTS="TestCoster/scan" From cc04b06d351191c5c706b5382191e52ca0565125 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Fri, 15 Feb 2019 19:48:39 -0500 Subject: [PATCH 2/3] opt: replace ColumnMeta.TableMeta with a TableID The TableMeta field is problematic: it needs to be fixed up when copying Metadata; and since it points into a slice that we append to, it may or may not be aliased with the corresponding entry in `tables`. Avoiding these complication by just storing the TabeID. The `md` backpointer is also removed and QualifiedAlias is moved to Metadata. Release note: None --- pkg/sql/opt/column_meta.go | 73 ++------------------------ pkg/sql/opt/exec/execbuilder/format.go | 2 +- pkg/sql/opt/memo/expr_format.go | 4 +- pkg/sql/opt/metadata.go | 67 +++++++++++++++++++++-- pkg/sql/opt/norm/join.go | 28 +++++----- 5 files changed, 85 insertions(+), 89 deletions(-) diff --git a/pkg/sql/opt/column_meta.go b/pkg/sql/opt/column_meta.go index 63efb6bf905f..8ddfb89c7175 100644 --- a/pkg/sql/opt/column_meta.go +++ b/pkg/sql/opt/column_meta.go @@ -15,9 +15,6 @@ package opt import ( - "strings" - - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" "github.com/cockroachdb/cockroach/pkg/util" ) @@ -58,73 +55,9 @@ type ColumnMeta struct { // Type is the scalar SQL type of this column. Type types.T - // TableMeta is the metadata for the base table to which this column belongs. - // If the column was synthesized (i.e. no base table), then is is null. - TableMeta *TableMeta - - // md is a back-reference to the query metadata. - md *Metadata -} - -// QualifiedAlias returns the column alias, possibly qualified with the table, -// schema, or database name: -// -// 1. If fullyQualify is true, then the returned alias is prefixed by the -// original, fully qualified name of the table: tab.Name().FQString(). -// -// 2. If there's another column in the metadata with the same column alias but -// a different table name, then prefix the column alias with the table -// name: "tabName.columnAlias". -// -func (cm *ColumnMeta) QualifiedAlias(fullyQualify bool) string { - if cm.TableMeta == nil { - // Column doesn't belong to a table, so no need to qualify it further. - return cm.Alias - } - md := cm.md - - // If a fully qualified alias has not been requested, then only qualify it if - // it would otherwise be ambiguous. - var tabAlias tree.TableName - qualify := fullyQualify - if !fullyQualify { - for i := range md.cols { - if i == int(cm.MetaID-1) { - continue - } - - // If there are two columns with same alias, then column is ambiguous. - cm2 := &md.cols[i] - if cm2.Alias == cm.Alias { - tabAlias = cm.TableMeta.Alias - if cm2.TableMeta == nil { - qualify = true - } else { - // Only qualify if the qualified names are actually different. - tabAlias2 := cm2.TableMeta.Alias - if tabAlias.String() != tabAlias2.String() { - qualify = true - } - } - } - } - } - - // If the column name should not even be partly qualified, then no more to do. - if !qualify { - return cm.Alias - } - - var sb strings.Builder - if fullyQualify { - s := cm.TableMeta.Table.Name().FQString() - sb.WriteString(s) - } else { - sb.WriteString(tabAlias.String()) - } - sb.WriteRune('.') - sb.WriteString(cm.Alias) - return sb.String() + // Table is the base table to which this column belongs. + // If the column was synthesized (i.e. no base table), then it is 0. + Table TableID } // ToSet converts a column id list to a column id set. diff --git a/pkg/sql/opt/exec/execbuilder/format.go b/pkg/sql/opt/exec/execbuilder/format.go index cdb1f397f7a7..82527de20dcf 100644 --- a/pkg/sql/opt/exec/execbuilder/format.go +++ b/pkg/sql/opt/exec/execbuilder/format.go @@ -64,7 +64,7 @@ func fmtInterceptor(f *memo.ExprFmtCtx, tp treeprinter.Node, nd opt.Expr) bool { fmtCtx := tree.NewFmtCtx(tree.FmtSimple) fmtCtx.SetIndexedVarFormat(func(ctx *tree.FmtCtx, idx int) { fullyQualify := !f.HasFlags(memo.ExprFmtHideQualifications) - alias := md.ColumnMeta(opt.ColumnID(idx + 1)).QualifiedAlias(fullyQualify) + alias := md.QualifiedAlias(opt.ColumnID(idx+1), fullyQualify) ctx.WriteString(alias) }) expr.Format(fmtCtx) diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index fa4f8ccc41fd..46bc9e3b6859 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -715,7 +715,7 @@ func formatCol( colMeta := md.ColumnMeta(id) if label == "" { fullyQualify := !f.HasFlags(ExprFmtHideQualifications) - label = colMeta.QualifiedAlias(fullyQualify) + label = md.QualifiedAlias(id, fullyQualify) } if !isSimpleColumnName(label) { @@ -752,7 +752,7 @@ func FormatPrivate(f *ExprFmtCtx, private interface{}, physProps *physical.Requi switch t := private.(type) { case *opt.ColumnID: fullyQualify := !f.HasFlags(ExprFmtHideQualifications) - label := f.Memo.metadata.ColumnMeta(*t).QualifiedAlias(fullyQualify) + label := f.Memo.metadata.QualifiedAlias(*t, fullyQualify) fmt.Fprintf(f.Buffer, " %s", label) case *TupleOrdinal: diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 6867d513d0b9..be5227aa67d9 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -18,6 +18,7 @@ import ( "context" "fmt" "math/bits" + "strings" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -274,7 +275,6 @@ func (md *Metadata) AddTableWithAlias(tab cat.Table, alias *tree.TableName) Tabl md.tables = make([]TableMeta, 0, 4) } md.tables = append(md.tables, TableMeta{MetaID: tabID, Table: tab, Alias: *alias}) - tabMeta := md.TableMeta(tabID) colCount := tab.DeletableColumnCount() if md.cols == nil { @@ -284,7 +284,7 @@ func (md *Metadata) AddTableWithAlias(tab cat.Table, alias *tree.TableName) Tabl for i := 0; i < colCount; i++ { col := tab.Column(i) colID := md.AddColumn(string(col.ColName()), col.DatumType()) - md.ColumnMeta(colID).TableMeta = tabMeta + md.ColumnMeta(colID).Table = tabID } return tabID @@ -327,7 +327,7 @@ func (md *Metadata) AddColumn(alias string, typ types.T) ColumnID { alias = fmt.Sprintf("column%d", len(md.cols)+1) } colID := ColumnID(len(md.cols) + 1) - md.cols = append(md.cols, ColumnMeta{MetaID: colID, Alias: alias, Type: typ, md: md}) + md.cols = append(md.cols, ColumnMeta{MetaID: colID, Alias: alias, Type: typ}) return colID } @@ -343,6 +343,67 @@ func (md *Metadata) ColumnMeta(colID ColumnID) *ColumnMeta { return &md.cols[colID.index()] } +// QualifiedAlias returns the column alias, possibly qualified with the table, +// schema, or database name: +// +// 1. If fullyQualify is true, then the returned alias is prefixed by the +// original, fully qualified name of the table: tab.Name().FQString(). +// +// 2. If there's another column in the metadata with the same column alias but +// a different table name, then prefix the column alias with the table +// name: "tabName.columnAlias". +// +func (md *Metadata) QualifiedAlias(colID ColumnID, fullyQualify bool) string { + cm := md.ColumnMeta(colID) + if cm.Table == 0 { + // Column doesn't belong to a table, so no need to qualify it further. + return cm.Alias + } + + // If a fully qualified alias has not been requested, then only qualify it if + // it would otherwise be ambiguous. + var tabAlias tree.TableName + qualify := fullyQualify + if !fullyQualify { + for i := range md.cols { + if i == int(cm.MetaID-1) { + continue + } + + // If there are two columns with same alias, then column is ambiguous. + cm2 := &md.cols[i] + if cm2.Alias == cm.Alias { + tabAlias = md.TableMeta(cm.Table).Alias + if cm2.Table == 0 { + qualify = true + } else { + // Only qualify if the qualified names are actually different. + tabAlias2 := md.TableMeta(cm2.Table).Alias + if tabAlias.String() != tabAlias2.String() { + qualify = true + } + } + } + } + } + + // If the column name should not even be partly qualified, then no more to do. + if !qualify { + return cm.Alias + } + + var sb strings.Builder + if fullyQualify { + s := md.TableMeta(cm.Table).Table.Name().FQString() + sb.WriteString(s) + } else { + sb.WriteString(tabAlias.String()) + } + sb.WriteRune('.') + sb.WriteString(cm.Alias) + return sb.String() +} + // SequenceID uniquely identifies the usage of a sequence within the scope of a // query. SequenceID 0 is reserved to mean "unknown sequence". type SequenceID uint64 diff --git a/pkg/sql/opt/norm/join.go b/pkg/sql/opt/norm/join.go index 2f8bdd5cecca..c25fae98baeb 100644 --- a/pkg/sql/opt/norm/join.go +++ b/pkg/sql/opt/norm/join.go @@ -349,7 +349,7 @@ func (c *CustomFuncs) JoinFiltersMatchAllLeftRows( md := c.f.Metadata() - var leftTabMeta, rightTabMeta *opt.TableMeta + var leftTab, rightTab opt.TableID // Any left columns that don't match conditions 1-4 end up in this set. var remainingLeftColIDs opt.ColSet @@ -386,25 +386,25 @@ func (c *CustomFuncs) JoinFiltersMatchAllLeftRows( return false } - if leftTabMeta == nil { - leftTabMeta = md.ColumnMeta(leftColID).TableMeta - rightTabMeta = md.ColumnMeta(rightColID).TableMeta - if leftTabMeta == nil || rightTabMeta == nil { + if leftTab == 0 { + leftTab = md.ColumnMeta(leftColID).Table + rightTab = md.ColumnMeta(rightColID).Table + if leftTab == 0 || rightTab == 0 { // Condition #2: Columns don't come from base tables. return false } - } else if md.ColumnMeta(leftColID).TableMeta != leftTabMeta { + } else if md.ColumnMeta(leftColID).Table != leftTab { // Condition #2: All left columns don't come from same table. return false - } else if md.ColumnMeta(rightColID).TableMeta != rightTabMeta { + } else if md.ColumnMeta(rightColID).Table != rightTab { // Condition #2: All right columns don't come from same table. return false } - if leftTabMeta.Table == rightTabMeta.Table { + if md.TableMeta(leftTab).Table == md.TableMeta(rightTab).Table { // Check self-join case. - leftColOrd := leftTabMeta.MetaID.ColumnOrdinal(leftColID) - rightColOrd := rightTabMeta.MetaID.ColumnOrdinal(rightColID) + leftColOrd := leftTab.ColumnOrdinal(leftColID) + rightColOrd := rightTab.ColumnOrdinal(rightColID) if leftColOrd != rightColOrd { // Condition #4: Left and right column ordinals do not match. return false @@ -421,6 +421,8 @@ func (c *CustomFuncs) JoinFiltersMatchAllLeftRows( var leftRightColMap map[opt.ColumnID]opt.ColumnID // Condition #5: All remaining left columns correspond to a foreign key relation. + leftTabMeta := md.TableMeta(leftTab) + rightTabMeta := md.TableMeta(rightTab) for i, cnt := 0, leftTabMeta.Table.IndexCount(); i < cnt; i++ { index := leftTabMeta.Table.Index(i) fkRef, ok := index.ForeignKey() @@ -458,7 +460,7 @@ func (c *CustomFuncs) JoinFiltersMatchAllLeftRows( var leftIndexCols opt.ColSet for j := 0; j < fkPrefix; j++ { ord := index.Column(j).Ordinal - leftIndexCols.Add(int(leftTabMeta.MetaID.ColumnID(ord))) + leftIndexCols.Add(int(leftTab.ColumnID(ord))) } if !remainingLeftColIDs.SubsetOf(leftIndexCols) { @@ -480,14 +482,14 @@ func (c *CustomFuncs) JoinFiltersMatchAllLeftRows( // referenced) that it's being equated to. fkMatch := true for j := 0; j < fkPrefix; j++ { - indexLeftCol := leftTabMeta.MetaID.ColumnID(index.Column(j).Ordinal) + indexLeftCol := leftTab.ColumnID(index.Column(j).Ordinal) // Not every fk column needs to be in the equality conditions. if !remainingLeftColIDs.Contains(int(indexLeftCol)) { continue } - indexRightCol := rightTabMeta.MetaID.ColumnID(fkIndex.Column(j).Ordinal) + indexRightCol := rightTab.ColumnID(fkIndex.Column(j).Ordinal) if rightCol, ok := leftRightColMap[indexLeftCol]; !ok || rightCol != indexRightCol { fkMatch = false From c1401e0b7c0fe14e4306ee6f4a58515b97a9a52f Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Sat, 16 Feb 2019 14:22:09 -0500 Subject: [PATCH 3/3] opt: fix race caused by shared table annotations When we assign placeholders on a detached memo, we copy the metadata. However, this does a shallow copy of table annotations; this is problematic for the statistics annotation which is a mutable object. The fix is to register a copy function for each type of table annotation as part of `NewTableAnnID`. For statistics, we simply clear out the annotation because it can be recalculated as needed (and it turns out to be faster than copying it). Fixes #34904. Release note (bug fix): Fixed a crash related to cached plans. --- pkg/sql/opt/metadata.go | 16 +++++++++++++++- pkg/sql/opt/table_meta.go | 13 +++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index be5227aa67d9..246281d6c64b 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -89,6 +89,8 @@ type Metadata struct { // might resolve to the same object now but to different objects later; we // want to verify the resolution of both names. deps []mdDep + + // NOTE! When adding fields here, update CopyFrom. } type mdDep struct { @@ -132,13 +134,25 @@ func (md *Metadata) Init() { // CopyFrom initializes the metadata with a copy of the provided metadata. // This metadata can then be modified independent of the copied metadata. +// +// Table annotations are not transferred over; all annotations are unset on +// the copy. func (md *Metadata) CopyFrom(from *Metadata) { - if len(md.cols) != 0 || len(md.tables) != 0 || len(md.deps) != 0 { + if len(md.schemas) != 0 || len(md.cols) != 0 || len(md.tables) != 0 || + len(md.sequences) != 0 || len(md.deps) != 0 { panic("CopyFrom requires empty destination") } md.schemas = append(md.schemas, from.schemas...) md.cols = append(md.cols, from.cols...) md.tables = append(md.tables, from.tables...) + + // Clear table annotations. These objects can be mutable and can't be safely + // shared between different metadata instances. + for i := range md.tables { + md.tables[i].clearAnnotations() + } + + md.sequences = append(md.sequences, from.sequences...) md.deps = append(md.deps, from.deps...) } diff --git a/pkg/sql/opt/table_meta.go b/pkg/sql/opt/table_meta.go index 084b4299d2f7..8297a37ce92c 100644 --- a/pkg/sql/opt/table_meta.go +++ b/pkg/sql/opt/table_meta.go @@ -83,6 +83,11 @@ func (t TableID) index() int { // metadata, which can be used to avoid recalculating base table properties or // other information each time it's needed. // +// WARNING! When copying memo metadata (which happens when we use a cached +// memo), the annotations are cleared. Any code using a annotation must treat +// this as a best-effort cache and be prepared to repopulate the annotation as +// necessary. +// // To create a TableAnnID, call NewTableAnnID during Go's program initialization // phase. The returned TableAnnID never clashes with other annotations on the // same table. Here is a usage example: @@ -128,6 +133,14 @@ type TableMeta struct { anns [maxTableAnnIDCount]interface{} } +// clearAnnotations resets all the table annotations; used when copying a +// Metadata. +func (tm *TableMeta) clearAnnotations() { + for i := range tm.anns { + tm.anns[i] = nil + } +} + // IndexColumns returns the metadata IDs for the set of columns in the given // index. // TODO(justin): cache this value in the table metadata.