Skip to content

Commit

Permalink
opt: test reproducing detached memo race
Browse files Browse the repository at this point in the history
Test that reproduces cockroachdb#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
  • Loading branch information
RaduBerinde committed Feb 20, 2019
1 parent 9b97dbc commit 3275eae
Show file tree
Hide file tree
Showing 7 changed files with 145 additions and 122 deletions.
36 changes: 5 additions & 31 deletions pkg/sql/opt/memo/memo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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() {
Expand Down Expand Up @@ -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 {
Expand Down
30 changes: 19 additions & 11 deletions pkg/sql/opt/norm/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -222,16 +228,18 @@ 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 {
panic(placeholderError{err})
}
return f.ConstructConstVal(d, placeholder.DataType())
}
return e
})
return f.CopyAndReplaceDefault(e, replaceFn)
}
f.CopyAndReplace(from.RootExpr().(memo.RelExpr), from.RootProps(), replaceFn)

return nil
}
Expand Down
27 changes: 7 additions & 20 deletions pkg/sql/opt/norm/factory_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -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())
Expand All @@ -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())
Expand Down
17 changes: 4 additions & 13 deletions pkg/sql/opt/optgen/cmd/optgen/factory_gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")

Expand Down Expand Up @@ -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")
}

Expand Down
16 changes: 4 additions & 12 deletions pkg/sql/opt/optgen/cmd/optgen/testdata/factory
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down Expand Up @@ -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 {
Expand Down
47 changes: 47 additions & 0 deletions pkg/sql/opt/testutils/build.go
Original file line number Diff line number Diff line change
@@ -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)
}
}
Loading

0 comments on commit 3275eae

Please sign in to comment.