Skip to content

Commit

Permalink
Merge #24817 #24842
Browse files Browse the repository at this point in the history
24817: cherrypick-2.0: sql: fix qualified index name resolution r=knz a=knz

Picks #24778.
cc @cockroachdb/release 

24842: cherry-pick 2.0: sql: use a reusable name resolution algo for star expansions r=knz a=knz

Picks #24811.
cc @cockroachdb/release 

Co-authored-by: Raphael 'kena' Poss <knz@cockroachlabs.com>
  • Loading branch information
craig[bot] and knz committed Apr 16, 2018
3 parents c92bc6e + 1ab5831 + 2f12b68 commit 024e598
Show file tree
Hide file tree
Showing 8 changed files with 193 additions and 41 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/partitionccl/zone_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -220,11 +220,11 @@ func TestInvalidIndexPartitionSetShowZones(t *testing.T) {
}{
{
"ALTER INDEX foo EXPERIMENTAL CONFIGURE ZONE ''",
`no database specified: "foo"`,
`no schema has been selected to search index: "foo"`,
},
{
"EXPERIMENTAL SHOW ZONE CONFIGURATION FOR INDEX foo",
`no database specified: "foo"`,
`no schema has been selected to search index: "foo"`,
},
{
"USE system; ALTER INDEX foo EXPERIMENTAL CONFIGURE ZONE ''",
Expand Down
10 changes: 1 addition & 9 deletions pkg/sql/data_source.go
Original file line number Diff line number Diff line change
Expand Up @@ -506,19 +506,11 @@ func expandStar(
}
}
case *tree.AllColumnsSelector:
tn, err := tree.NormalizeTableName(&sel.TableName)
if err != nil {
return nil, nil, err
}
resolver := sqlbase.ColumnResolver{Sources: src}
numRes, _, _, err := resolver.FindSourceMatchingName(ctx, tn)
_, _, err := sel.Resolve(ctx, &resolver)
if err != nil {
return nil, nil, err
}
if numRes == tree.NoResults {
return nil, nil, pgerror.NewErrorf(pgerror.CodeUndefinedColumnError,
"no data source named %q", tree.ErrString(&tn))
}
ds := src[resolver.ResolverState.SrcIdx]
colSet := ds.SourceAliases[resolver.ResolverState.ColSetIdx].ColumnSet
for i, ok := colSet.Next(0); ok; i, ok = colSet.Next(i + 1) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/insert
Original file line number Diff line number Diff line change
Expand Up @@ -511,7 +511,7 @@ INSERT INTO return AS r VALUES (5, 6)
statement ok
INSERT INTO return VALUES (5, 6) RETURNING test.return.a

statement error no data source named "x"
statement error no data source matches pattern: x.\*
INSERT INTO return VALUES (1, 2) RETURNING x.*[1]

statement error column name "x" not found
Expand Down
51 changes: 44 additions & 7 deletions pkg/sql/logictest/testdata/logic_test/namespace
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,15 @@ t
# Of course one can also list the tables in "public" by making it the
# current database.
statement ok
SET DATABASE = public
SET database = public

query T
SHOW TABLES
----
t

statement ok
SET DATABASE = test; DROP DATABASE public
SET database = test; DROP DATABASE public

# Unqualified pg_type resolves from pg_catalog.
query T
Expand All @@ -40,7 +40,7 @@ date

# Override table and check name resolves properly.
statement ok
SET SEARCH_PATH=public,pg_catalog
SET search_path=public,pg_catalog

statement ok
CREATE TABLE pg_type(x INT); INSERT INTO pg_type VALUES(42)
Expand All @@ -53,23 +53,60 @@ SELECT x FROM pg_type
# Leave database, check name resolves to default.
# The expected error can only occur on the virtual pg_type, not the physical one.
query error no database specified
SET DATABASE = ''; SELECT * FROM pg_type
SET database = ''; SELECT * FROM pg_type

# Go to different database, check name still resolves to default.
query T
CREATE DATABASE foo; SET DATABASE = foo; SELECT typname FROM pg_type WHERE typname = 'date'
CREATE DATABASE foo; SET database = foo; SELECT typname FROM pg_type WHERE typname = 'date'
----
date

# Verify that pg_catalog at the beginning of the search path takes precedence.
query T
SET DATABASE = test; SET SEARCH_PATH = pg_catalog,public; SELECT typname FROM pg_type WHERE typname = 'date'
SET database = test; SET search_path = pg_catalog,public; SELECT typname FROM pg_type WHERE typname = 'date'
----
date

# Now set the search path to the testdb, placing pg_catalog explicitly
# at the end.
query I
SET SEARCH_PATH = public,pg_catalog; SELECT x FROM pg_type
SET search_path = public,pg_catalog; SELECT x FROM pg_type
----
42

statement ok
DROP TABLE pg_type; RESET search_path; SET database = test

# Unqualified index name resolution.
statement ok
ALTER INDEX "primary" RENAME TO a_pk

# Schema-qualified index name resolution.
statement ok
ALTER INDEX public.a_pk RENAME TO a_pk2

# DB-qualified index name resolution (CRDB 1.x compat).
statement ok
ALTER INDEX test.a_pk2 RENAME TO a_pk3

statement ok
CREATE DATABASE public; CREATE TABLE public.public.t(a INT)

# We can't see the DB "public" with DB-qualified index name resolution.
statement error index "primary" does not exist
ALTER INDEX public."primary" RENAME TO t_pk

# But we can see it with sufficient qualification.
statement ok
ALTER INDEX public.public."primary" RENAME TO t_pk

# If the search path is invalid, we get a special error.
statement ok
SET search_path = invalid

statement error no schema has been selected to search index: "a_pk3"
ALTER INDEX a_pk3 RENAME TO a_pk4

# But qualification resolves the problem.
statement ok
ALTER INDEX public.a_pk3 RENAME TO a_pk4
25 changes: 23 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/select
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,28 @@ SELECT kv.* FROM kv
----
a NULL

query error no data source named "foo"
# Regression tests for #24169
query TT
SELECT test.kv.* FROM kv
----
a NULL

query TT
SELECT test.public.kv.* FROM kv
----
a NULL

query TT
SELECT test.public.kv.* FROM test.kv
----
a NULL

query TT
SELECT test.kv.* FROM test.public.kv
----
a NULL

query error no data source matches pattern: foo.\*
SELECT foo.* FROM kv

query error cannot use "\*" without a FROM clause
Expand All @@ -156,7 +177,7 @@ SELECT 1, * FROM nocols
query error "kv.*" cannot be aliased
SELECT kv.* AS foo FROM kv

query error no data source named "bar.kv"
query error no data source matches pattern: bar.kv.\*
SELECT bar.kv.* FROM kv

# Don't panic with invalid names (#8024)
Expand Down
51 changes: 31 additions & 20 deletions pkg/sql/resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -290,23 +290,26 @@ func (p *planner) getQualifiedTableName(
return tbName.String(), nil
}

// findTableContainingIndex returns the name of the table containing an
// index of the given name and its descriptor.
// findTableContainingIndex returns the descriptor of a table
// containing the index of the given name.
// This is used by expandIndexName().
//
// An error is returned if the index name is ambiguous (i.e. exists in
// multiple tables). If no table is found and requireTable is true, an
// error will be returned, otherwise the TableName and descriptor
// returned will be nil.
func findTableContainingIndex(
sc SchemaAccessor, dbName string, idxName tree.UnrestrictedName, lookupFlags CommonLookupFlags,
sc SchemaAccessor,
dbName, scName string,
idxName tree.UnrestrictedName,
lookupFlags CommonLookupFlags,
) (result *tree.TableName, desc *sqlbase.TableDescriptor, err error) {
dbDesc, err := sc.GetDatabaseDesc(dbName, lookupFlags)
if dbDesc == nil || err != nil {
return nil, nil, err
}

tns, err := sc.GetObjectNames(dbDesc, tree.PublicSchema,
tns, err := sc.GetObjectNames(dbDesc, scName,
DatabaseListFlags{CommonLookupFlags: lookupFlags, explicitPrefix: true})
if err != nil {
return nil, nil, err
Expand Down Expand Up @@ -377,32 +380,40 @@ func expandIndexName(
// subsequent call to expandIndexName() will generate tn using the
// new value of index.Table, which is a table name.

// Just an assertion: if we got there, there cannot be a path prefix
// in tn or a value in index.Index yet.
if tn.ExplicitSchema || tn.ExplicitCatalog || index.Index != "" {
// Just an assertion: if we got there, there cannot be a value in index.Index yet.
if index.Index != "" {
return nil, nil, pgerror.NewErrorf(pgerror.CodeInternalError,
"programmer error: not-searched index name found already qualified: %s@%s", tn, index.Index)
}

curDb := sc.CurrentDatabase()
if curDb == "" {
return nil, nil, pgerror.NewErrorf(pgerror.CodeUndefinedObjectError,
"no database specified: %q", tree.ErrString(index))
index.Index = tree.UnrestrictedName(tn.TableName)

// Look up the table prefix.
found, _, err := tn.TableNamePrefix.Resolve(ctx, sc, sc.CurrentDatabase(), sc.CurrentSearchPath())
if err != nil {
return nil, nil, err
}
if !found {
if requireTable {
return nil, nil, pgerror.NewErrorf(pgerror.CodeUndefinedObjectError,
"no schema has been selected to search index: %q",
tree.ErrString(&index.Index)).SetHintf(
"check the current database and search_path are valid")
}
return nil, nil, nil
}

index.Index = tree.UnrestrictedName(tn.TableName)
lookupFlags := sc.CommonLookupFlags(ctx, requireTable)
tn, desc, err = findTableContainingIndex(
sc.LogicalSchemaAccessor(), curDb, index.Index, lookupFlags)
var foundTn *tree.TableName
foundTn, desc, err = findTableContainingIndex(
sc.LogicalSchemaAccessor(), tn.Catalog(), tn.Schema(), index.Index, lookupFlags)
if err != nil {
return nil, nil, err
} else if tn == nil {
// NB: tn is nil here if and only if requireTable is
// false, otherwise err would be non-nil.
return nil, nil, nil
} else if foundTn != nil {
// Memoize the table name that was found. tn is a reference to the table name
// stored in index.Table.
*tn = *foundTn
}
// Memoize the resolved table name in case expandIndexName() is called again.
index.Table.TableNameReference = tn
}
return tn, desc, nil
}
Expand Down
30 changes: 30 additions & 0 deletions pkg/sql/sem/tree/name_resolution.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,36 @@ type ColumnResolutionResult interface {
ColumnResolutionResult()
}

// Resolve performs name resolution for a qualified star using a resolver.
func (a *AllColumnsSelector) Resolve(
ctx context.Context, r ColumnItemResolver,
) (srcName *TableName, srcMeta ColumnSourceMeta, err error) {
prefix := makeTableNameFromUnresolvedName(&a.TableName)

// Is there a data source with this prefix?
var res NumResolutionResults
res, srcName, srcMeta, err = r.FindSourceMatchingName(ctx, prefix)
if err != nil {
return nil, nil, err
}
if res == NoResults && a.TableName.NumParts == 2 {
// No, but name of form db.tbl.*?
// Special rule for compatibility with CockroachDB v1.x:
// search name db.public.tbl.* instead.
prefix.ExplicitCatalog = true
prefix.CatalogName = prefix.SchemaName
prefix.SchemaName = PublicSchemaName
res, srcName, srcMeta, err = r.FindSourceMatchingName(ctx, prefix)
if err != nil {
return nil, nil, err
}
}
if res == NoResults {
return nil, nil, newSourceNotFoundError("no data source matches pattern: %s", a)
}
return srcName, srcMeta, nil
}

// Resolve performs name resolution for a column item using a resolver.
func (c *ColumnItem) Resolve(
ctx context.Context, r ColumnItemResolver,
Expand Down
61 changes: 61 additions & 0 deletions pkg/sql/sem/tree/name_resolution_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -390,6 +390,67 @@ func newFakeSource() *fakeSource {
}
}

func TestResolveQualifiedStar(t *testing.T) {
testCases := []struct {
in string
tnout string
csout string
err string
}{
{`a.*`, ``, ``, `no data source matches pattern: a.*`},
{`foo.*`, ``, ``, `ambiguous table name: foo`},
{`db1.public.foo.*`, `db1.public.foo`, `x`, ``},
{`db1.foo.*`, `db1.public.foo`, `x`, ``},
{`dbx.foo.*`, ``, ``, `no data source matches pattern: dbx.foo.*`},
{`kv.*`, `db1.public.kv`, `k, v`, ``},
}
fakeFrom := newFakeSource()
for _, tc := range testCases {
t.Run(tc.in, func(t *testing.T) {
fakeFrom.t = t
tnout, csout, err := func() (string, string, error) {
stmt, err := parser.ParseOne(fmt.Sprintf("SELECT %s", tc.in))
if err != nil {
return "", "", err
}
v := stmt.(*tree.Select).Select.(*tree.SelectClause).Exprs[0].Expr.(tree.VarName)
c, err := v.NormalizeVarName()
if err != nil {
return "", "", err
}
acs, ok := c.(*tree.AllColumnsSelector)
if !ok {
return "", "", fmt.Errorf("var name %s (%T) did not resolve to AllColumnsSelector, found %T instead",
v, v, c)
}
tn, res, err := acs.Resolve(context.Background(), fakeFrom)
if err != nil {
return "", "", err
}
cs, ok := res.(colsRes)
if !ok {
return "", "", fmt.Errorf("fake resolver did not return colsRes, found %T instead", res)
}
nl := tree.NameList(cs)
return tn.String(), nl.String(), nil
}()
if !testutils.IsError(err, tc.err) {
t.Fatalf("%s: expected %s, but found %v", tc.in, tc.err, err)
}
if tc.err != "" {
return
}

if tc.tnout != tnout {
t.Fatalf("%s: expected tn %s, but found %s", tc.in, tc.tnout, tnout)
}
if tc.csout != csout {
t.Fatalf("%s: expected cs %s, but found %s", tc.in, tc.csout, csout)
}
})
}
}

func TestResolveColumnItem(t *testing.T) {
testCases := []struct {
in string
Expand Down

0 comments on commit 024e598

Please sign in to comment.