Skip to content

Commit

Permalink
Fix release 18 again (#17069)
Browse files Browse the repository at this point in the history
Signed-off-by: Harshit Gangal <[email protected]>
Signed-off-by: Andres Taylor <[email protected]>
Co-authored-by: Harshit Gangal <[email protected]>
  • Loading branch information
systay and harshit-gangal authored Oct 28, 2024
1 parent c58abd9 commit ed60cc0
Show file tree
Hide file tree
Showing 15 changed files with 181 additions and 83 deletions.
59 changes: 29 additions & 30 deletions go/vt/vtgate/planbuilder/operator_transformers.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,9 +79,8 @@ func transformFkCascade(ctx *plancontext.PlanningContext, fkc *operators.FkCasca
return nil, nil
}

// Once we have the parent logical plan, we can create the selection logical plan and the primitives for the children operators.
// For all of these, we don't need the semTable anymore. We set it to nil, to avoid using an incorrect one.
ctx.SemTable = nil
// Using the correct semantics.SemTable for the selection query created during planning.
ctx.SemTable = fkc.SSemTable
selLP, err := transformToLogicalPlan(ctx, fkc.Selection)
if err != nil {
return nil, err
Expand All @@ -90,6 +89,8 @@ func transformFkCascade(ctx *plancontext.PlanningContext, fkc *operators.FkCasca
// Go over the children and convert them to Primitives too.
var children []*engine.FkChild
for _, child := range fkc.Children {
// Using the correct semantics.SemTable for the child table cascade query created during planning.
ctx.SemTable = child.ST
childLP, err := transformToLogicalPlan(ctx, child.Op)
if err != nil {
return nil, err
Expand All @@ -109,6 +110,31 @@ func transformFkCascade(ctx *plancontext.PlanningContext, fkc *operators.FkCasca
return newFkCascade(parentLP, selLP, children), nil
}

// transformFkVerify transforms a FkVerify operator into a logical plan.
func transformFkVerify(ctx *plancontext.PlanningContext, fkv *operators.FkVerify) (logicalPlan, error) {
inputLP, err := transformToLogicalPlan(ctx, fkv.Input)
if err != nil {
return nil, err
}

// Go over the children and convert them to Primitives too.
var verify []*verifyLP
for _, v := range fkv.Verify {
// Using the correct semantics.SemTable for the parent table verify query created during planning.
ctx.SemTable = v.ST
lp, err := transformToLogicalPlan(ctx, v.Op)
if err != nil {
return nil, err
}
verify = append(verify, &verifyLP{
verify: lp,
typ: v.Typ,
})
}

return newFkVerify(inputLP, verify), nil
}

func transformSubQuery(ctx *plancontext.PlanningContext, op *operators.SubQuery) (logicalPlan, error) {
outer, err := transformToLogicalPlan(ctx, op.Outer)
if err != nil {
Expand Down Expand Up @@ -136,33 +162,6 @@ func transformSubQuery(ctx *plancontext.PlanningContext, op *operators.SubQuery)
return newSemiJoin(outer, inner, op.Vars, lhsCols), nil
}

// transformFkVerify transforms a FkVerify operator into a logical plan.
func transformFkVerify(ctx *plancontext.PlanningContext, fkv *operators.FkVerify) (logicalPlan, error) {
inputLP, err := transformToLogicalPlan(ctx, fkv.Input)
if err != nil {
return nil, err
}

// Once we have the input logical plan, we can create the primitives for the verification operators.
// For all of these, we don't need the semTable anymore. We set it to nil, to avoid using an incorrect one.
ctx.SemTable = nil

// Go over the children and convert them to Primitives too.
var verify []*verifyLP
for _, v := range fkv.Verify {
lp, err := transformToLogicalPlan(ctx, v.Op)
if err != nil {
return nil, err
}
verify = append(verify, &verifyLP{
verify: lp,
typ: v.Typ,
})
}

return newFkVerify(inputLP, verify), nil
}

func transformAggregator(ctx *plancontext.PlanningContext, op *operators.Aggregator) (logicalPlan, error) {
plan, err := transformToLogicalPlan(ctx, op.Source)
if err != nil {
Expand Down
25 changes: 13 additions & 12 deletions go/vt/vtgate/planbuilder/operators/SQL_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ func (qb *queryBuilder) addTableExpr(
}

func (qb *queryBuilder) addPredicate(expr sqlparser.Expr) {
if _, toBeSkipped := qb.ctx.SkipPredicates[expr]; toBeSkipped {
if qb.ctx.ShouldSkip(expr) {
// This is a predicate that was added to the RHS of an ApplyJoin.
// The original predicate will be added, so we don't have to add this here
return
Expand Down Expand Up @@ -564,23 +564,24 @@ func buildProjection(op *Projection, qb *queryBuilder) error {
}

func buildApplyJoin(op *ApplyJoin, qb *queryBuilder) error {
predicates := slice.Map(op.JoinPredicates, func(jc JoinColumn) sqlparser.Expr {
predicates, err := slice.MapWithError(op.JoinPredicates, func(jc JoinColumn) (sqlparser.Expr, error) {
// since we are adding these join predicates, we need to mark to broken up version (RHSExpr) of it as done
qb.ctx.SkipPredicates[jc.RHSExpr] = nil

return jc.Original.Expr
err := qb.ctx.SkipJoinPredicates(jc.Original.Expr)
if err != nil {
return nil, err
}
return jc.Original.Expr, nil
})
pred := sqlparser.AndExpressions(predicates...)
err := buildQuery(op.LHS, qb)
if err != nil {
return err
}
// If we are going to add the predicate used in join here
// We should not add the predicate's copy of when it was split into
// two parts. To avoid this, we use the SkipPredicates map.
for _, pred := range op.JoinPredicates {
qb.ctx.SkipPredicates[pred.RHSExpr] = nil

pred := sqlparser.AndExpressions(predicates...)
err = buildQuery(op.LHS, qb)
if err != nil {
return err
}

qbR := &queryBuilder{ctx: qb.ctx}
err = buildQuery(op.RHS, qbR)
if err != nil {
Expand Down
15 changes: 11 additions & 4 deletions go/vt/vtgate/planbuilder/operators/ast_to_op.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,16 +204,23 @@ func createOperatorFromUnion(ctx *plancontext.PlanningContext, node *sqlparser.U
// createOpFromStmt creates an operator from the given statement. It takes in two additional arguments—
// 1. verifyAllFKs: For this given statement, do we need to verify validity of all the foreign keys on the vtgate level.
// 2. fkToIgnore: The foreign key constraint to specifically ignore while planning the statement.
func createOpFromStmt(ctx *plancontext.PlanningContext, stmt sqlparser.Statement, verifyAllFKs bool, fkToIgnore string) (ops.Operator, error) {
func createOpFromStmt(ctx *plancontext.PlanningContext, stmt sqlparser.Statement, verifyAllFKs bool, fkToIgnore string) (*semantics.SemTable, ops.Operator, error) {
newCtx, err := plancontext.CreatePlanningContext(stmt, ctx.ReservedVars, ctx.VSchema, ctx.PlannerVersion)
if err != nil {
return nil, err
return nil, nil, err
}

newCtx.VerifyAllFKs = verifyAllFKs
newCtx.ParentFKToIgnore = fkToIgnore

return PlanQuery(newCtx, stmt)
query, err := PlanQuery(newCtx, stmt)
if err != nil {
return nil, nil, err
}

ctx.KeepPredicateInfo(newCtx)

return newCtx.SemTable, query, err
}

func getOperatorFromTableExpr(ctx *plancontext.PlanningContext, tableExpr sqlparser.TableExpr, onlyTable bool) (ops.Operator, error) {
Expand Down Expand Up @@ -377,7 +384,7 @@ func createSelectionOp(
where *sqlparser.Where,
limit *sqlparser.Limit,
lock sqlparser.Lock,
) (ops.Operator, error) {
) (*semantics.SemTable, ops.Operator, error) {
selectionStmt := &sqlparser.Select{
SelectExprs: selectExprs,
From: tableExprs,
Expand Down
7 changes: 5 additions & 2 deletions go/vt/vtgate/planbuilder/operators/delete.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,7 +195,7 @@ func createFkCascadeOpForDelete(ctx *plancontext.PlanningContext, parentOp ops.O
}
fkChildren = append(fkChildren, fkChild)
}
selectionOp, err := createSelectionOp(ctx, selectExprs, delStmt.TableExprs, delStmt.Where, nil, sqlparser.ForUpdateLock)
st, selectionOp, err := createSelectionOp(ctx, selectExprs, delStmt.TableExprs, delStmt.Where, nil, sqlparser.ForUpdateLock)
if err != nil {
return nil, err
}
Expand All @@ -204,6 +204,8 @@ func createFkCascadeOpForDelete(ctx *plancontext.PlanningContext, parentOp ops.O
Selection: selectionOp,
Children: fkChildren,
Parent: parentOp,

SSemTable: st,
}, nil
}

Expand Down Expand Up @@ -247,7 +249,7 @@ func createFkChildForDelete(ctx *plancontext.PlanningContext, fk vindexes.ChildF
}

// For the child statement of a DELETE query, we don't need to verify all the FKs on VTgate or ignore any foreign key explicitly.
childOp, err := createOpFromStmt(ctx, childStmt, false /* verifyAllFKs */, "" /* fkToIgnore */)
cST, childOp, err := createOpFromStmt(ctx, childStmt, false /* verifyAllFKs */, "" /* fkToIgnore */)
if err != nil {
return nil, err
}
Expand All @@ -256,5 +258,6 @@ func createFkChildForDelete(ctx *plancontext.PlanningContext, fk vindexes.ChildF
BVName: bvName,
Cols: cols,
Op: childOp,
ST: cST,
}, nil
}
5 changes: 1 addition & 4 deletions go/vt/vtgate/planbuilder/operators/expressions.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,10 +52,7 @@ func BreakExpressionInLHSandRHS(
cursor.Replace(arg)
}, nil).(sqlparser.Expr)

if err != nil {
return JoinColumn{}, err
}
ctx.JoinPredicates[expr] = append(ctx.JoinPredicates[expr], rewrittenExpr)
ctx.AddJoinPredicates(expr, rewrittenExpr)
col.RHSExpr = rewrittenExpr
return
}
5 changes: 5 additions & 0 deletions go/vt/vtgate/planbuilder/operators/fk_cascade.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,15 @@ import (
"slices"

"vitess.io/vitess/go/vt/vtgate/planbuilder/operators/ops"
"vitess.io/vitess/go/vt/vtgate/semantics"
)

// FkChild is used to represent a foreign key child table operation
type FkChild struct {
BVName string
Cols []int // indexes
Op ops.Operator
ST *semantics.SemTable

noColumns
noPredicates
Expand All @@ -37,6 +39,7 @@ type FkChild struct {
// cascades (for example, ON DELETE CASCADE).
type FkCascade struct {
Selection ops.Operator
SSemTable *semantics.SemTable
Children []*FkChild
Parent ops.Operator

Expand Down Expand Up @@ -80,6 +83,7 @@ func (fkc *FkCascade) Clone(inputs []ops.Operator) ops.Operator {
newFkc := &FkCascade{
Parent: inputs[0],
Selection: inputs[1],
SSemTable: fkc.SSemTable,
}
for idx, operator := range inputs {
if idx < 2 {
Expand All @@ -90,6 +94,7 @@ func (fkc *FkCascade) Clone(inputs []ops.Operator) ops.Operator {
BVName: fkc.Children[idx-2].BVName,
Cols: slices.Clone(fkc.Children[idx-2].Cols),
Op: operator,
ST: fkc.Children[idx-2].ST,
})
}
return newFkc
Expand Down
5 changes: 4 additions & 1 deletion go/vt/vtgate/planbuilder/operators/fk_verify.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,15 @@ package operators

import (
"vitess.io/vitess/go/vt/vtgate/planbuilder/operators/ops"
"vitess.io/vitess/go/vt/vtgate/semantics"
)

// VerifyOp keeps the information about the foreign key verification operation.
// It is a Parent verification or a Child verification.
type VerifyOp struct {
Op ops.Operator
Op ops.Operator
ST *semantics.SemTable

Typ string
}

Expand Down
29 changes: 17 additions & 12 deletions go/vt/vtgate/planbuilder/operators/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -344,13 +344,14 @@ func createFKCascadeOp(ctx *plancontext.PlanningContext, parentOp ops.Operator,
fkChildren = append(fkChildren, fkChild)
}

selectionOp, err := createSelectionOp(ctx, selectExprs, updStmt.TableExprs, updStmt.Where, nil, sqlparser.ForUpdateLock)
st, selectionOp, err := createSelectionOp(ctx, selectExprs, updStmt.TableExprs, updStmt.Where, nil, sqlparser.ForUpdateLock)
if err != nil {
return nil, err
}

return &FkCascade{
Selection: selectionOp,
SSemTable: st,
Children: fkChildren,
Parent: parentOp,
}, nil
Expand All @@ -370,13 +371,14 @@ func createFkChildForUpdate(ctx *plancontext.PlanningContext, fk vindexes.ChildF
compExpr := sqlparser.NewComparisonExpr(sqlparser.InOp, valTuple, sqlparser.NewListArg(bvName), nil)
var childWhereExpr sqlparser.Expr = compExpr

var st *semantics.SemTable
var childOp ops.Operator
var err error
switch fk.OnUpdate {
case sqlparser.Cascade:
childOp, err = buildChildUpdOpForCascade(ctx, fk, updStmt, childWhereExpr, updatedTable)
st, childOp, err = buildChildUpdOpForCascade(ctx, fk, updStmt, childWhereExpr, updatedTable)
case sqlparser.SetNull:
childOp, err = buildChildUpdOpForSetNull(ctx, fk, updStmt, childWhereExpr)
st, childOp, err = buildChildUpdOpForSetNull(ctx, fk, updStmt, childWhereExpr)
case sqlparser.SetDefault:
return nil, vterrors.VT09016()
}
Expand All @@ -388,14 +390,15 @@ func createFkChildForUpdate(ctx *plancontext.PlanningContext, fk vindexes.ChildF
BVName: bvName,
Cols: cols,
Op: childOp,
ST: st,
}, nil
}

// buildChildUpdOpForCascade builds the child update statement operator for the CASCADE type foreign key constraint.
// The query looks like this -
//
// `UPDATE <child_table> SET <child_column_updated_using_update_exprs_from_parent_update_query> WHERE <child_columns_in_fk> IN (<bind variable for the output from SELECT>)`
func buildChildUpdOpForCascade(ctx *plancontext.PlanningContext, fk vindexes.ChildFKInfo, updStmt *sqlparser.Update, childWhereExpr sqlparser.Expr, updatedTable *vindexes.Table) (ops.Operator, error) {
func buildChildUpdOpForCascade(ctx *plancontext.PlanningContext, fk vindexes.ChildFKInfo, updStmt *sqlparser.Update, childWhereExpr sqlparser.Expr, updatedTable *vindexes.Table) (*semantics.SemTable, ops.Operator, error) {
// The update expressions are the same as the update expressions in the parent update query
// with the column names replaced with the child column names.
var childUpdateExprs sqlparser.UpdateExprs
Expand Down Expand Up @@ -436,7 +439,7 @@ func buildChildUpdOpForCascade(ctx *plancontext.PlanningContext, fk vindexes.Chi
// `UPDATE <child_table> SET <child_column_updated_using_update_exprs_from_parent_update_query>
// WHERE <child_columns_in_fk> IN (<bind variable for the output from SELECT>)
// [AND ({<bind variables in the SET clause of the original update> IS NULL OR}... <child_columns_in_fk> NOT IN (<bind variables in the SET clause of the original update>))]`
func buildChildUpdOpForSetNull(ctx *plancontext.PlanningContext, fk vindexes.ChildFKInfo, updStmt *sqlparser.Update, childWhereExpr sqlparser.Expr) (ops.Operator, error) {
func buildChildUpdOpForSetNull(ctx *plancontext.PlanningContext, fk vindexes.ChildFKInfo, updStmt *sqlparser.Update, childWhereExpr sqlparser.Expr) (*semantics.SemTable, ops.Operator, error) {
// For the SET NULL type constraint, we need to set all the child columns to NULL.
var childUpdateExprs sqlparser.UpdateExprs
for _, column := range fk.ChildColumns {
Expand Down Expand Up @@ -479,23 +482,25 @@ func createFKVerifyOp(ctx *plancontext.PlanningContext, childOp ops.Operator, up
var Verify []*VerifyOp
// This validates that new values exists on the parent table.
for _, fk := range parentFks {
op, err := createFkVerifyOpForParentFKForUpdate(ctx, updStmt, fk)
st, op, err := createFkVerifyOpForParentFKForUpdate(ctx, updStmt, fk)
if err != nil {
return nil, err
}
Verify = append(Verify, &VerifyOp{
Op: op,
ST: st,
Typ: engine.ParentVerify,
})
}
// This validates that the old values don't exist on the child table.
for _, fk := range restrictChildFks {
op, err := createFkVerifyOpForChildFKForUpdate(ctx, updStmt, fk)
st, op, err := createFkVerifyOpForChildFKForUpdate(ctx, updStmt, fk)
if err != nil {
return nil, err
}
Verify = append(Verify, &VerifyOp{
Op: op,
ST: st,
Typ: engine.ChildVerify,
})
}
Expand All @@ -517,11 +522,11 @@ func createFKVerifyOp(ctx *plancontext.PlanningContext, childOp ops.Operator, up
// where Parent.p1 is null and Parent.p2 is null and Child.id = 1
// and Child.c2 is not null
// limit 1
func createFkVerifyOpForParentFKForUpdate(ctx *plancontext.PlanningContext, updStmt *sqlparser.Update, pFK vindexes.ParentFKInfo) (ops.Operator, error) {
func createFkVerifyOpForParentFKForUpdate(ctx *plancontext.PlanningContext, updStmt *sqlparser.Update, pFK vindexes.ParentFKInfo) (*semantics.SemTable, ops.Operator, error) {
childTblExpr := updStmt.TableExprs[0].(*sqlparser.AliasedTableExpr)
childTbl, err := childTblExpr.TableName()
if err != nil {
return nil, err
return nil, nil, err
}
parentTbl := pFK.Table.GetTableName()
var whereCond sqlparser.Expr
Expand Down Expand Up @@ -594,16 +599,16 @@ func createFkVerifyOpForParentFKForUpdate(ctx *plancontext.PlanningContext, updS
// verify query:
// select 1 from Child join Parent on Parent.p1 = Child.c1 and Parent.p2 = Child.c2
// where Parent.id = 1 and (1 IS NULL OR (child.c1) NOT IN ((1))) limit 1
func createFkVerifyOpForChildFKForUpdate(ctx *plancontext.PlanningContext, updStmt *sqlparser.Update, cFk vindexes.ChildFKInfo) (ops.Operator, error) {
func createFkVerifyOpForChildFKForUpdate(ctx *plancontext.PlanningContext, updStmt *sqlparser.Update, cFk vindexes.ChildFKInfo) (*semantics.SemTable, ops.Operator, error) {
// ON UPDATE RESTRICT foreign keys that require validation, should only be allowed in the case where we
// are verifying all the FKs on vtgate level.
if !ctx.VerifyAllFKs {
return nil, vterrors.VT12002()
return nil, nil, vterrors.VT12002()
}
parentTblExpr := updStmt.TableExprs[0].(*sqlparser.AliasedTableExpr)
parentTbl, err := parentTblExpr.TableName()
if err != nil {
return nil, err
return nil, nil, err
}
childTbl := cFk.Table.GetTableName()
var joinCond sqlparser.Expr
Expand Down
Loading

0 comments on commit ed60cc0

Please sign in to comment.