Skip to content

Commit c1e83c9

Browse files
craig[bot]RaduBerinde
craig[bot]
andcommitted
Merge #37049
37049: opt: support SHOW GRANTS r=RaduBerinde a=RaduBerinde The first commit rewrites `expandTableGlob` to use the catalog. The second commit moves the ShowGrants implementation to `delegate`. Co-authored-by: Radu Berinde <[email protected]>
2 parents be35db9 + eed3adc commit c1e83c9

File tree

12 files changed

+121
-67
lines changed

12 files changed

+121
-67
lines changed

pkg/server/admin_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -343,7 +343,7 @@ func TestAdminAPIDatabaseSQLInjection(t *testing.T) {
343343

344344
const fakedb = "system;DROP DATABASE system;"
345345
const path = "databases/" + fakedb
346-
const errPattern = `database \\"` + fakedb + `\\" does not exist`
346+
const errPattern = `target database or schema does not exist`
347347
if err := getAdminJSONProto(s, path, nil); !testutils.IsError(err, errPattern) {
348348
t.Fatalf("unexpected error: %v\nexpected: %s", err, errPattern)
349349
}

pkg/sql/delegate/delegate.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,9 @@ func TryDelegate(
5858
case *tree.ShowConstraints:
5959
return d.delegateShowConstraints(t)
6060

61+
case *tree.ShowGrants:
62+
return d.delegateShowGrants(t)
63+
6164
case *tree.ShowJobs:
6265
return d.delegateShowJobs(t)
6366

pkg/sql/show_grants.go renamed to pkg/sql/delegate/show_grants.go

Lines changed: 23 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
// Copyright 2017 The Cockroach Authors.
1+
// Copyright 2019 The Cockroach Authors.
22
//
33
// Licensed under the Apache License, Version 2.0 (the "License");
44
// you may not use this file except in compliance with the License.
@@ -12,25 +12,25 @@
1212
// implied. See the License for the specific language governing
1313
// permissions and limitations under the License.
1414

15-
package sql
15+
package delegate
1616

1717
import (
1818
"bytes"
19-
"context"
2019
"fmt"
2120
"strings"
2221

2322
"github.com/cockroachdb/cockroach/pkg/sql/lex"
23+
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
2424
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
2525
)
2626

27-
// ShowGrants returns grant details for the specified objects and users.
27+
// delegateShowGrants implements SHOW GRANTS which returns grant details for the
28+
// specified objects and users.
2829
// Privileges: None.
2930
// Notes: postgres does not have a SHOW GRANTS statement.
3031
// mysql only returns the user's privileges.
31-
func (p *planner) ShowGrants(ctx context.Context, n *tree.ShowGrants) (planNode, error) {
32+
func (d *delegator) delegateShowGrants(n *tree.ShowGrants) (tree.Statement, error) {
3233
var params []string
33-
var initCheck func(context.Context) error
3434

3535
const dbPrivQuery = `
3636
SELECT table_catalog AS database_name,
@@ -55,19 +55,17 @@ FROM "".information_schema.table_privileges`
5555
// if the type of target is database.
5656
dbNames := n.Targets.Databases.ToStrings()
5757

58-
initCheck = func(ctx context.Context) error {
59-
for _, db := range dbNames {
60-
// Check if the database exists by using the security.RootUser.
61-
if _, err := p.PhysicalSchemaAccessor().GetDatabaseDesc(
62-
ctx, p.txn, db, p.CommonLookupFlags(true /*required*/),
63-
); err != nil {
64-
return err
65-
}
66-
}
67-
return nil
68-
}
69-
7058
for _, db := range dbNames {
59+
name := cat.SchemaName{
60+
CatalogName: tree.Name(db),
61+
SchemaName: tree.Name(tree.PublicSchema),
62+
ExplicitCatalog: true,
63+
ExplicitSchema: true,
64+
}
65+
_, _, err := d.catalog.ResolveSchema(d.ctx, cat.Flags{AvoidDescriptorCaches: true}, &name)
66+
if err != nil {
67+
return nil, err
68+
}
7169
params = append(params, lex.EscapeSQLString(db))
7270
}
7371

@@ -94,22 +92,17 @@ FROM "".information_schema.table_privileges`
9492
if err != nil {
9593
return nil, err
9694
}
97-
var tables tree.TableNames
9895
// We avoid the cache so that we can observe the grants taking
9996
// a lease, like other SHOW commands.
100-
//
101-
// TODO(vivek): check if the cache can be used.
102-
p.runWithOptions(resolveFlags{skipCache: true}, func() {
103-
tables, err = expandTableGlob(ctx, p.txn, p, tableGlob)
104-
})
97+
tables, err := cat.ExpandDataSourceGlob(
98+
d.ctx, d.catalog, cat.Flags{AvoidDescriptorCaches: true}, tableGlob,
99+
)
105100
if err != nil {
106101
return nil, err
107102
}
108103
allTables = append(allTables, tables...)
109104
}
110105

111-
initCheck = func(ctx context.Context) error { return nil }
112-
113106
for i := range allTables {
114107
params = append(params, fmt.Sprintf("(%s,%s,%s)",
115108
lex.EscapeSQLString(allTables[i].Catalog()),
@@ -132,8 +125,8 @@ FROM "".information_schema.table_privileges`
132125
source.WriteString(dbPrivQuery)
133126
source.WriteByte(')')
134127
// If the current database is set, restrict the command to it.
135-
if p.CurrentDatabase() != "" {
136-
fmt.Fprintf(&cond, ` WHERE database_name = %s`, lex.EscapeSQLString(p.CurrentDatabase()))
128+
if currDB := d.evalCtx.SessionData.Database; currDB != "" {
129+
fmt.Fprintf(&cond, ` WHERE database_name = %s`, lex.EscapeSQLString(currDB))
137130
} else {
138131
cond.WriteString(`WHERE true`)
139132
}
@@ -147,7 +140,6 @@ FROM "".information_schema.table_privileges`
147140
}
148141
fmt.Fprintf(&cond, ` AND grantee IN (%s)`, strings.Join(params, ","))
149142
}
150-
return p.delegateQuery(ctx, "SHOW GRANTS",
151-
fmt.Sprintf("SELECT * FROM (%s) %s ORDER BY %s", source.String(), cond.String(), orderBy),
152-
initCheck, nil)
143+
query := fmt.Sprintf("SELECT * FROM (%s) %s ORDER BY %s", source.String(), cond.String(), orderBy)
144+
return parse(query)
153145
}

pkg/sql/logictest/testdata/logic_test/grant_table

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -145,7 +145,7 @@ test public NULL root ALL
145145
statement ok
146146
SET DATABASE = ''
147147

148-
query TTTTT colnames
148+
query TTTTT colnames,rowsort
149149
SELECT * FROM [SHOW GRANTS]
150150
WHERE schema_name NOT IN ('crdb_internal', 'pg_catalog', 'information_schema')
151151
----
@@ -702,7 +702,7 @@ SET DATABASE = ""
702702
statement error pq: syntax error at or near "@"
703703
GRANT ALL ON a.t@xyz TO readwrite
704704

705-
statement error pq: "\*" does not match any valid database or schema
705+
statement error pq: target database or schema does not exist
706706
GRANT ALL ON * TO readwrite
707707

708708
statement error pgcode 42P01 relation "a.tt" does not exist

pkg/sql/logictest/testdata/logic_test/rename_database

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ SET sql_safe_updates = FALSE;
5050
statement error pgcode 42P01 relation "kv" does not exist
5151
SELECT * FROM kv
5252

53-
statement error database "test" does not exist
53+
statement error target database or schema does not exist
5454
SHOW GRANTS ON DATABASE test
5555

5656
query T

pkg/sql/opt/cat/schema.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,8 @@
1414

1515
package cat
1616

17+
import "context"
18+
1719
// Schema is an interface to a database schema, which is a namespace that
1820
// contains other database objects, like tables and views. Examples of schema
1921
// are "public" and "crdb_internal".
@@ -25,4 +27,8 @@ type Schema interface {
2527
// and ExplicitSchema fields will always be true, since all parts of the
2628
// name are always specified.
2729
Name() *SchemaName
30+
31+
// GetDataSourceNames returns the list of names for the data sources that the
32+
// schema contains.
33+
GetDataSourceNames(ctx context.Context) ([]DataSourceName, error)
2834
}

pkg/sql/opt/cat/utils.go

Lines changed: 49 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,49 @@
1+
// Copyright 2018 The Cockroach Authors.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
12+
// implied. See the License for the specific language governing
13+
// permissions and limitations under the License.
14+
15+
package cat
16+
17+
import (
18+
"context"
19+
20+
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
21+
"github.com/pkg/errors"
22+
)
23+
24+
// ExpandDataSourceGlob is a utility function that expands a tree.TablePattern
25+
// into a list of object names.
26+
func ExpandDataSourceGlob(
27+
ctx context.Context, catalog Catalog, flags Flags, pattern tree.TablePattern,
28+
) ([]DataSourceName, error) {
29+
30+
switch p := pattern.(type) {
31+
case *tree.TableName:
32+
_, name, err := catalog.ResolveDataSource(ctx, flags, p)
33+
if err != nil {
34+
return nil, err
35+
}
36+
return []DataSourceName{name}, nil
37+
38+
case *tree.AllTablesSelector:
39+
schema, _, err := catalog.ResolveSchema(ctx, flags, &p.TableNamePrefix)
40+
if err != nil {
41+
return nil, err
42+
}
43+
44+
return schema.GetDataSourceNames(ctx)
45+
46+
default:
47+
return nil, errors.Errorf("invalid TablePattern type %T", p)
48+
}
49+
}

pkg/sql/opt/exec/execbuilder/testdata/explain

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -250,12 +250,14 @@ render · ·
250250
query TTT
251251
SELECT * FROM [EXPLAIN SHOW GRANTS ON foo] WHERE field != 'size'
252252
----
253-
sort · ·
254-
│ order +database_name,+schema_name,+table_name,+grantee,+privilege_type
255-
└── render · ·
256-
└── filter · ·
257-
│ filter (table_catalog, table_schema, table_name) IN (('test', 'public', 'foo'),)
258-
└── values · ·
253+
render · ·
254+
└── sort · ·
255+
│ order +grantee,+privilege_type
256+
└── render · ·
257+
└── filter · ·
258+
│ filter (table_catalog, table_schema, table_name) IN (('test', 'public', 'foo'),)
259+
└── virtual table · ·
260+
· source ·
259261

260262
query TTT
261263
EXPLAIN SHOW INDEX FROM foo

pkg/sql/opt/testutils/testcat/test_catalog.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -411,6 +411,11 @@ func (s *Schema) Name() *cat.SchemaName {
411411
return &s.SchemaName
412412
}
413413

414+
// GetDataSourceNames is part of the cat.Schema interface.
415+
func (s *Schema) GetDataSourceNames(ctx context.Context) ([]cat.DataSourceName, error) {
416+
panic("not implemented")
417+
}
418+
414419
// View implements the cat.View interface for testing purposes.
415420
type View struct {
416421
ViewID cat.StableID

pkg/sql/opt_catalog.go

Lines changed: 16 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -79,7 +79,8 @@ func (oc *optCatalog) reset() {
7979
// optSchema is a wrapper around sqlbase.DatabaseDescriptor that implements the
8080
// cat.Object and cat.Schema interfaces.
8181
type optSchema struct {
82-
desc *sqlbase.DatabaseDescriptor
82+
planner *planner
83+
desc *sqlbase.DatabaseDescriptor
8384

8485
name cat.SchemaName
8586
}
@@ -100,6 +101,15 @@ func (os *optSchema) Name() *cat.SchemaName {
100101
return &os.name
101102
}
102103

104+
// GetDataSourceNames is part of the cat.Schema interface.
105+
func (os *optSchema) GetDataSourceNames(ctx context.Context) ([]cat.DataSourceName, error) {
106+
return GetObjectNames(
107+
ctx, os.planner.Txn(), os.planner, os.desc,
108+
os.name.Schema(),
109+
true, /* explicitPrefix */
110+
)
111+
}
112+
103113
// ResolveSchema is part of the cat.Catalog interface.
104114
func (oc *optCatalog) ResolveSchema(
105115
ctx context.Context, flags cat.Flags, name *cat.SchemaName,
@@ -131,7 +141,11 @@ func (oc *optCatalog) ResolveSchema(
131141
return nil, cat.SchemaName{}, pgerror.Newf(pgerror.CodeInvalidSchemaNameError,
132142
"target database or schema does not exist")
133143
}
134-
return &optSchema{desc: desc.(*DatabaseDescriptor)}, oc.tn.TableNamePrefix, nil
144+
return &optSchema{
145+
planner: oc.planner,
146+
desc: desc.(*DatabaseDescriptor),
147+
name: oc.tn.TableNamePrefix,
148+
}, oc.tn.TableNamePrefix, nil
135149
}
136150

137151
// ResolveDataSource is part of the cat.Catalog interface.

pkg/sql/plan.go

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -693,8 +693,6 @@ func (p *planner) newPlan(
693693
return p.ShowClusterSetting(ctx, n)
694694
case *tree.ShowVar:
695695
return p.ShowVar(ctx, n)
696-
case *tree.ShowGrants:
697-
return p.ShowGrants(ctx, n)
698696
case *tree.ShowHistogram:
699697
return p.ShowHistogram(ctx, n)
700698
case *tree.ShowRoles:
@@ -808,8 +806,6 @@ func (p *planner) doPrepare(ctx context.Context, stmt tree.Statement) (planNode,
808806
return p.ShowClusterSetting(ctx, n)
809807
case *tree.ShowVar:
810808
return p.ShowVar(ctx, n)
811-
case *tree.ShowGrants:
812-
return p.ShowGrants(ctx, n)
813809
case *tree.ShowRoles:
814810
return p.ShowRoles(ctx, n)
815811
case *tree.ShowTables:

pkg/sql/resolver.go

Lines changed: 7 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ import (
1919
"fmt"
2020

2121
"github.com/cockroachdb/cockroach/pkg/internal/client"
22+
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
2223
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
2324
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
2425
"github.com/cockroachdb/cockroach/pkg/sql/row"
@@ -320,7 +321,7 @@ func getDescriptorsFromTargetList(
320321
if err != nil {
321322
return nil, err
322323
}
323-
tableNames, err := expandTableGlob(ctx, p.txn, p, tableGlob)
324+
tableNames, err := expandTableGlob(ctx, p, tableGlob)
324325
if err != nil {
325326
return nil, err
326327
}
@@ -531,27 +532,13 @@ func (p *planner) getTableAndIndex(
531532
// expandTableGlob expands pattern into a list of tables represented
532533
// as a tree.TableNames.
533534
func expandTableGlob(
534-
ctx context.Context, txn *client.Txn, sc SchemaResolver, pattern tree.TablePattern,
535+
ctx context.Context, p *planner, pattern tree.TablePattern,
535536
) (tree.TableNames, error) {
536-
if t, ok := pattern.(*tree.TableName); ok {
537-
_, err := ResolveExistingObject(ctx, sc, t, true /*required*/, anyDescType)
538-
if err != nil {
539-
return nil, err
540-
}
541-
return tree.TableNames{*t}, nil
542-
}
543-
544-
glob := pattern.(*tree.AllTablesSelector)
545-
found, descI, err := glob.TableNamePrefix.Resolve(
546-
ctx, sc, sc.CurrentDatabase(), sc.CurrentSearchPath())
547-
if err != nil {
548-
return nil, err
549-
}
550-
if !found {
551-
return nil, sqlbase.NewInvalidWildcardError(tree.ErrString(glob))
552-
}
537+
var catalog optCatalog
538+
catalog.init(p)
539+
catalog.reset()
553540

554-
return GetObjectNames(ctx, txn, sc, descI.(*DatabaseDescriptor), glob.Schema(), glob.ExplicitSchema)
541+
return cat.ExpandDataSourceGlob(ctx, &catalog, cat.Flags{}, pattern)
555542
}
556543

557544
// fkSelfResolver is a SchemaResolver that inserts itself between a

0 commit comments

Comments
 (0)