Skip to content

Commit

Permalink
executor: fix sort result for batch-point-get by unsigned pk (#20108) (
Browse files Browse the repository at this point in the history
…#20205)

Signed-off-by: ti-srebot <ti-srebot@pingcap.com>
  • Loading branch information
ti-srebot authored Oct 3, 2020
1 parent e463853 commit 7eab75c
Show file tree
Hide file tree
Showing 2 changed files with 24 additions and 2 deletions.
14 changes: 12 additions & 2 deletions executor/batch_point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv"
Expand Down Expand Up @@ -231,12 +232,21 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
failpoint.InjectContext(ctx, "batchPointGetRepeatableReadTest-step2", nil)
})
} else if e.keepOrder {
sort.Slice(e.handles, func(i int, j int) bool {
less := func(i int, j int) bool {
if e.desc {
return e.handles[i] > e.handles[j]
}
return e.handles[i] < e.handles[j]
})
}
if e.tblInfo.PKIsHandle && mysql.HasUnsignedFlag(e.tblInfo.GetPkColInfo().Flag) {
less = func(i int, j int) bool {
if e.desc {
return uint64(e.handles[i]) > uint64(e.handles[j])
}
return uint64(e.handles[i]) < uint64(e.handles[j])
}
}
sort.Slice(e.handles, less)
}

keys := make([]kv.Key, len(e.handles))
Expand Down
12 changes: 12 additions & 0 deletions executor/batch_point_get_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,3 +138,15 @@ func (s *testBatchPointGetSuite) TestBatchPointGetCache(c *C) {
tk.MustQuery("SELECT id, token FROM test.customers WHERE id IN (28)")
tk.MustQuery("SELECT id, token FROM test.customers WHERE id IN (28, 29);").Check(testkit.Rows("28 07j", "29 03j"))
}

func (s *testBatchPointGetSuite) TestBatchPointGetUnsignedHandleWithSort(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t2")
tk.MustExec("create table t2 (id bigint(20) unsigned, primary key(id))")
tk.MustExec("insert into t2 values (8738875760185212610)")
tk.MustExec("insert into t2 values (9814441339970117597)")
tk.MustExec("insert into t2 values (1)")
tk.MustQuery("select id from t2 where id in (8738875760185212610, 1, 9814441339970117597) order by id").Check(testkit.Rows("1", "8738875760185212610", "9814441339970117597"))
tk.MustQuery("select id from t2 where id in (8738875760185212610, 1, 9814441339970117597) order by id desc").Check(testkit.Rows("9814441339970117597", "8738875760185212610", "1"))
}

0 comments on commit 7eab75c

Please sign in to comment.