Skip to content

Commit

Permalink
opt: add opaque operator flavors for mutations and DDL
Browse files Browse the repository at this point in the history
The opaque operator assumes the worst and is tagged with Mutation and
DDL. This prevents all opaque statements from running in read-only
transactions.

This change splits `OpaqueRel` into three flavors: `OpaqueRel`,
`OpaqueMutation`, and `OpaqueDDL`. The difference is in the operator
tags, which causes different behavior in the execbuilder.

In addition, the opaque metadata `String()` now returns the
`StatementTag` and is used in error messages. This was tested manually
as all opaque operators are read-only so far.

Fixes cockroachdb#39204.

Release note: None
  • Loading branch information
RaduBerinde committed Aug 2, 2019
1 parent 13c802c commit ed733ad
Show file tree
Hide file tree
Showing 9 changed files with 114 additions and 27 deletions.
12 changes: 12 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/show_fingerprints
Original file line number Diff line number Diff line change
Expand Up @@ -110,3 +110,15 @@ query TT
SHOW EXPERIMENTAL_FINGERPRINTS FROM TABLE blocks
----
primary 590700560494856555

# Verify that we can show fingerprints from a read-only transaction (#39204).
statement ok
BEGIN TRANSACTION AS OF SYSTEM TIME '-1us'

query TT
SHOW EXPERIMENTAL_FINGERPRINTS FROM TABLE t
----
primary 1205834892498753533

statement ok
COMMIT
9 changes: 4 additions & 5 deletions pkg/sql/opaque.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package sql
import (
"context"
"reflect"
"strings"

"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder"
Expand Down Expand Up @@ -65,22 +64,22 @@ func buildOpaque(
return nil, nil, err
}
res := &opaqueMetadata{
info: strings.TrimPrefix(reflect.TypeOf(stmt).String(), "*tree."),
info: stmt.StatementTag(),
plan: plan,
}
return res, planColumns(plan), nil
}

func init() {
opaqueStmts := []reflect.Type{
opaqueReadOnlyStmts := []reflect.Type{
reflect.TypeOf(&tree.ShowClusterSetting{}),
reflect.TypeOf(&tree.ShowHistogram{}),
reflect.TypeOf(&tree.ShowTableStats{}),
reflect.TypeOf(&tree.ShowTraceForSession{}),
reflect.TypeOf(&tree.ShowZoneConfig{}),
reflect.TypeOf(&tree.ShowFingerprints{}),
}
for _, t := range opaqueStmts {
optbuilder.RegisterOpaque(t, buildOpaque)
for _, t := range opaqueReadOnlyStmts {
optbuilder.RegisterOpaque(t, optbuilder.OpaqueReadOnly, buildOpaque)
}
}
26 changes: 18 additions & 8 deletions pkg/sql/opt/exec/execbuilder/relational.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,11 +155,9 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) {
}

// Raise error if mutation op is part of a read-only transaction.
if opt.IsMutationOp(e) {
if b.evalCtx.TxnReadOnly {
return execPlan{}, pgerror.Newf(pgcode.ReadOnlySQLTransaction,
"cannot execute %s in a read-only transaction", e.Op().SyntaxTag())
}
if opt.IsMutationOp(e) && b.evalCtx.TxnReadOnly {
return execPlan{}, pgerror.Newf(pgcode.ReadOnlySQLTransaction,
"cannot execute %s in a read-only transaction", b.statementTag(e))
}

// Collect usage telemetry for relational node, if appropriate.
Expand Down Expand Up @@ -261,8 +259,8 @@ func (b *Builder) buildRelational(e memo.RelExpr) (execPlan, error) {
case *memo.ShowTraceForSessionExpr:
ep, err = b.buildShowTrace(t)

case *memo.OpaqueRelExpr:
ep, err = b.buildOpaque(t)
case *memo.OpaqueRelExpr, *memo.OpaqueMutationExpr, *memo.OpaqueDDLExpr:
ep, err = b.buildOpaque(t.Private().(*memo.OpaqueRelPrivate))

case *memo.AlterTableSplitExpr:
ep, err = b.buildAlterTableSplit(t)
Expand Down Expand Up @@ -1778,7 +1776,7 @@ func (b *Builder) applySaveTable(
return input, err
}

func (b *Builder) buildOpaque(opaque *memo.OpaqueRelExpr) (execPlan, error) {
func (b *Builder) buildOpaque(opaque *memo.OpaqueRelPrivate) (execPlan, error) {
node, err := b.factory.ConstructOpaque(opaque.Metadata)
if err != nil {
return execPlan{}, err
Expand Down Expand Up @@ -1902,3 +1900,15 @@ func (b *Builder) buildSortedInput(input execPlan, ordering opt.Ordering) (execP
}
return execPlan{root: node, outputCols: input.outputCols}, nil
}

// statementTag returns a string that can be used in an error message regarding
// the given expression.
func (b *Builder) statementTag(expr memo.RelExpr) string {
switch expr.Op() {
case opt.OpaqueRelOp, opt.OpaqueMutationOp, opt.OpaqueDDLOp:
return expr.Private().(*memo.OpaqueRelPrivate).Metadata.String()

default:
return expr.Op().SyntaxTag()
}
}
7 changes: 4 additions & 3 deletions pkg/sql/opt/memo/expr_format.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,9 +193,10 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) {

case *ScanExpr, *VirtualScanExpr, *IndexJoinExpr, *ShowTraceForSessionExpr,
*InsertExpr, *UpdateExpr, *UpsertExpr, *DeleteExpr, *SequenceSelectExpr,
*WindowExpr, *OpaqueRelExpr, *AlterTableSplitExpr, *AlterTableUnsplitExpr,
*AlterTableUnsplitAllExpr, *AlterTableRelocateExpr, *ControlJobsExpr,
*CancelQueriesExpr, *CancelSessionsExpr:
*WindowExpr, *OpaqueRelExpr, *OpaqueMutationExpr, *OpaqueDDLExpr,
*AlterTableSplitExpr, *AlterTableUnsplitExpr, *AlterTableUnsplitAllExpr,
*AlterTableRelocateExpr, *ControlJobsExpr, *CancelQueriesExpr,
*CancelSessionsExpr:
fmt.Fprintf(f.Buffer, "%v", e.Op())
FormatPrivate(f, e.Private(), required)

Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/opt/memo/logical_props_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -811,6 +811,16 @@ func (b *logicalPropsBuilder) buildOpaqueRelProps(op *OpaqueRelExpr, rel *props.
b.buildBasicProps(op, op.Columns, rel)
}

func (b *logicalPropsBuilder) buildOpaqueMutationProps(
op *OpaqueMutationExpr, rel *props.Relational,
) {
b.buildBasicProps(op, op.Columns, rel)
}

func (b *logicalPropsBuilder) buildOpaqueDDLProps(op *OpaqueDDLExpr, rel *props.Relational) {
b.buildBasicProps(op, op.Columns, rel)
}

func (b *logicalPropsBuilder) buildAlterTableSplitProps(
split *AlterTableSplitExpr, rel *props.Relational,
) {
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/memo/statistics_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,8 @@ func (sb *statisticsBuilder) colStat(colSet opt.ColSet, e RelExpr) *props.Column
case opt.SequenceSelectOp:
return sb.colStatSequenceSelect(colSet, e.(*SequenceSelectExpr))

case opt.ExplainOp, opt.ShowTraceForSessionOp, opt.OpaqueRelOp:
case opt.ExplainOp, opt.ShowTraceForSessionOp,
opt.OpaqueRelOp, opt.OpaqueMutationOp, opt.OpaqueDDLOp:
return sb.colStatUnknown(colSet, e.Relational())

case opt.WithOp:
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/operator.go
Original file line number Diff line number Diff line change
Expand Up @@ -281,8 +281,8 @@ func AggregateIsNullOnEmpty(op Operator) bool {
type OpaqueMetadata interface {
ImplementsOpaqueMetadata()

// String is used when printing optimizer trees and should contain a short
// description of the statement.
// String is a short description used when printing optimizer trees and when
// forming error messages; it should be the SQL statement tag.
String() string
}

Expand Down
16 changes: 15 additions & 1 deletion pkg/sql/opt/ops/statement.opt
Original file line number Diff line number Diff line change
Expand Up @@ -104,11 +104,25 @@ define ShowTracePrivate {
#
# OpaqueRel can produce data and can be used as a data source as part of a
# larger enclosing query.
[Relational, DDL, Mutation]
[Relational]
define OpaqueRel {
_ OpaqueRelPrivate
}

# OpaqueMutation is a variant of OpaqueRel for operators that can mutate data as
# part of the transaction.
[Relational, Mutation]
define OpaqueMutation {
_ OpaqueRelPrivate
}

# OpaqueMutation is a variant of OpaqueRel for operators that cause a schema
# change and cannot be executed following a mutation in the same transaction.
[Relational, Mutation, DDL]
define OpaqueDDL {
_ OpaqueRelPrivate
}

[Private]
define OpaqueRelPrivate {
Columns ColList
Expand Down
54 changes: 47 additions & 7 deletions pkg/sql/opt/optbuilder/opaque.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,34 +18,74 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/errors"
)

// BuildOpaqueFn is a handler for building the metadata for an opaque statement.
type BuildOpaqueFn func(
context.Context, *tree.SemaContext, *tree.EvalContext, tree.Statement,
) (opt.OpaqueMetadata, sqlbase.ResultColumns, error)

// OpaqueType indicates whether an opaque statement can mutate data or change
// schema.
type OpaqueType int

const (
// OpaqueReadOnly is used for statements that do not mutate state as part of
// the transaction, and can be run in read-only transactions.
OpaqueReadOnly OpaqueType = iota

// OpaqueMutation is used for statements that mutate data and cannot be run as
// part of read-only transactions.
OpaqueMutation

// OpaqueDDL is used for statements that change a schema and cannot be
// executed following a mutation in the same transaction.
OpaqueDDL
)

// RegisterOpaque registers an opaque handler for a specific statement type.
func RegisterOpaque(stmtType reflect.Type, fn BuildOpaqueFn) {
opaqueStatements[stmtType] = fn
func RegisterOpaque(stmtType reflect.Type, opaqueType OpaqueType, fn BuildOpaqueFn) {
if _, ok := opaqueStatements[stmtType]; ok {
panic(errors.AssertionFailedf("opaque statement %s already registered", stmtType))
}
opaqueStatements[stmtType] = opaqueStmtInfo{
typ: opaqueType,
buildFn: fn,
}
}

type opaqueStmtInfo struct {
typ OpaqueType
buildFn BuildOpaqueFn
}

var opaqueStatements = make(map[reflect.Type]BuildOpaqueFn)
var opaqueStatements = make(map[reflect.Type]opaqueStmtInfo)

func (b *Builder) tryBuildOpaque(stmt tree.Statement, inScope *scope) (outScope *scope) {
fn, ok := opaqueStatements[reflect.TypeOf(stmt)]
info, ok := opaqueStatements[reflect.TypeOf(stmt)]
if !ok {
return nil
}
obj, cols, err := fn(b.ctx, b.semaCtx, b.evalCtx, stmt)
obj, cols, err := info.buildFn(b.ctx, b.semaCtx, b.evalCtx, stmt)
if err != nil {
panic(err)
}
outScope = inScope.push()
b.synthesizeResultColumns(outScope, cols)
outScope.expr = b.factory.ConstructOpaqueRel(&memo.OpaqueRelPrivate{
private := &memo.OpaqueRelPrivate{
Columns: colsToColList(outScope.cols),
Metadata: obj,
})
}
switch info.typ {
case OpaqueReadOnly:
outScope.expr = b.factory.ConstructOpaqueRel(private)
case OpaqueMutation:
outScope.expr = b.factory.ConstructOpaqueMutation(private)
case OpaqueDDL:
outScope.expr = b.factory.ConstructOpaqueDDL(private)
default:
panic(errors.AssertionFailedf("invalid opaque statement type %d", info.typ))
}
return outScope
}

0 comments on commit ed733ad

Please sign in to comment.