Skip to content

Commit

Permalink
sql: fix data race in ResolvableFunctionReference
Browse files Browse the repository at this point in the history
Previously, ResolvableFunctionReference was the site of a data race in
which statements like SHOW QUERIES, which need to format an AST
concurrently with query planning and running, raced with the query
planner which as a side effect of planning mutates
ResolvableFunctionReference. Ideally, ResolvableFunctionReference would
be immutable like a good little AST node.

This commit separates ResolvableFunctionReference into two fields, one
immutable (the UnresolvedName that it starts with from the parser) and
one mutable (the resolved function itself). The formatter only reads the
immutable part, so the planner is free to mutate the resolved part as it
wishes.

Release note: None
  • Loading branch information
jordanlewis committed Mar 10, 2020
1 parent c28acc2 commit 7316e7a
Show file tree
Hide file tree
Showing 8 changed files with 31 additions and 39 deletions.
2 changes: 1 addition & 1 deletion pkg/internal/sqlsmith/relational.go
Original file line number Diff line number Diff line change
Expand Up @@ -651,7 +651,7 @@ var countStar = func() tree.TypedExpr {
fn := tree.FunDefs["count"]
typ := types.Int
return tree.NewTypedFuncExpr(
tree.ResolvableFunctionReference{FunctionReference: fn},
tree.ResolvableFunctionReference{ResolvedFunction: fn},
0, /* aggQualifier */
tree.TypedExprs{tree.UnqualifiedStar{}},
nil, /* filter */
Expand Down
2 changes: 1 addition & 1 deletion pkg/internal/sqlsmith/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -441,7 +441,7 @@ func makeFunc(s *Smither, ctx Context, typ *types.T, refs colRefs) (tree.TypedEx
// Cast the return and arguments to prevent ambiguity during function
// implementation choosing.
return castType(tree.NewTypedFuncExpr(
tree.ResolvableFunctionReference{FunctionReference: fn.def},
tree.ResolvableFunctionReference{ResolvedFunction: fn.def},
0, /* aggQualifier */
args,
nil, /* filter */
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1757,7 +1757,7 @@ func makeHashShardComputeExpr(colNames []string, buckets int) *string {
}
unresolvedFunc := func(funcName string) tree.ResolvableFunctionReference {
return tree.ResolvableFunctionReference{
FunctionReference: unresolvedName(funcName),
Name: unresolvedName(funcName),
}
}
hashedColumnExpr := func(colName string) tree.Expr {
Expand Down
4 changes: 1 addition & 3 deletions pkg/sql/opt/optbuilder/scope.go
Original file line number Diff line number Diff line change
Expand Up @@ -1224,9 +1224,7 @@ func (s *scope) replaceCount(
// take any arguments).
e := &tree.FuncExpr{
Func: tree.ResolvableFunctionReference{
FunctionReference: &tree.UnresolvedName{
NumParts: 1, Parts: tree.NameParts{"count_rows"},
},
ResolvedFunction: tree.FunDefs["count_rows"],
},
}
// We call TypeCheck to fill in FuncExpr internals. This is a fixed
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/parser/help.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ func helpWithFunction(sqllex sqlLexer, f tree.ResolvableFunctionReference) int {
}

msg := HelpMessage{
Function: f.String(),
Function: f.ResolvedFunction.String(),
HelpMessageBody: HelpMessageBody{
Category: d.Category,
SeeAlso: base.DocsURL("functions-and-operators.html"),
Expand Down Expand Up @@ -133,7 +133,7 @@ func helpWithFunction(sqllex sqlLexer, f tree.ResolvableFunctionReference) int {

func helpWithFunctionByName(sqllex sqlLexer, s string) int {
un := &tree.UnresolvedName{NumParts: 1, Parts: tree.NameParts{s}}
return helpWithFunction(sqllex, tree.ResolvableFunctionReference{FunctionReference: un})
return helpWithFunction(sqllex, tree.ResolvableFunctionReference{Name: un})
}

const (
Expand Down
5 changes: 1 addition & 4 deletions pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -154,9 +154,6 @@ func (u *sqlSymUnion) unresolvedName() *tree.UnresolvedName {
func (u *sqlSymUnion) unresolvedObjectName() *tree.UnresolvedObjectName {
return u.val.(*tree.UnresolvedObjectName)
}
func (u *sqlSymUnion) functionReference() tree.FunctionReference {
return u.val.(tree.FunctionReference)
}
func (u *sqlSymUnion) tablePatterns() tree.TablePatterns {
return u.val.(tree.TablePatterns)
}
Expand Down Expand Up @@ -482,7 +479,7 @@ func (u *sqlSymUnion) scrubOption() tree.ScrubOption {
return u.val.(tree.ScrubOption)
}
func (u *sqlSymUnion) resolvableFuncRefFromName() tree.ResolvableFunctionReference {
return tree.ResolvableFunctionReference{FunctionReference: u.unresolvedName()}
return tree.ResolvableFunctionReference{Name: u.unresolvedName()}
}
func (u *sqlSymUnion) rowsFromExpr() *tree.RowsFromExpr {
return u.val.(*tree.RowsFromExpr)
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/sem/tree/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -1336,8 +1336,9 @@ func (node *FuncExpr) Format(ctx *FmtCtx) {
}

// We need to remove name anonymization for the function name in
// particular. Do this by overriding the flags.
ctx.WithFlags(ctx.flags&^FmtAnonymize, func() {
// particular, as well as show-types, which will be handled by this node.
// Do this by overriding the flags.
ctx.WithFlags(ctx.flags&^FmtAnonymize&^FmtShowTypes, func() {
ctx.FormatNode(&node.Func)
})

Expand Down
46 changes: 21 additions & 25 deletions pkg/sql/sem/tree/function_name.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,6 @@
package tree

import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
Expand All @@ -28,15 +26,27 @@ import (
// first call to its Resolve() method.

// ResolvableFunctionReference implements the editable reference cell
// of a FuncExpr. The FunctionRerence is updated by the Normalize()
// method.
// of a FuncExpr.
type ResolvableFunctionReference struct {
FunctionReference
Name *UnresolvedName
ResolvedFunction *FunctionDefinition
}

// Format implements the NodeFormatter interface.
func (fn *ResolvableFunctionReference) Format(ctx *FmtCtx) {
ctx.FormatNode(fn.FunctionReference)
// This Format method tries to use the UnresolvedName component first, because
// ResolvableFunctionReference is unfortunately mutable at the current time,
// which leads to data races when formatting ASTs for SHOW QUERIES and the
// like, because the Optimizer might be mutating this node while someone is
// asking to read it. To get around this, we just print out the UnresolvedName
// if it exists. If it doesn't, which sometimes happen when we synthesize an
// RFR internally, we fall back to formatting the resolved function itself,
// which in that case will be immutable as well.
if fn.Name != nil {
ctx.FormatNode(fn.Name)
} else {
ctx.FormatNode(fn.ResolvedFunction)
}
}
func (fn *ResolvableFunctionReference) String() string { return AsString(fn) }

Expand All @@ -45,21 +55,14 @@ func (fn *ResolvableFunctionReference) String() string { return AsString(fn) }
func (fn *ResolvableFunctionReference) Resolve(
searchPath sessiondata.SearchPath,
) (*FunctionDefinition, error) {
switch t := fn.FunctionReference.(type) {
case *FunctionDefinition:
return t, nil
case *UnresolvedName:
fd, err := t.ResolveFunction(searchPath)
if fn.ResolvedFunction == nil {
fd, err := fn.Name.ResolveFunction(searchPath)
if err != nil {
return nil, err
}
fn.FunctionReference = fd
return fd, nil
default:
return nil, errors.AssertionFailedf("unknown function name type: %+v (%T)",
fn.FunctionReference, fn.FunctionReference,
)
fn.ResolvedFunction = fd
}
return fn.ResolvedFunction, nil
}

// WrapFunction creates a new ResolvableFunctionReference
Expand All @@ -69,14 +72,7 @@ func WrapFunction(n string) ResolvableFunctionReference {
if !ok {
panic(errors.AssertionFailedf("function %s() not defined", log.Safe(n)))
}
return ResolvableFunctionReference{fd}
}

// FunctionReference is the common interface to UnresolvedName and QualifiedFunctionName.
type FunctionReference interface {
fmt.Stringer
NodeFormatter
functionReference()
return ResolvableFunctionReference{ResolvedFunction: fd}
}

func (*UnresolvedName) functionReference() {}
Expand Down

0 comments on commit 7316e7a

Please sign in to comment.