Skip to content

Commit

Permalink
plpgsql: allow nested blocks in a block with an exception handler
Browse files Browse the repository at this point in the history
This commit removes the restriction on nesting PL/pgSQL blocks within
a block that has an exception handler. This is accomplished by tracking
the number of variables that are in scope for each block, and using that
information to determine which arguments to supply to the exception
handler for a block. This relies on the invariant that the variables of
a parent block always form a prefix of the variables of a child block.

Fixes #118551

Release note (sql change): PL/pgSQL blocks can now be nested in a block
that has an exception handler.
  • Loading branch information
DrewKimball committed Mar 11, 2024
1 parent ccb5b6f commit dc19dfc
Show file tree
Hide file tree
Showing 5 changed files with 138 additions and 58 deletions.
50 changes: 31 additions & 19 deletions pkg/ccl/logictestccl/testdata/logic_test/plpgsql_block
Original file line number Diff line number Diff line change
Expand Up @@ -405,6 +405,37 @@ NOTICE: outer handler
NOTICE: inner block
NOTICE: inner handler 100

# A block can be nested inside another block that has an exception handler.
statement ok
DROP PROCEDURE p;
CREATE PROCEDURE p() AS $$
DECLARE
x INT := 0;
BEGIN
RAISE NOTICE 'outer block: %', x;
BEGIN
x := x + 1;
RAISE NOTICE 'inner block: %', x;
SELECT 1 // 0;
EXCEPTION WHEN division_by_zero THEN
x := x + 1;
RAISE NOTICE 'inner handler: %', x;
SELECT 1 // 0;
END;
EXCEPTION WHEN division_by_zero THEN
x := x + 1;
RAISE NOTICE 'outer handler: %', x;
END
$$ LANGUAGE PLpgSQL;

query T noticetrace
CALL p();
----
NOTICE: outer block: 0
NOTICE: inner block: 1
NOTICE: inner handler: 2
NOTICE: outer handler: 3

subtest error

statement ok
Expand Down Expand Up @@ -457,25 +488,6 @@ CREATE PROCEDURE p() AS $$
END
$$ LANGUAGE PLpgSQL;

# A block cannot currently be nested in a block with an exception handler
# (tracked in #118551).
statement error pgcode 0A000 pq: unimplemented: PL/pgSQL blocks cannot yet be nested within a block that has an exception handler
CREATE PROCEDURE p() AS $$
DECLARE
x INT := 0;
BEGIN
RAISE NOTICE '%', x;
DECLARE
y INT := 1 // x;
BEGIN
RAISE NOTICE '% %', x, y;
END;
EXCEPTION
WHEN division_by_zero THEN
RAISE NOTICE 'oops!';
END
$$ LANGUAGE PLpgSQL;

# Regression test for the internal error in #119492.
subtest regression_119492

Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/opt/exec/execbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -953,8 +953,10 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ
}
}

if udf.Def.BlockState != nil {
b.initRoutineExceptionHandler(udf.Def.BlockState, udf.Def.ExceptionBlock)
blockState := udf.Def.BlockState
if blockState != nil {
blockState.VariableCount = len(udf.Def.Params)
b.initRoutineExceptionHandler(blockState, udf.Def.ExceptionBlock)
}

// Create a tree.RoutinePlanFn that can plan the statements in the UDF body.
Expand Down Expand Up @@ -984,7 +986,7 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ
udf.Def.SetReturning,
udf.TailCall,
false, /* procedure */
udf.Def.BlockState,
blockState,
udf.Def.CursorDeclaration,
), nil
}
Expand All @@ -995,8 +997,6 @@ func (b *Builder) initRoutineExceptionHandler(
blockState *tree.BlockState, exceptionBlock *memo.ExceptionBlock,
) {
if exceptionBlock == nil {
// Building the exception block is currently the only necessary
// initialization.
return
}
exceptionHandler := &tree.RoutineExceptionHandler{
Expand Down
30 changes: 21 additions & 9 deletions pkg/sql/opt/optbuilder/plpgsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,9 @@ type plBlock struct {
label string

// vars is an ordered list of variables declared in a PL/pgSQL block.
//
// INVARIANT: the variables of a parent (ancestor) block *always* form a
// prefix of the variables of a child (descendant) block.
vars []ast.Variable

// varTypes maps from the name of each variable in the scope to its type.
Expand Down Expand Up @@ -280,11 +283,6 @@ func (b *plpgsqlBuilder) buildBlock(astBlock *ast.Block, s *scope) *scope {
if astBlock.Label != "" {
panic(blockLabelErr)
}
if b.block().hasExceptionHandler {
// The parent block has an exception handler. Exception handlers and nested
// blocks are not yet compatible.
panic(nestedBlockExceptionErr)
}
b.ensureScopeHasExpr(s)
block := b.pushBlock(plBlock{
label: astBlock.Label,
Expand All @@ -294,6 +292,15 @@ func (b *plpgsqlBuilder) buildBlock(astBlock *ast.Block, s *scope) *scope {
cursors: make(map[ast.Variable]ast.CursorDeclaration),
})
defer b.popBlock()
if len(astBlock.Exceptions) > 0 || b.hasExceptionHandler() {
// If the current block or some ancestor block has an exception handler, it
// is necessary to maintain the BlockState with a reference to the parent
// BlockState (if any).
block.state = &tree.BlockState{}
if parent := b.parentBlock(); parent != nil {
block.state.Parent = parent.state
}
}
// First, handle the variable declarations.
for i := range astBlock.Decls {
switch dec := astBlock.Decls[i].(type) {
Expand Down Expand Up @@ -351,7 +358,6 @@ func (b *plpgsqlBuilder) buildBlock(astBlock *ast.Block, s *scope) *scope {
// The routine is volatile to prevent inlining. Only the block and
// variable-assignment routines need to be volatile; see the buildExceptions
// comment for details.
block.state = &tree.BlockState{}
block.hasExceptionHandler = true
blockCon := b.makeContinuation("exception_block")
blockCon.def.ExceptionBlock = exceptions
Expand Down Expand Up @@ -1681,6 +1687,15 @@ func (b *plpgsqlBuilder) block() *plBlock {
return &b.blocks[len(b.blocks)-1]
}

// parentBlock returns the parent block for the current PL/pgSQL block. It
// returns nil if the current block does not have a parent.
func (b *plpgsqlBuilder) parentBlock() *plBlock {
if len(b.blocks) <= 1 {
return nil
}
return &b.blocks[len(b.blocks)-2]
}

// pushBlock puts the given block on the stack. It is used when entering the
// scope of a PL/pgSQL block.
func (b *plpgsqlBuilder) pushBlock(bs plBlock) *plBlock {
Expand Down Expand Up @@ -1841,9 +1856,6 @@ var (
nonCompositeErr = pgerror.New(pgcode.DatatypeMismatch,
"cannot return non-composite value from function returning composite type",
)
nestedBlockExceptionErr = unimplemented.New("exception handler for nested blocks",
"PL/pgSQL blocks cannot yet be nested within a block that has an exception handler",
)
returnWithOUTParameterErr = pgerror.New(pgcode.DatatypeMismatch,
"RETURN cannot have a parameter in function with OUT parameters",
)
Expand Down
75 changes: 50 additions & 25 deletions pkg/sql/routine.go
Original file line number Diff line number Diff line change
Expand Up @@ -306,47 +306,72 @@ func (g *routineGenerator) startInternal(ctx context.Context, txn *kv.Txn) (err
// in which case the savepoint will be rolled back either by a parent PLpgSQL
// block (if the error is eventually caught), or when the transaction aborts.
func (g *routineGenerator) handleException(ctx context.Context, err error) error {
blockState := g.expr.BlockState
if err == nil || blockState == nil || blockState.ExceptionHandler == nil {
return err
}
caughtCode := pgerror.GetPGCode(err)
if caughtCode == pgcode.Uncategorized {
// It is not safe to catch an uncategorized error.
return err
}
if !g.p.Txn().CanUseSavepoint(ctx, blockState.SavepointTok.(kv.SavepointToken)) {
// The current transaction state does not allow roll-back.
// TODO(111446): some retryable errors allow the transaction to be rolled
// back partially (e.g. for read committed). We should be able to take
// advantage of that mechanism here as well.
return err
}
// Unset the exception handler to indicate that it has already encountered an
// error.
exceptionHandler := blockState.ExceptionHandler
blockState.ExceptionHandler = nil
for i, code := range exceptionHandler.Codes {
caughtException := code == caughtCode
if code.String() == "OTHERS" {
// The special OTHERS condition matches any error code apart from
// query_canceled and assert_failure (though they can still be caught
// explicitly).
caughtException = caughtCode != pgcode.QueryCanceled && caughtCode != pgcode.AssertFailure
// Attempt to catch the error, starting with the exception handler for the
// current block, and propagating the error up to ancestor exception handlers
// if necessary.
for blockState := g.expr.BlockState; blockState != nil; blockState = blockState.Parent {
if blockState.ExceptionHandler == nil {
// This block has no exception handler.
continue
}
if caughtException {
if !g.p.Txn().CanUseSavepoint(ctx, blockState.SavepointTok.(kv.SavepointToken)) {
// The current transaction state does not allow roll-back.
// TODO(111446): some retryable errors allow the transaction to be rolled
// back partially (e.g. for read committed). We should be able to take
// advantage of that mechanism here as well.
return err
}
// Unset the exception handler to indicate that it has already encountered an
// error.
exceptionHandler := blockState.ExceptionHandler
blockState.ExceptionHandler = nil
var branch *tree.RoutineExpr
for i, code := range exceptionHandler.Codes {
caughtException := code == caughtCode
if code.String() == "OTHERS" {
// The special OTHERS condition matches any error code apart from
// query_canceled and assert_failure (though they can still be caught
// explicitly).
caughtException = caughtCode != pgcode.QueryCanceled && caughtCode != pgcode.AssertFailure
}
if caughtException {
branch = exceptionHandler.Actions[i]
break
}
}
if branch != nil {
cursErr := g.closeCursors(blockState)
if cursErr != nil {
// This error is unexpected, so return immediately.
return errors.CombineErrors(err, cursErr)
}
spErr := g.p.Txn().RollbackToSavepoint(ctx, blockState.SavepointTok.(kv.SavepointToken))
if spErr != nil {
// This error is unexpected, so return immediately.
return errors.CombineErrors(err, spErr)
}
g.reset(ctx, g.p, exceptionHandler.Actions[i], g.args)
return g.startInternal(ctx, g.p.Txn())
// Truncate the arguments using the number of variables in scope for the
// current block. This is necessary because the error may originate from
// a child block, but propagate up to a parent block. See the BlockState
// comments for further details.
args := g.args[:blockState.VariableCount]
g.reset(ctx, g.p, branch, args)

// If handling the exception results in another error, that error can in
// turn be caught by a parent exception handler. Otherwise, the exception
// was handled, so just return.
err = g.startInternal(ctx, g.p.Txn())
if err == nil {
return nil
}
}
}
// We reached the end of the exception handlers without handling this error.
return err
}

Expand Down
31 changes: 31 additions & 0 deletions pkg/sql/sem/tree/routine.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,6 +221,37 @@ type RoutineOpenCursor struct {
// BlockState is shared state between all routines that make up a PLpgSQL block.
// It allows for coordination between the routines for exception handling.
type BlockState struct {
// Parent is a pointer to this block's parent, if any. Note that this refers
// to a parent within the same routine; nested routine calls currently do not
// interact with one another directly (e.g. through TCO, see #119956).
Parent *BlockState

// VariableCount tracks the number of variables that are in scope for this
// block, so that the correct arguments can be supplied to an exception
// handler when an error originates from a "descendant" block. Example:
//
// DECLARE
// x INT := 0;
// BEGIN
// DECLARE
// y INT := 1;
// BEGIN
// y = 1 // 0;
// END;
// EXCEPTION WHEN division_by_zero THEN
// RETURN 0;
// END
//
// In this example, the error is thrown from the inner block, where variables
// "x" and "y" are both in scope. Therefore, we will have access to values for
// both variables. However, the error will be caught by the outer block, for
// which only "x" is in scope. Therefore, the outer block must truncate the
// values before supplying them to its exception handler as arguments.
//
// NOTE: the list of variables in an outer block *always* form a prefix of the
// variables in a nested block.
VariableCount int

// ExceptionHandler is the exception handler for the current block, if any.
ExceptionHandler *RoutineExceptionHandler

Expand Down

0 comments on commit dc19dfc

Please sign in to comment.