Skip to content

Commit

Permalink
executor: use EncodeKey to encode groupkey in StreamAggExec (#49974) (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-chi-bot authored Jan 8, 2024
1 parent cd073ea commit 6a5b2a3
Show file tree
Hide file tree
Showing 2 changed files with 19 additions and 2 deletions.
4 changes: 2 additions & 2 deletions executor/aggregate.go
Original file line number Diff line number Diff line change
Expand Up @@ -1509,12 +1509,12 @@ func (e *vecGroupChecker) splitIntoGroups(chk *chunk.Chunk) (isFirstGroupSameAsP
return false, err
}
}
e.firstGroupKey, err = codec.EncodeValue(e.ctx.GetSessionVars().StmtCtx, e.firstGroupKey, e.firstRowDatums...)
e.firstGroupKey, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, e.firstGroupKey, e.firstRowDatums...)
if err != nil {
return false, err
}

e.lastGroupKey, err = codec.EncodeValue(e.ctx.GetSessionVars().StmtCtx, e.lastGroupKey, e.lastRowDatums...)
e.lastGroupKey, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, e.lastGroupKey, e.lastRowDatums...)
if err != nil {
return false, err
}
Expand Down
17 changes: 17 additions & 0 deletions executor/issuetest/executor_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1446,3 +1446,20 @@ PARTITION BY HASH (c5) PARTITIONS 4;`)
// Again, a simpler reproduce.
tk.MustQuery("select /*+ inl_join (t1, t2) */ t2.c5 from t1 right join t2 on t1.c2 = t2.c5 where not( t1.c2 between '4s7ht' and 'mj' );").Check(testkit.Rows())
}

func TestIssue49902(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set @@tidb_max_chunk_size = 32;")
tk.MustExec("drop table if exists t, s;")
tk.MustExec("CREATE TABLE `t` (`c` char(1)) COLLATE=utf8_general_ci ;")
tk.MustExec("insert into t values(\"V\"),(\"v\");")
tk.MustExec("insert into t values(\"V\"),(\"v\"),(\"v\");")
tk.MustExec("CREATE TABLE `s` (`col_61` int);")
tk.MustExec("insert into s values(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1);")
tk.MustExec("insert into s values(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1),(1);")
tk.MustQuery("SELECT /*+ stream_agg()*/ count(`t`.`c`) FROM (`s`) JOIN `t` GROUP BY `t`.`c`;").Check(testkit.Rows("170"))
tk.MustQuery("SELECT count(`t`.`c`) FROM (`s`) JOIN `t` GROUP BY `t`.`c`;").Check(testkit.Rows("170"))
tk.MustExec("set @@tidb_max_chunk_size = default;")
}

0 comments on commit 6a5b2a3

Please sign in to comment.