diff --git a/pkg/sql/schema_resolver.go b/pkg/sql/schema_resolver.go index 54ea58865f97..c6046e678340 100644 --- a/pkg/sql/schema_resolver.go +++ b/pkg/sql/schema_resolver.go @@ -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 + } } } diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 940ecaae9657..7b1d44e4aa87 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -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", diff --git a/pkg/sql/sem/tree/function_definition.go b/pkg/sql/sem/tree/function_definition.go index 0e0b56751c1b..7ab8b1eaf63f 100644 --- a/pkg/sql/sem/tree/function_definition.go +++ b/pkg/sql/sem/tree/function_definition.go @@ -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" ) @@ -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 } } @@ -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 diff --git a/pkg/sql/sem/tree/name_resolution.go b/pkg/sql/sem/tree/name_resolution.go index 0f0c352e1676..92f01cc89ed1 100644 --- a/pkg/sql/sem/tree/name_resolution.go +++ b/pkg/sql/sem/tree/name_resolution.go @@ -129,12 +129,12 @@ 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. @@ -142,9 +142,8 @@ 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, diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 86c78284929a..b7ea132e7c1a 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -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" @@ -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 diff --git a/pkg/sql/sessiondata/BUILD.bazel b/pkg/sql/sessiondata/BUILD.bazel index 9915928a8845..131d760a51ce 100644 --- a/pkg/sql/sessiondata/BUILD.bazel +++ b/pkg/sql/sessiondata/BUILD.bazel @@ -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", diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go index 2e3301ad8b9d..ed0e64f7aee9 100644 --- a/pkg/sql/sessiondata/search_path.go +++ b/pkg/sql/sessiondata/search_path.go @@ -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. @@ -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 "" }