Skip to content

Commit

Permalink
opt: add remaining plan nodes as opaque operators
Browse files Browse the repository at this point in the history
Allow all misc plan nodes to be created through the optimizer via an
Opaque operator. The code for `Grant/Revoke` needed some reworking
because it did its work during planning (!) instead of execution.

Release note: None
  • Loading branch information
RaduBerinde committed Aug 6, 2019
1 parent e3e370a commit 06581b3
Show file tree
Hide file tree
Showing 21 changed files with 341 additions and 114 deletions.
79 changes: 48 additions & 31 deletions pkg/server/updates_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -828,41 +828,58 @@ func TestReportUsage(t *testing.T) {
// Let's ignore all internal queries for this test.
continue
}
foundKeys = append(foundKeys,
fmt.Sprintf("[%v,%v,%v] %s", s.Key.Opt, s.Key.DistSQL, s.Key.Failed, s.Key.Query))
var tags []string
if s.Key.Opt {
tags = append(tags, "opt")
}

if s.Key.DistSQL {
tags = append(tags, "dist")
} else {
tags = append(tags, "nodist")
}

if s.Key.Failed {
tags = append(tags, "failed")
} else {
tags = append(tags, "ok")
}

foundKeys = append(foundKeys, fmt.Sprintf("[%s] %s", strings.Join(tags, ","), s.Key.Query))
}
sort.Strings(foundKeys)
expectedKeys := []string{
`[false,false,false] ALTER DATABASE _ CONFIGURE ZONE = _`,
`[false,false,false] ALTER TABLE _ CONFIGURE ZONE = _`,
`[false,false,false] CREATE DATABASE _`,
`[false,false,false] SET CLUSTER SETTING "cluster.organization" = _`,
`[false,false,false] SET CLUSTER SETTING "diagnostics.reporting.send_crash_reports" = _`,
`[false,false,false] SET CLUSTER SETTING "server.time_until_store_dead" = _`,
`[false,false,false] SET application_name = $1`,
`[false,false,false] SET application_name = DEFAULT`,
`[false,false,false] SET application_name = _`,
`[true,false,false] CREATE TABLE _ (_ INT8 NOT NULL DEFAULT unique_rowid())`,
`[true,false,false] CREATE TABLE _ (_ INT8, CONSTRAINT _ CHECK (_ > _))`,
`[true,false,false] INSERT INTO _ SELECT unnest(ARRAY[_, _, __more2__])`,
`[true,false,false] INSERT INTO _ VALUES (_), (__more2__)`,
`[true,false,false] INSERT INTO _ VALUES (length($1::STRING)), (__more1__)`,
`[true,false,false] INSERT INTO _(_, _) VALUES (_, _)`,
`[true,false,false] SELECT (_, _, __more2__) = (SELECT _, _, _, _ FROM _ LIMIT _)`,
`[true,false,false] SELECT _ FROM (VALUES (_)) AS _ (_) WHERE EXISTS (SELECT * FROM (VALUES (_)) AS _ (_) WHERE _._ = _._)`,
"[true,false,false] SELECT _::STRING::INET, _::JSONB - _, ARRAY (SELECT _)[_]",
`[true,false,false] UPDATE _ SET _ = _ + _`,
"[true,false,false] WITH _ AS (SELECT _) SELECT * FROM _",
`[true,false,true] CREATE TABLE _ (_ INT8 PRIMARY KEY, _ INT8, INDEX (_) INTERLEAVE IN PARENT _ (_))`,
`[true,false,true] SELECT _ / $1`,
`[true,false,true] SELECT _ / _`,
`[true,false,true] SELECT crdb_internal.force_assertion_error(_)`,
`[true,false,true] SELECT crdb_internal.force_error(_, $1)`,
`[true,false,true] SELECT crdb_internal.set_vmodule(_)`,
`[true,true,false] SELECT * FROM _ WHERE (_ = _) AND (_ = _)`,
`[true,true,false] SELECT * FROM _ WHERE (_ = length($1::STRING)) OR (_ = $2)`,
`[true,true,false] SELECT _ FROM _ WHERE (_ = _) AND (lower(_) = lower(_))`,
`[opt,nodist,ok] ALTER DATABASE _ CONFIGURE ZONE = _`,
`[opt,nodist,ok] ALTER TABLE _ CONFIGURE ZONE = _`,
`[opt,nodist,ok] CREATE DATABASE _`,
`[opt,nodist,ok] SET CLUSTER SETTING "cluster.organization" = _`,
`[opt,nodist,ok] SET CLUSTER SETTING "diagnostics.reporting.send_crash_reports" = _`,
`[opt,nodist,ok] SET CLUSTER SETTING "server.time_until_store_dead" = _`,
`[opt,nodist,ok] SET application_name = $1`,
`[opt,nodist,ok] SET application_name = DEFAULT`,
`[opt,nodist,ok] SET application_name = _`,
`[opt,nodist,ok] CREATE TABLE _ (_ INT8 NOT NULL DEFAULT unique_rowid())`,
`[opt,nodist,ok] CREATE TABLE _ (_ INT8, CONSTRAINT _ CHECK (_ > _))`,
`[opt,nodist,ok] INSERT INTO _ SELECT unnest(ARRAY[_, _, __more2__])`,
`[opt,nodist,ok] INSERT INTO _ VALUES (_), (__more2__)`,
`[opt,nodist,ok] INSERT INTO _ VALUES (length($1::STRING)), (__more1__)`,
`[opt,nodist,ok] INSERT INTO _(_, _) VALUES (_, _)`,
`[opt,nodist,ok] SELECT (_, _, __more2__) = (SELECT _, _, _, _ FROM _ LIMIT _)`,
`[opt,nodist,ok] SELECT _ FROM (VALUES (_)) AS _ (_) WHERE EXISTS (SELECT * FROM (VALUES (_)) AS _ (_) WHERE _._ = _._)`,
"[opt,nodist,ok] SELECT _::STRING::INET, _::JSONB - _, ARRAY (SELECT _)[_]",
`[opt,nodist,ok] UPDATE _ SET _ = _ + _`,
"[opt,nodist,ok] WITH _ AS (SELECT _) SELECT * FROM _",
`[opt,nodist,failed] CREATE TABLE _ (_ INT8 PRIMARY KEY, _ INT8, INDEX (_) INTERLEAVE IN PARENT _ (_))`,
`[opt,nodist,failed] SELECT _ / $1`,
`[opt,nodist,failed] SELECT _ / _`,
`[opt,nodist,failed] SELECT crdb_internal.force_assertion_error(_)`,
`[opt,nodist,failed] SELECT crdb_internal.force_error(_, $1)`,
`[opt,nodist,failed] SELECT crdb_internal.set_vmodule(_)`,
`[opt,dist,ok] SELECT * FROM _ WHERE (_ = _) AND (_ = _)`,
`[opt,dist,ok] SELECT * FROM _ WHERE (_ = length($1::STRING)) OR (_ = $2)`,
`[opt,dist,ok] SELECT _ FROM _ WHERE (_ = _) AND (lower(_) = lower(_))`,
}
sort.Strings(expectedKeys)
t.Logf("expected:\n%s\ngot:\n%s", pretty.Sprint(expectedKeys), pretty.Sprint(foundKeys))
for i, found := range foundKeys {
if i >= len(expectedKeys) {
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -819,6 +819,14 @@ func canFallbackFromOpt(err error, optMode sessiondata.OptimizerMode, stmt *Stat
// We only fallback on "feature not supported" errors.
return false
}
if err.Error() == "unimplemented: schema change statement cannot follow a statement that has written in the same transaction" {
// This is a special error generated when SetSystemConfigTrigger fails. If
// we fall back to the heuristic planner, the second call to that method
// succeeds.
// TODO(radu): this will go away very soon when we remove fallback
// altogether.
return false
}

if optMode == sessiondata.OptimizerAlways {
// In Always mode we never fallback, with one exception: SET commands (or
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/expand_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -354,6 +354,7 @@ func doExpandPlan(
case *alterTableNode:
case *alterSequenceNode:
case *alterUserSetPasswordNode:
case *changePrivilegesNode:
case *commentOnColumnNode:
case *commentOnDatabaseNode:
case *commentOnTableNode:
Expand Down Expand Up @@ -877,6 +878,7 @@ func (p *planner) simplifyOrderings(plan planNode, usefulOrdering sqlbase.Column
case *alterTableNode:
case *alterSequenceNode:
case *alterUserSetPasswordNode:
case *changePrivilegesNode:
case *commentOnColumnNode:
case *commentOnDatabaseNode:
case *commentOnTableNode:
Expand Down
66 changes: 39 additions & 27 deletions pkg/sql/grant_revoke.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,13 @@ import (
// Notes: postgres requires the object owner.
// mysql requires the "grant option" and the same privileges, and sometimes superuser.
func (p *planner) Grant(ctx context.Context, n *tree.Grant) (planNode, error) {
return p.changePrivileges(ctx, n.Targets, n.Grantees, func(privDesc *sqlbase.PrivilegeDescriptor, grantee string) {
privDesc.Grant(grantee, n.Privileges)
})
return &changePrivilegesNode{
targets: n.Targets,
grantees: n.Grantees,
changePrivilege: func(privDesc *sqlbase.PrivilegeDescriptor, grantee string) {
privDesc.Grant(grantee, n.Privileges)
},
}, nil
}

// Revoke removes privileges from users.
Expand All @@ -44,65 +48,72 @@ func (p *planner) Grant(ctx context.Context, n *tree.Grant) (planNode, error) {
// Notes: postgres requires the object owner.
// mysql requires the "grant option" and the same privileges, and sometimes superuser.
func (p *planner) Revoke(ctx context.Context, n *tree.Revoke) (planNode, error) {
return p.changePrivileges(ctx, n.Targets, n.Grantees, func(privDesc *sqlbase.PrivilegeDescriptor, grantee string) {
privDesc.Revoke(grantee, n.Privileges)
})
return &changePrivilegesNode{
targets: n.Targets,
grantees: n.Grantees,
changePrivilege: func(privDesc *sqlbase.PrivilegeDescriptor, grantee string) {
privDesc.Revoke(grantee, n.Privileges)
},
}, nil
}

type changePrivilegesNode struct {
targets tree.TargetList
grantees tree.NameList
changePrivilege func(*sqlbase.PrivilegeDescriptor, string)
}

func (p *planner) changePrivileges(
ctx context.Context,
targets tree.TargetList,
grantees tree.NameList,
changePrivilege func(*sqlbase.PrivilegeDescriptor, string),
) (planNode, error) {
func (n *changePrivilegesNode) startExec(params runParams) error {
ctx := params.ctx
p := params.p
// Check whether grantees exists
users, err := p.GetAllUsersAndRoles(ctx)
if err != nil {
return nil, err
return err
}

// We're allowed to grant/revoke privileges to/from the "public" role even though
// it does not exist: add it to the list of all users and roles.
users[sqlbase.PublicRole] = true // isRole

for _, grantee := range grantees {
for _, grantee := range n.grantees {
if _, ok := users[string(grantee)]; !ok {
return nil, errors.Errorf("user or role %s does not exist", &grantee)
return errors.Errorf("user or role %s does not exist", &grantee)
}
}

var descriptors []sqlbase.DescriptorProto
// DDL statements avoid the cache to avoid leases, and can view non-public descriptors.
// TODO(vivek): check if the cache can be used.
p.runWithOptions(resolveFlags{skipCache: true}, func() {
descriptors, err = getDescriptorsFromTargetList(ctx, p, targets)
descriptors, err = getDescriptorsFromTargetList(ctx, p, n.targets)
})
if err != nil {
return nil, err
return err
}

// First, update the descriptors. We want to catch all errors before
// we update them in KV below.
b := p.txn.NewBatch()
for _, descriptor := range descriptors {
if err := p.CheckPrivilege(ctx, descriptor, privilege.GRANT); err != nil {
return nil, err
return err
}
privileges := descriptor.GetPrivileges()
for _, grantee := range grantees {
changePrivilege(privileges, string(grantee))
for _, grantee := range n.grantees {
n.changePrivilege(privileges, string(grantee))
}

// Validate privilege descriptors directly as the db/table level Validate
// may fix up the descriptor.
if err := privileges.Validate(descriptor.GetID()); err != nil {
return nil, err
return err
}

switch d := descriptor.(type) {
case *sqlbase.DatabaseDescriptor:
if err := d.Validate(); err != nil {
return nil, err
return err
}
writeDescToBatch(ctx, p.extendedEvalCtx.Tracing.KVTracingEnabled(), p.execCfg.Settings,
b, descriptor.GetID(), descriptor)
Expand All @@ -111,15 +122,16 @@ func (p *planner) changePrivileges(
if !d.Dropped() {
if err := p.writeSchemaChangeToBatch(
ctx, d, sqlbase.InvalidMutationID, b); err != nil {
return nil, err
return err
}
}
}
}

// Now update the descriptors transactionally.
if err := p.txn.Run(ctx, b); err != nil {
return nil, err
}
return newZeroNode(nil /* columns */), nil
return p.txn.Run(ctx, b)
}

func (*changePrivilegesNode) Next(runParams) (bool, error) { return false, nil }
func (*changePrivilegesNode) Values() tree.Datums { return tree.Datums{} }
func (*changePrivilegesNode) Close(context.Context) {}
5 changes: 0 additions & 5 deletions pkg/sql/logictest/testdata/logic_test/backup

This file was deleted.

9 changes: 9 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/ccl
Original file line number Diff line number Diff line change
Expand Up @@ -19,3 +19,12 @@ GRANT foo TO testuser

statement error pgcode XXC01 a CCL binary is required to use this statement type: \*tree\.RevokeRole
REVOKE foo FROM testuser

statement error pgcode XXC01 a CCL binary is required to use this statement type: \*tree\.CreateChangefeed
CREATE CHANGEFEED FOR foo

query error pgcode XXC01 a CCL binary is required to use this statement type
BACKUP DATABASE foo TO '/bar' INCREMENTAL FROM '/baz'

query error pgcode XXC01 a CCL binary is required to use this statement type
RESTORE DATABASE foo FROM '/bar'
6 changes: 0 additions & 6 deletions pkg/sql/logictest/testdata/logic_test/optimizer
Original file line number Diff line number Diff line change
Expand Up @@ -78,12 +78,6 @@ SELECT * FROM tview
3 30
4 40

statement ok
SET OPTIMIZER = ALWAYS

query error pq: unimplemented: unsupported statement: \*tree\.AlterTable
ALTER TABLE test DROP COLUMN v;

statement ok
SET OPTIMIZER = LOCAL

Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/statement_statistics
Original file line number Diff line number Diff line change
Expand Up @@ -123,10 +123,10 @@ SELECT x FROM test WHERE y IN (_, _, _ + x, _, _) ·
SELECT x FROM test WHERE y IN (_, _, __more3__) ·
SELECT x FROM test WHERE y IN (_, _, __more3__) +
SELECT x FROM test WHERE y NOT IN (_, _, __more3__) ·
SET CLUSTER SETTING "debug.panic_on_failed_assertions" = DEFAULT -
SET CLUSTER SETTING "debug.panic_on_failed_assertions" = _ -
SET application_name = _ -
SET distsql = "on" -
SET CLUSTER SETTING "debug.panic_on_failed_assertions" = DEFAULT ·
SET CLUSTER SETTING "debug.panic_on_failed_assertions" = _ ·
SET application_name = _ ·
SET distsql = "on" ·
SHOW CLUSTER SETTING "debug.panic_on_failed_assertions" ·
SHOW application_name ·

Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/metric_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ func TestQueryCounts(t *testing.T) {

var testcases = []queryCounter{
// The counts are deltas for each query.
{query: "SET OPTIMIZER = 'off'", miscCount: 1, miscExecutedCount: 1, fallbackCount: 1},
{query: "SET OPTIMIZER = 'off'", miscCount: 1, miscExecutedCount: 1, optCount: 1},
{query: "SET DISTSQL = 'off'", miscCount: 1, miscExecutedCount: 1},
{query: "BEGIN; END", txnBeginCount: 1, txnCommitCount: 1},
{query: "SELECT 1", selectCount: 1, selectExecutedCount: 1, txnCommitCount: 1},
Expand Down Expand Up @@ -105,7 +105,7 @@ func TestQueryCounts(t *testing.T) {
{query: "SELECT 3", selectCount: 1, selectExecutedCount: 1, optCount: 1},
{query: "CREATE TABLE mt.n (num INTEGER PRIMARY KEY)", ddlCount: 1, optCount: 1},
{query: "UPDATE mt.n SET num = num + 1", updateCount: 1, optCount: 1},
{query: "SET OPTIMIZER = 'off'", miscCount: 1, miscExecutedCount: 1, fallbackCount: 1},
{query: "SET OPTIMIZER = 'off'", miscCount: 1, miscExecutedCount: 1, optCount: 1},
}

accum := initializeQueryCounter(s)
Expand Down
Loading

0 comments on commit 06581b3

Please sign in to comment.