Skip to content

Commit

Permalink
sql,*: rework plan hooks to type check separately from execution
Browse files Browse the repository at this point in the history
Prior to this change, plan hooks would be invokved twice, once during prepare
and again during execution. At prepare time, the placeholder values are not
known, but, critically, the placeholders were being type-checked. This type
checking informs the sql layer how to interpret the placeholder values when
we are trying to bind them. Importantly, the sql layer does not ever use the
`planNode` produced via the `PlanHookRowFn` returned from the planning
invocation.

The above dance meant that these plan hooks had quite a bit of complexity to
eagerly type check but lazily evaluate expressions passed in via the statement.
This complexity became a problem more recently: one cannot and should not
access many planner data structures from the row func because such functions
can, theoretically, run concurrently and these data structures are not thread
safe.

Note that the above problem is not commonly a real problem. Most plan hook
statements don't permit concurrency and those than do, via, say, `UNION ALL`
don't commonly get executed in such a setting. In some ways, the real reason
for this change is to unwind some unneeded complexity and clarify the bounds
of what one can do in a plan hook.

The big change which unearthed the problem is that we now support privileges on
external URLs. These URLs may be provided via placeholders or expressions. If
we couldn't access the expressions until we were in the row fn, then we'd
be at risk of performing privilege checking concurrently, which is not safe.

One thing that this commit does is introduce a new package `exprutil` with
helpers for type checking. It is so named because I'd like to also remove the
methods from the `PlanHookState` which we now use for expr evaluation and place
them in this new package.

Fixes #88385

Release note: None
  • Loading branch information
ajwerner committed Oct 21, 2022
1 parent 78ac7cf commit 7351c0a
Show file tree
Hide file tree
Showing 42 changed files with 1,334 additions and 992 deletions.
5 changes: 5 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,7 @@ ALL_TESTS = [
"//pkg/sql/execinfrapb:execinfrapb_test",
"//pkg/sql/execstats:execstats_disallowed_imports_test",
"//pkg/sql/execstats:execstats_test",
"//pkg/sql/exprutil:exprutil_test",
"//pkg/sql/flowinfra:flowinfra_disallowed_imports_test",
"//pkg/sql/flowinfra:flowinfra_test",
"//pkg/sql/gcjob/gcjobnotifier:gcjobnotifier_test",
Expand Down Expand Up @@ -1464,6 +1465,9 @@ GO_TARGETS = [
"//pkg/sql/execinfrapb:execinfrapb_test",
"//pkg/sql/execstats:execstats",
"//pkg/sql/execstats:execstats_test",
"//pkg/sql/exprutil:evalexpr",
"//pkg/sql/exprutil:exprutil",
"//pkg/sql/exprutil:exprutil_test",
"//pkg/sql/faketreeeval:faketreeeval",
"//pkg/sql/flowinfra:flowinfra",
"//pkg/sql/flowinfra:flowinfra_test",
Expand Down Expand Up @@ -2619,6 +2623,7 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/execinfra/execreleasable:get_x_data",
"//pkg/sql/execinfrapb:get_x_data",
"//pkg/sql/execstats:get_x_data",
"//pkg/sql/exprutil:get_x_data",
"//pkg/sql/faketreeeval:get_x_data",
"//pkg/sql/flowinfra:get_x_data",
"//pkg/sql/gcjob:get_x_data",
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ go_library(
"//pkg/sql/doctor",
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/exprutil",
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
Expand Down
62 changes: 33 additions & 29 deletions pkg/ccl/backupccl/alter_backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,30 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/exprutil"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/errors"
)

func alterBackupTypeCheck(
ctx context.Context, stmt tree.Statement, p sql.PlanHookState,
) (ok bool, _ colinfo.ResultColumns, _ error) {
alterBackupStmt, ok := stmt.(*tree.AlterBackup)
if !ok {
return false, nil, nil
}
if err := exprutil.TypeCheck(
ctx, "ALTER BACKUP", p.SemaCtx(),
exprutil.Strings{
alterBackupStmt.Backup,
alterBackupStmt.Subdir,
},
); err != nil {
return false, nil, err
}
return true, nil, nil
}

func alterBackupPlanHook(
ctx context.Context, stmt tree.Statement, p sql.PlanHookState,
) (sql.PlanHookRowFn, colinfo.ResultColumns, []sql.PlanNode, bool, error) {
Expand All @@ -41,46 +61,38 @@ func alterBackupPlanHook(
return nil, nil, nil, false, err
}

fromFn, err := p.TypeAsString(ctx, alterBackupStmt.Backup, "ALTER BACKUP")
exprEval := p.ExprEvaluator("ALTER BACKUP")
backup, err := exprEval.String(ctx, alterBackupStmt.Backup)
if err != nil {
return nil, nil, nil, false, err
}

subdirFn := func() (string, error) { return "", nil }
var subdir string
if alterBackupStmt.Subdir != nil {
subdirFn, err = p.TypeAsString(ctx, alterBackupStmt.Subdir, "ALTER BACKUP")
subdir, err = exprEval.String(ctx, alterBackupStmt.Subdir)
if err != nil {
return nil, nil, nil, false, err
}
}

var newKmsFn func() ([]string, error)
var oldKmsFn func() ([]string, error)
var newKms []string
var oldKms []string

for _, cmd := range alterBackupStmt.Cmds {
switch v := cmd.(type) {
case *tree.AlterBackupKMS:
newKmsFn, err = p.TypeAsStringArray(ctx, tree.Exprs(v.KMSInfo.NewKMSURI), "ALTER BACKUP")
newKms, err = exprEval.StringArray(ctx, tree.Exprs(v.KMSInfo.NewKMSURI))
if err != nil {
return nil, nil, nil, false, err
}
oldKmsFn, err = p.TypeAsStringArray(ctx, tree.Exprs(v.KMSInfo.OldKMSURI), "ALTER BACKUP")
oldKms, err = exprEval.StringArray(ctx, tree.Exprs(v.KMSInfo.OldKMSURI))
if err != nil {
return nil, nil, nil, false, err
}
}
}

fn := func(ctx context.Context, _ []sql.PlanNode, resultsCh chan<- tree.Datums) error {
backup, err := fromFn()
if err != nil {
return err
}

subdir, err := subdirFn()
if err != nil {
return err
}

if subdir != "" {
if strings.EqualFold(subdir, "LATEST") {
Expand All @@ -107,18 +119,6 @@ func alterBackupPlanHook(
}
}

var newKms []string
newKms, err = newKmsFn()
if err != nil {
return err
}

var oldKms []string
oldKms, err = oldKmsFn()
if err != nil {
return err
}

return doAlterBackupPlan(ctx, alterBackupStmt, p, backup, newKms, oldKms)
}

Expand Down Expand Up @@ -213,5 +213,9 @@ func doAlterBackupPlan(
}

func init() {
sql.AddPlanHook("alter backup", alterBackupPlanHook)
sql.AddPlanHook(
"alter backup",
alterBackupPlanHook,
alterBackupTypeCheck,
)
}
Loading

0 comments on commit 7351c0a

Please sign in to comment.