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

Merged
merged 6 commits into from
Dec 29, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
36 changes: 30 additions & 6 deletions pkg/parser/parser.go
Original file line number Diff line number Diff line change
Expand Up @@ -19931,15 +19931,19 @@ yynewstate:
}
var setOprList2 []ast.Node
var with2 *ast.WithClause
var limit2 *ast.Limit
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add test cases for it ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I'm doing it.

Copy link
Contributor Author

@Defined2014 Defined2014 Dec 29, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. @hawkingrei

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 @@ -19955,15 +19959,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 @@ -19979,15 +19987,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 @@ -19999,47 +20011,59 @@ yynewstate:
{
var setOprList []ast.Node
var with *ast.WithClause
var limit *ast.Limit
var orderBy *ast.OrderByClause
switch x := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
case *ast.SetOprStmt:
setOprList = 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}, With: with, Limit: limit, OrderBy: orderBy}
setOpr.OrderBy = yyS[yypt-0].item.(*ast.OrderByClause)
parser.yyVAL.statement = setOpr
}
case 1843:
{
var setOprList []ast.Node
var with *ast.WithClause
var limit *ast.Limit
var orderBy *ast.OrderByClause
switch x := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
case *ast.SetOprStmt:
setOprList = 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}, With: with, Limit: limit, OrderBy: orderBy}
setOpr.Limit = yyS[yypt-0].item.(*ast.Limit)
parser.yyVAL.statement = setOpr
}
case 1844:
{
var setOprList []ast.Node
var with *ast.WithClause
var limit *ast.Limit
var orderBy *ast.OrderByClause
switch x := yyS[yypt-2].expr.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
case *ast.SetOprStmt:
setOprList = 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}, With: with, Limit: limit, OrderBy: orderBy}
setOpr.OrderBy = yyS[yypt-1].item.(*ast.OrderByClause)
setOpr.Limit = yyS[yypt-0].item.(*ast.Limit)
parser.yyVAL.statement = setOpr
Expand Down
36 changes: 30 additions & 6 deletions pkg/parser/parser.y
Original file line number Diff line number Diff line change
Expand Up @@ -10260,15 +10260,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 @@ -10284,15 +10288,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 @@ -10308,15 +10316,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 @@ -10328,47 +10340,59 @@ SetOprStmtWithLimitOrderBy:
{
var setOprList []ast.Node
var with *ast.WithClause
var limit *ast.Limit
var orderBy *ast.OrderByClause
switch x := $1.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
case *ast.SetOprStmt:
setOprList = 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}, With: with, Limit: limit, OrderBy: orderBy}
setOpr.OrderBy = $2.(*ast.OrderByClause)
$$ = setOpr
}
| SubSelect SelectStmtLimit
{
var setOprList []ast.Node
var with *ast.WithClause
var limit *ast.Limit
var orderBy *ast.OrderByClause
switch x := $1.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
case *ast.SetOprStmt:
setOprList = 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}, With: with, Limit: limit, OrderBy: orderBy}
setOpr.Limit = $2.(*ast.Limit)
$$ = setOpr
}
| SubSelect OrderBy SelectStmtLimit
{
var setOprList []ast.Node
var with *ast.WithClause
var limit *ast.Limit
var orderBy *ast.OrderByClause
switch x := $1.(*ast.SubqueryExpr).Query.(type) {
case *ast.SelectStmt:
setOprList = []ast.Node{x}
with = x.With
case *ast.SetOprStmt:
setOprList = 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}, With: with, Limit: limit, OrderBy: orderBy}
setOpr.OrderBy = $2.(*ast.OrderByClause)
setOpr.Limit = $3.(*ast.Limit)
$$ = setOpr
Expand Down