Skip to content

Commit

Permalink
sql,tree: change SearchPath to avoid allocations
Browse files Browse the repository at this point in the history
The closure-oriented interface was forcing the closures and the variables they
referenced to escape to the heap. This change, while not beautiful, ends up being
much more efficient.

Release note: None
  • Loading branch information
ajwerner committed Oct 4, 2022
1 parent 0b63e88 commit e99a182
Show file tree
Hide file tree
Showing 7 changed files with 48 additions and 48 deletions.
16 changes: 8 additions & 8 deletions pkg/sql/schema_resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -401,23 +401,23 @@ func (sr *schemaResolver) ResolveFunction(
sc := prefix.Schema
udfDef, _ = sc.GetResolvedFuncDefinition(fn.Object())
} else {
if err := path.IterateSearchPath(func(schema string) error {
for i, n := 0, path.NumElements(); i < n; i++ {
schema := path.GetSchema(i)
found, prefix, err := sr.LookupSchema(ctx, sr.CurrentDatabase(), schema)
if err != nil {
return err
return nil, err
}
if !found {
return nil
continue
}
curUdfDef, found := prefix.Schema.GetResolvedFuncDefinition(fn.Object())
if !found {
return nil
continue
}

udfDef, err = udfDef.MergeWith(curUdfDef)
return err
}); err != nil {
return nil, err
if err != nil {
return nil, err
}
}
}

Expand Down
1 change: 0 additions & 1 deletion pkg/sql/sem/tree/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,6 @@ go_library(
"//pkg/util/encoding",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/ipaddr",
"//pkg/util/iterutil",
"//pkg/util/json",
"//pkg/util/pretty",
"//pkg/util/stringencoding",
Expand Down
20 changes: 6 additions & 14 deletions pkg/sql/sem/tree/function_definition.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)
Expand Down Expand Up @@ -292,15 +291,10 @@ func (fd *ResolvedFunctionDefinition) MatchOverload(
if explicitSchema != "" {
findMatches(explicitSchema)
} else {
err := searchPath.IterateSearchPath(func(schema string) error {
findMatches(schema)
if found {
return iterutil.StopIteration()
for i, n := 0, searchPath.NumElements(); i < n; i++ {
if findMatches(searchPath.GetSchema(i)); found {
break
}
return nil
})
if err != nil {
return QualifiedOverload{}, err
}
}

Expand Down Expand Up @@ -430,15 +424,13 @@ func GetBuiltinFuncDefinition(

// If not in pg_catalog, go through search path.
var resolvedDef *ResolvedFunctionDefinition
if err := searchPath.IterateSearchPath(func(schema string) error {
for i, n := 0, searchPath.NumElements(); i < n; i++ {
schema := searchPath.GetSchema(i)
fullName := schema + "." + fName.Object()
if def, ok := ResolvedBuiltinFuncDefs[fullName]; ok {
resolvedDef = def
return iterutil.StopIteration()
break
}
return nil
}); err != nil {
return nil, err
}

return resolvedDef, nil
Expand Down
15 changes: 7 additions & 8 deletions pkg/sql/sem/tree/name_resolution.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,22 +129,21 @@ type QualifiedNameResolver interface {
// SearchPath encapsulates the ordered list of schemas in the current database
// to search during name resolution.
type SearchPath interface {
// NumElements returns the number of elements in the SearchPath.
NumElements() int

// IterateSearchPath calls the passed function for every element of the
// SearchPath in order. If an error is returned, iteration stops. If the
// error is iterutil.StopIteration, no error will be returned from the
// method.
IterateSearchPath(func(schema string) error) error
// GetSchema returns the schema at the ord offset in the SearchPath.
// Note that it will return the empty string if the ordinal is out of range.
GetSchema(ord int) string
}

// EmptySearchPath is a SearchPath with no members.
var EmptySearchPath SearchPath = emptySearchPath{}

type emptySearchPath struct{}

func (emptySearchPath) IterateSearchPath(func(string) error) error {
return nil
}
func (emptySearchPath) NumElements() int { return 0 }
func (emptySearchPath) GetSchema(i int) string { return "" }

func newInvColRef(n *UnresolvedName) error {
return pgerror.NewWithDepthf(1, pgcode.InvalidColumnReference,
Expand Down
12 changes: 4 additions & 8 deletions pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
Expand Down Expand Up @@ -2989,23 +2988,20 @@ func getMostSignificantOverload(

found := false
var ret QualifiedOverload
err := searchPath.IterateSearchPath(func(schema string) error {
for i, n := 0, searchPath.NumElements(); i < n; i++ {
schema := searchPath.GetSchema(i)
for i := range overloads {
if overloads[i].(QualifiedOverload).Schema == schema {
if found {
return ambiguousError()
return QualifiedOverload{}, ambiguousError()
}
found = true
ret = overloads[i].(QualifiedOverload)
}
}
if found {
return iterutil.StopIteration()
break
}
return nil
})
if err != nil {
return QualifiedOverload{}, err
}
if !found {
// This should never happen. Otherwise, it means we get function from a
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/sessiondata/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ go_library(
"//pkg/sql/sem/catconstants",
"//pkg/sql/sessiondatapb",
"//pkg/util/duration",
"//pkg/util/iterutil",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/timeutil/pgdate",
Expand Down
31 changes: 23 additions & 8 deletions pkg/sql/sessiondata/search_path.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
)

// DefaultSearchPath is the search path used by virgin sessions.
Expand Down Expand Up @@ -292,15 +291,31 @@ func (iter *SearchPathIter) Next() (path string, ok bool) {
return "", false
}

// IterateSearchPath iterates the search path. If a non-nil error is
// returned, iteration is stopped. If iterutils.StopIteration() is returned
// from the iteration function, a nil error is returned to the caller.
func (s *SearchPath) IterateSearchPath(f func(schema string) error) error {
// NumElements returns the number of elements in the search path.
func (s *SearchPath) NumElements() int {
// TODO(ajwerner): Refactor this so that we don't need to do an O(N)
// operation to find the number of elements. In practice it doesn't matter
// much because search paths tend to be short.
iter := s.Iter()
var i int
for _, ok := iter.Next(); ok; _, ok = iter.Next() {
i++
}
return i
}

// GetSchema returns the ith schema element if it is in range.
func (s *SearchPath) GetSchema(ord int) string {
// TODO(ajwerner): Refactor this so that we don't need to do an O(n)
// operation to find the nth element. In practice it doesn't matter
// much because search paths tend to be short.
iter := s.Iter()
var i int
for schema, ok := iter.Next(); ok; schema, ok = iter.Next() {
if err := f(schema); err != nil {
return iterutil.Map(err)
if ord == i {
return schema
}
i++
}
return nil
return ""
}

0 comments on commit e99a182

Please sign in to comment.