Skip to content

Commit

Permalink
Merge #37049
Browse files Browse the repository at this point in the history
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 <radu@cockroachlabs.com>
  • Loading branch information
craig[bot] and RaduBerinde committed Apr 26, 2019
2 parents be35db9 + eed3adc commit c1e83c9
Show file tree
Hide file tree
Showing 12 changed files with 121 additions and 67 deletions.
2 changes: 1 addition & 1 deletion pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -343,7 +343,7 @@ func TestAdminAPIDatabaseSQLInjection(t *testing.T) {

const fakedb = "system;DROP DATABASE system;"
const path = "databases/" + fakedb
const errPattern = `database \\"` + fakedb + `\\" does not exist`
const errPattern = `target database or schema does not exist`
if err := getAdminJSONProto(s, path, nil); !testutils.IsError(err, errPattern) {
t.Fatalf("unexpected error: %v\nexpected: %s", err, errPattern)
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/delegate/delegate.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,9 @@ func TryDelegate(
case *tree.ShowConstraints:
return d.delegateShowConstraints(t)

case *tree.ShowGrants:
return d.delegateShowGrants(t)

case *tree.ShowJobs:
return d.delegateShowJobs(t)

Expand Down
54 changes: 23 additions & 31 deletions pkg/sql/show_grants.go → pkg/sql/delegate/show_grants.go
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
// Copyright 2017 The Cockroach Authors.
// Copyright 2019 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
Expand All @@ -12,25 +12,25 @@
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package sql
package delegate

import (
"bytes"
"context"
"fmt"
"strings"

"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

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

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

initCheck = func(ctx context.Context) error {
for _, db := range dbNames {
// Check if the database exists by using the security.RootUser.
if _, err := p.PhysicalSchemaAccessor().GetDatabaseDesc(
ctx, p.txn, db, p.CommonLookupFlags(true /*required*/),
); err != nil {
return err
}
}
return nil
}

for _, db := range dbNames {
name := cat.SchemaName{
CatalogName: tree.Name(db),
SchemaName: tree.Name(tree.PublicSchema),
ExplicitCatalog: true,
ExplicitSchema: true,
}
_, _, err := d.catalog.ResolveSchema(d.ctx, cat.Flags{AvoidDescriptorCaches: true}, &name)
if err != nil {
return nil, err
}
params = append(params, lex.EscapeSQLString(db))
}

Expand All @@ -94,22 +92,17 @@ FROM "".information_schema.table_privileges`
if err != nil {
return nil, err
}
var tables tree.TableNames
// We avoid the cache so that we can observe the grants taking
// a lease, like other SHOW commands.
//
// TODO(vivek): check if the cache can be used.
p.runWithOptions(resolveFlags{skipCache: true}, func() {
tables, err = expandTableGlob(ctx, p.txn, p, tableGlob)
})
tables, err := cat.ExpandDataSourceGlob(
d.ctx, d.catalog, cat.Flags{AvoidDescriptorCaches: true}, tableGlob,
)
if err != nil {
return nil, err
}
allTables = append(allTables, tables...)
}

initCheck = func(ctx context.Context) error { return nil }

for i := range allTables {
params = append(params, fmt.Sprintf("(%s,%s,%s)",
lex.EscapeSQLString(allTables[i].Catalog()),
Expand All @@ -132,8 +125,8 @@ FROM "".information_schema.table_privileges`
source.WriteString(dbPrivQuery)
source.WriteByte(')')
// If the current database is set, restrict the command to it.
if p.CurrentDatabase() != "" {
fmt.Fprintf(&cond, ` WHERE database_name = %s`, lex.EscapeSQLString(p.CurrentDatabase()))
if currDB := d.evalCtx.SessionData.Database; currDB != "" {
fmt.Fprintf(&cond, ` WHERE database_name = %s`, lex.EscapeSQLString(currDB))
} else {
cond.WriteString(`WHERE true`)
}
Expand All @@ -147,7 +140,6 @@ FROM "".information_schema.table_privileges`
}
fmt.Fprintf(&cond, ` AND grantee IN (%s)`, strings.Join(params, ","))
}
return p.delegateQuery(ctx, "SHOW GRANTS",
fmt.Sprintf("SELECT * FROM (%s) %s ORDER BY %s", source.String(), cond.String(), orderBy),
initCheck, nil)
query := fmt.Sprintf("SELECT * FROM (%s) %s ORDER BY %s", source.String(), cond.String(), orderBy)
return parse(query)
}
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/grant_table
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ test public NULL root ALL
statement ok
SET DATABASE = ''

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

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

statement error pgcode 42P01 relation "a.tt" does not exist
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/rename_database
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ SET sql_safe_updates = FALSE;
statement error pgcode 42P01 relation "kv" does not exist
SELECT * FROM kv

statement error database "test" does not exist
statement error target database or schema does not exist
SHOW GRANTS ON DATABASE test

query T
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/opt/cat/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@

package cat

import "context"

// Schema is an interface to a database schema, which is a namespace that
// contains other database objects, like tables and views. Examples of schema
// are "public" and "crdb_internal".
Expand All @@ -25,4 +27,8 @@ type Schema interface {
// and ExplicitSchema fields will always be true, since all parts of the
// name are always specified.
Name() *SchemaName

// GetDataSourceNames returns the list of names for the data sources that the
// schema contains.
GetDataSourceNames(ctx context.Context) ([]DataSourceName, error)
}
49 changes: 49 additions & 0 deletions pkg/sql/opt/cat/utils.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package cat

import (
"context"

"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/pkg/errors"
)

// ExpandDataSourceGlob is a utility function that expands a tree.TablePattern
// into a list of object names.
func ExpandDataSourceGlob(
ctx context.Context, catalog Catalog, flags Flags, pattern tree.TablePattern,
) ([]DataSourceName, error) {

switch p := pattern.(type) {
case *tree.TableName:
_, name, err := catalog.ResolveDataSource(ctx, flags, p)
if err != nil {
return nil, err
}
return []DataSourceName{name}, nil

case *tree.AllTablesSelector:
schema, _, err := catalog.ResolveSchema(ctx, flags, &p.TableNamePrefix)
if err != nil {
return nil, err
}

return schema.GetDataSourceNames(ctx)

default:
return nil, errors.Errorf("invalid TablePattern type %T", p)
}
}
14 changes: 8 additions & 6 deletions pkg/sql/opt/exec/execbuilder/testdata/explain
Original file line number Diff line number Diff line change
Expand Up @@ -250,12 +250,14 @@ render · ·
query TTT
SELECT * FROM [EXPLAIN SHOW GRANTS ON foo] WHERE field != 'size'
----
sort · ·
│ order +database_name,+schema_name,+table_name,+grantee,+privilege_type
└── render · ·
└── filter · ·
│ filter (table_catalog, table_schema, table_name) IN (('test', 'public', 'foo'),)
└── values · ·
render · ·
└── sort · ·
│ order +grantee,+privilege_type
└── render · ·
└── filter · ·
│ filter (table_catalog, table_schema, table_name) IN (('test', 'public', 'foo'),)
└── virtual table · ·
· source ·

query TTT
EXPLAIN SHOW INDEX FROM foo
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/opt/testutils/testcat/test_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -411,6 +411,11 @@ func (s *Schema) Name() *cat.SchemaName {
return &s.SchemaName
}

// GetDataSourceNames is part of the cat.Schema interface.
func (s *Schema) GetDataSourceNames(ctx context.Context) ([]cat.DataSourceName, error) {
panic("not implemented")
}

// View implements the cat.View interface for testing purposes.
type View struct {
ViewID cat.StableID
Expand Down
18 changes: 16 additions & 2 deletions pkg/sql/opt_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,8 @@ func (oc *optCatalog) reset() {
// optSchema is a wrapper around sqlbase.DatabaseDescriptor that implements the
// cat.Object and cat.Schema interfaces.
type optSchema struct {
desc *sqlbase.DatabaseDescriptor
planner *planner
desc *sqlbase.DatabaseDescriptor

name cat.SchemaName
}
Expand All @@ -100,6 +101,15 @@ func (os *optSchema) Name() *cat.SchemaName {
return &os.name
}

// GetDataSourceNames is part of the cat.Schema interface.
func (os *optSchema) GetDataSourceNames(ctx context.Context) ([]cat.DataSourceName, error) {
return GetObjectNames(
ctx, os.planner.Txn(), os.planner, os.desc,
os.name.Schema(),
true, /* explicitPrefix */
)
}

// ResolveSchema is part of the cat.Catalog interface.
func (oc *optCatalog) ResolveSchema(
ctx context.Context, flags cat.Flags, name *cat.SchemaName,
Expand Down Expand Up @@ -131,7 +141,11 @@ func (oc *optCatalog) ResolveSchema(
return nil, cat.SchemaName{}, pgerror.Newf(pgerror.CodeInvalidSchemaNameError,
"target database or schema does not exist")
}
return &optSchema{desc: desc.(*DatabaseDescriptor)}, oc.tn.TableNamePrefix, nil
return &optSchema{
planner: oc.planner,
desc: desc.(*DatabaseDescriptor),
name: oc.tn.TableNamePrefix,
}, oc.tn.TableNamePrefix, nil
}

// ResolveDataSource is part of the cat.Catalog interface.
Expand Down
4 changes: 0 additions & 4 deletions pkg/sql/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -693,8 +693,6 @@ func (p *planner) newPlan(
return p.ShowClusterSetting(ctx, n)
case *tree.ShowVar:
return p.ShowVar(ctx, n)
case *tree.ShowGrants:
return p.ShowGrants(ctx, n)
case *tree.ShowHistogram:
return p.ShowHistogram(ctx, n)
case *tree.ShowRoles:
Expand Down Expand Up @@ -808,8 +806,6 @@ func (p *planner) doPrepare(ctx context.Context, stmt tree.Statement) (planNode,
return p.ShowClusterSetting(ctx, n)
case *tree.ShowVar:
return p.ShowVar(ctx, n)
case *tree.ShowGrants:
return p.ShowGrants(ctx, n)
case *tree.ShowRoles:
return p.ShowRoles(ctx, n)
case *tree.ShowTables:
Expand Down
27 changes: 7 additions & 20 deletions pkg/sql/resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/row"
Expand Down Expand Up @@ -320,7 +321,7 @@ func getDescriptorsFromTargetList(
if err != nil {
return nil, err
}
tableNames, err := expandTableGlob(ctx, p.txn, p, tableGlob)
tableNames, err := expandTableGlob(ctx, p, tableGlob)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -531,27 +532,13 @@ func (p *planner) getTableAndIndex(
// expandTableGlob expands pattern into a list of tables represented
// as a tree.TableNames.
func expandTableGlob(
ctx context.Context, txn *client.Txn, sc SchemaResolver, pattern tree.TablePattern,
ctx context.Context, p *planner, pattern tree.TablePattern,
) (tree.TableNames, error) {
if t, ok := pattern.(*tree.TableName); ok {
_, err := ResolveExistingObject(ctx, sc, t, true /*required*/, anyDescType)
if err != nil {
return nil, err
}
return tree.TableNames{*t}, nil
}

glob := pattern.(*tree.AllTablesSelector)
found, descI, err := glob.TableNamePrefix.Resolve(
ctx, sc, sc.CurrentDatabase(), sc.CurrentSearchPath())
if err != nil {
return nil, err
}
if !found {
return nil, sqlbase.NewInvalidWildcardError(tree.ErrString(glob))
}
var catalog optCatalog
catalog.init(p)
catalog.reset()

return GetObjectNames(ctx, txn, sc, descI.(*DatabaseDescriptor), glob.Schema(), glob.ExplicitSchema)
return cat.ExpandDataSourceGlob(ctx, &catalog, cat.Flags{}, pattern)
}

// fkSelfResolver is a SchemaResolver that inserts itself between a
Expand Down

0 comments on commit c1e83c9

Please sign in to comment.