executor: fix bug when use limit in CTE (#25261)
This commit is contained in:
@ -328,7 +328,7 @@ func (e *CTEExec) nextChunkLimit(req *chunk.Chunk) error {
|
||||
numRows := uint64(res.NumRows())
|
||||
if e.cursor+numRows > e.limitEnd {
|
||||
numRows = e.limitEnd - e.cursor
|
||||
req.Append(res.CopyConstructSel(), 0, int(numRows)+1)
|
||||
req.Append(res.CopyConstructSel(), 0, int(numRows))
|
||||
} else {
|
||||
req.SwapColumns(res.CopyConstructSel())
|
||||
}
|
||||
|
||||
@ -373,4 +373,75 @@ func (test *CTETestSuite) TestCTEWithLimit(c *check.C) {
|
||||
err = tk.QueryToErr("with recursive cte1 as (select 1/c1 c1 from t1 union select c1 + 1 c1 from cte1 where c1 < 2 limit 1) select * from cte1;")
|
||||
c.Assert(err, check.NotNil)
|
||||
c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value")
|
||||
|
||||
tk.MustExec("set cte_max_recursion_depth = 1000;")
|
||||
tk.MustExec("drop table if exists t1;")
|
||||
tk.MustExec("create table t1(c1 int);")
|
||||
tk.MustExec("insert into t1 values(1), (2), (3);")
|
||||
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 0 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows())
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 1 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 2 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "4"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 3 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "4", "5"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 4 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "4", "5", "6"))
|
||||
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 0 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows())
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 1 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows("4"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 2 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows("4", "5"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 3 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows("4", "5", "6"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 4 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows("4", "5", "6", "7"))
|
||||
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 0 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows())
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 1 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows("5"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 2 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows("5", "6"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 3 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows("5", "6", "7"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 4 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows("5", "6", "7", "8"))
|
||||
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 0 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows())
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 1 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 2 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "2"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 3 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "2", "3"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 4 offset 2) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "2", "3", "4"))
|
||||
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 0 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows())
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 1 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows("2"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 2 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows("2", "3"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 3 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows("2", "3", "4"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 4 offset 3) select * from cte1;")
|
||||
rows.Check(testkit.Rows("2", "3", "4", "3"))
|
||||
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 0 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows())
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 1 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 2 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "4"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 3 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "4", "3"))
|
||||
rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 4 offset 4) select * from cte1;")
|
||||
rows.Check(testkit.Rows("3", "4", "3", "4"))
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user