Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

parser: fix limit lost in SetOprStmt (#49898) #49932

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
29 changes: 29 additions & 0 deletions pkg/parser/ast/dml.go
Original file line number Diff line number Diff line change
Expand Up @@ -1569,6 +1569,7 @@ func (n *SetOprSelectList) Restore(ctx *format.RestoreCtx) error {
return errors.Annotate(err, "An error occurred while restore SetOprSelectList.With")
}
}

for i, stmt := range n.Selects {
switch selectStmt := stmt.(type) {
case *SelectStmt:
Expand All @@ -1590,6 +1591,20 @@ func (n *SetOprSelectList) Restore(ctx *format.RestoreCtx) error {
ctx.WritePlain(")")
}
}

if n.OrderBy != nil {
ctx.WritePlain(" ")
if err := n.OrderBy.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore SetOprSelectList.OrderBy")
}
}

if n.Limit != nil {
ctx.WritePlain(" ")
if err := n.Limit.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore SetOprSelectList.Limit")
}
}
return nil
}

Expand All @@ -1614,6 +1629,20 @@ func (n *SetOprSelectList) Accept(v Visitor) (Node, bool) {
}
n.Selects[i] = node
}
if n.OrderBy != nil {
node, ok := n.OrderBy.Accept(v)
if !ok {
return n, false
}
n.OrderBy = node.(*OrderByClause)
}
if n.Limit != nil {
node, ok := n.Limit.Accept(v)
if !ok {
return n, false
}
n.Limit = node.(*Limit)
}
return v.Leave(n)
}

Expand Down
47 changes: 25 additions & 22 deletions pkg/parser/parser.go
Original file line number Diff line number Diff line change
Expand Up @@ -19908,15 +19908,19 @@ yynewstate:
}
var setOprList2 []ast.Node
var with2 *ast.WithClause
var limit2 *ast.Limit
var orderBy2 *ast.OrderByClause
switch x := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList2 = []ast.Node{x}
with2 = x.With
case *ast.SetOprStmt:
setOprList2 = x.SelectList.Selects
with2 = x.With
limit2 = x.Limit
orderBy2 = x.OrderBy
}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2, Limit: limit2, OrderBy: orderBy2}
nextSetOprList.AfterSetOperator = yyS[yypt-2].item.(*ast.SetOprType)
setOprList := append(setOprList1, nextSetOprList)
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
Expand All @@ -19932,15 +19936,19 @@ yynewstate:
}
var setOprList2 []ast.Node
var with2 *ast.WithClause
var limit2 *ast.Limit
var orderBy2 *ast.OrderByClause
switch x := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList2 = []ast.Node{x}
with2 = x.With
case *ast.SetOprStmt:
setOprList2 = x.SelectList.Selects
with2 = x.With
limit2 = x.Limit
orderBy2 = x.OrderBy
}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2, Limit: limit2, OrderBy: orderBy2}
nextSetOprList.AfterSetOperator = yyS[yypt-2].item.(*ast.SetOprType)
setOprList := append(setOprList1, nextSetOprList)
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
Expand All @@ -19956,15 +19964,19 @@ yynewstate:
}
var setOprList2 []ast.Node
var with2 *ast.WithClause
var limit2 *ast.Limit
var orderBy2 *ast.OrderByClause
switch x := yyS[yypt-2].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList2 = []ast.Node{x}
with2 = x.With
case *ast.SetOprStmt:
setOprList2 = x.SelectList.Selects
with2 = x.With
limit2 = x.Limit
orderBy2 = x.OrderBy
}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2, Limit: limit2, OrderBy: orderBy2}
nextSetOprList.AfterSetOperator = yyS[yypt-3].item.(*ast.SetOprType)
setOprList := append(setOprList1, nextSetOprList)
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
Expand All @@ -19975,48 +19987,39 @@ yynewstate:
case 1839:
{
var setOprList []ast.Node
var with *ast.WithClause
switch x := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: []ast.Node{x}}}
case *ast.SetOprStmt:
setOprList = x.SelectList.Selects
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With, Limit: x.Limit, OrderBy: x.OrderBy}}
}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}, With: with}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
setOpr.OrderBy = yyS[yypt-0].item.(*ast.OrderByClause)
parser.yyVAL.statement = setOpr
}
case 1840:
{
var setOprList []ast.Node
var with *ast.WithClause
switch x := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: []ast.Node{x}, With: x.With}}
case *ast.SetOprStmt:
setOprList = x.SelectList.Selects
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With, Limit: x.Limit, OrderBy: x.OrderBy}}
}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}, With: with}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
setOpr.Limit = yyS[yypt-0].item.(*ast.Limit)
parser.yyVAL.statement = setOpr
}
case 1841:
{
var setOprList []ast.Node
var with *ast.WithClause
switch x := yyS[yypt-2].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: []ast.Node{x}, With: x.With}}
case *ast.SetOprStmt:
setOprList = x.SelectList.Selects
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With, Limit: x.Limit, OrderBy: x.OrderBy}}
}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}, With: with}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
setOpr.OrderBy = yyS[yypt-1].item.(*ast.OrderByClause)
setOpr.Limit = yyS[yypt-0].item.(*ast.Limit)
parser.yyVAL.statement = setOpr
Expand Down Expand Up @@ -20048,7 +20051,7 @@ yynewstate:
case *ast.SelectStmt:
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: []ast.Node{x}}}
case *ast.SetOprStmt:
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With}}
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With, Limit: x.Limit, OrderBy: x.OrderBy}}
}
parser.yyVAL.item = setOprList
}
Expand Down
47 changes: 25 additions & 22 deletions pkg/parser/parser.y
Original file line number Diff line number Diff line change
Expand Up @@ -10274,15 +10274,19 @@ SetOprStmtWithLimitOrderBy:
}
var setOprList2 []ast.Node
var with2 *ast.WithClause
var limit2 *ast.Limit
var orderBy2 *ast.OrderByClause
switch x := $3.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList2 = []ast.Node{x}
with2 = x.With
case *ast.SetOprStmt:
setOprList2 = x.SelectList.Selects
with2 = x.With
limit2 = x.Limit
orderBy2 = x.OrderBy
}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2, Limit: limit2, OrderBy: orderBy2}
nextSetOprList.AfterSetOperator = $2.(*ast.SetOprType)
setOprList := append(setOprList1, nextSetOprList)
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
Expand All @@ -10298,15 +10302,19 @@ SetOprStmtWithLimitOrderBy:
}
var setOprList2 []ast.Node
var with2 *ast.WithClause
var limit2 *ast.Limit
var orderBy2 *ast.OrderByClause
switch x := $3.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList2 = []ast.Node{x}
with2 = x.With
case *ast.SetOprStmt:
setOprList2 = x.SelectList.Selects
with2 = x.With
limit2 = x.Limit
orderBy2 = x.OrderBy
}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2, Limit: limit2, OrderBy: orderBy2}
nextSetOprList.AfterSetOperator = $2.(*ast.SetOprType)
setOprList := append(setOprList1, nextSetOprList)
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
Expand All @@ -10322,15 +10330,19 @@ SetOprStmtWithLimitOrderBy:
}
var setOprList2 []ast.Node
var with2 *ast.WithClause
var limit2 *ast.Limit
var orderBy2 *ast.OrderByClause
switch x := $3.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList2 = []ast.Node{x}
with2 = x.With
case *ast.SetOprStmt:
setOprList2 = x.SelectList.Selects
with2 = x.With
limit2 = x.Limit
orderBy2 = x.OrderBy
}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2}
nextSetOprList := &ast.SetOprSelectList{Selects: setOprList2, With: with2, Limit: limit2, OrderBy: orderBy2}
nextSetOprList.AfterSetOperator = $2.(*ast.SetOprType)
setOprList := append(setOprList1, nextSetOprList)
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
Expand All @@ -10341,48 +10353,39 @@ SetOprStmtWithLimitOrderBy:
| SubSelect OrderBy
{
var setOprList []ast.Node
var with *ast.WithClause
switch x := $1.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: []ast.Node{x}}}
case *ast.SetOprStmt:
setOprList = x.SelectList.Selects
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With, Limit: x.Limit, OrderBy: x.OrderBy}}
}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}, With: with}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
setOpr.OrderBy = $2.(*ast.OrderByClause)
$$ = setOpr
}
| SubSelect SelectStmtLimit
{
var setOprList []ast.Node
var with *ast.WithClause
switch x := $1.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: []ast.Node{x}, With: x.With}}
case *ast.SetOprStmt:
setOprList = x.SelectList.Selects
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With, Limit: x.Limit, OrderBy: x.OrderBy}}
}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}, With: with}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
setOpr.Limit = $2.(*ast.Limit)
$$ = setOpr
}
| SubSelect OrderBy SelectStmtLimit
{
var setOprList []ast.Node
var with *ast.WithClause
switch x := $1.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: []ast.Node{x}, With: x.With}}
case *ast.SetOprStmt:
setOprList = x.SelectList.Selects
with = x.With
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With, Limit: x.Limit, OrderBy: x.OrderBy}}
}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}, With: with}
setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}}
setOpr.OrderBy = $2.(*ast.OrderByClause)
setOpr.Limit = $3.(*ast.Limit)
$$ = setOpr
Expand Down Expand Up @@ -10419,7 +10422,7 @@ SetOprClause:
case *ast.SelectStmt:
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: []ast.Node{x}}}
case *ast.SetOprStmt:
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With}}
setOprList = []ast.Node{&ast.SetOprSelectList{Selects: x.SelectList.Selects, With: x.With, Limit: x.Limit, OrderBy: x.OrderBy}}
}
$$ = setOprList
}
Expand Down
10 changes: 10 additions & 0 deletions pkg/parser/parser_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5147,6 +5147,16 @@ func TestSetOperator(t *testing.T) {
{"(select c1 from t1) union all (select c2 from t2 except select c3 from t3) order by c1 limit 1", true, "(SELECT `c1` FROM `t1`) UNION ALL (SELECT `c2` FROM `t2` EXCEPT SELECT `c3` FROM `t3`) ORDER BY `c1` LIMIT 1"},
{"((select c1 from t1) except select c2 from t2) intersect all (select c3 from t3) order by c1 limit 1", true, "((SELECT `c1` FROM `t1`) EXCEPT SELECT `c2` FROM `t2`) INTERSECT ALL (SELECT `c3` FROM `t3`) ORDER BY `c1` LIMIT 1"},
{"select 1 union distinct (select 1 except all select 1 intersect select 1)", true, "SELECT 1 UNION (SELECT 1 EXCEPT ALL SELECT 1 INTERSECT SELECT 1)"},

// https://github.com/pingcap/tidb/issues/49874
{"select * from a where PK = 0 union all (select * from b where PK = 0 union all (select * from b where PK != 0) order by pk limit 1)", true,
"SELECT * FROM `a` WHERE `PK`=0 UNION ALL (SELECT * FROM `b` WHERE `PK`=0 UNION ALL (SELECT * FROM `b` WHERE `PK`!=0) ORDER BY `pk` LIMIT 1)"},
{"select * from a where PK = 0 union all (select * from b where PK = 0 union all (select * from b where PK != 0) order by pk limit 1) order by pk limit 2", true,
"SELECT * FROM `a` WHERE `PK`=0 UNION ALL (SELECT * FROM `b` WHERE `PK`=0 UNION ALL (SELECT * FROM `b` WHERE `PK`!=0) ORDER BY `pk` LIMIT 1) ORDER BY `pk` LIMIT 2"},
{"(select * from b where pk= 0 union all (select * from b where pk !=0) order by pk limit 1) order by pk limit 2", true,
"(SELECT * FROM `b` WHERE `pk`=0 UNION ALL (SELECT * FROM `b` WHERE `pk`!=0) ORDER BY `pk` LIMIT 1) ORDER BY `pk` LIMIT 2"},
{"(select * from b where pk= 0 union all (select * from b where pk !=0) order by pk limit 1) order by pk", true,
"(SELECT * FROM `b` WHERE `pk`=0 UNION ALL (SELECT * FROM `b` WHERE `pk`!=0) ORDER BY `pk` LIMIT 1) ORDER BY `pk`"},
}
RunTest(t, table, false)
}
Expand Down