executor,store: fix a bug caused by HashJoinExec encode/decode (#3225)
HashJoinExec encodeRow method doesn't consider timezone, but decodeRow method does, this introduce a bug.
This commit is contained in:
@ -21,6 +21,7 @@ import (
|
||||
"github.com/pingcap/tidb/context"
|
||||
"github.com/pingcap/tidb/expression"
|
||||
"github.com/pingcap/tidb/sessionctx/variable"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
"github.com/pingcap/tidb/util/mvmap"
|
||||
"github.com/pingcap/tidb/util/types"
|
||||
@ -277,8 +278,15 @@ func (e *HashJoinExec) encodeRow(b []byte, row *Row) ([]byte, error) {
|
||||
e.rowKeyCache[i] = rk
|
||||
}
|
||||
}
|
||||
b, err := codec.EncodeValue(b, row.Data...)
|
||||
return b, errors.Trace(err)
|
||||
loc := e.ctx.GetSessionVars().GetTimeZone()
|
||||
for _, datum := range row.Data {
|
||||
tmp, err := tablecodec.EncodeValue(datum, loc)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
b = append(b, tmp...)
|
||||
}
|
||||
return b, nil
|
||||
}
|
||||
|
||||
func (e *HashJoinExec) decodeRow(data []byte) (*Row, error) {
|
||||
|
||||
@ -551,3 +551,16 @@ func (s *testSuite) TestJoinLeak(c *C) {
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
result.Close()
|
||||
}
|
||||
|
||||
func (s *testSuite) TestHashJoinExecEncodeDecodeRow(c *C) {
|
||||
tk := testkit.NewTestKit(c, s.store)
|
||||
tk.MustExec("use test")
|
||||
tk.MustExec("drop table if exists t1")
|
||||
tk.MustExec("drop table if exists t2")
|
||||
tk.MustExec("create table t1 (id int)")
|
||||
tk.MustExec("create table t2 (id int, name varchar(255), ts timestamp)")
|
||||
tk.MustExec("insert into t1 values (1)")
|
||||
tk.MustExec("insert into t2 values (1, 'xxx', '2003-06-09 10:51:26')")
|
||||
result := tk.MustQuery("select ts from t1 inner join t2 where t2.name = 'xxx'")
|
||||
result.Check(testkit.Rows("2003-06-09 10:51:26"))
|
||||
}
|
||||
|
||||
@ -650,8 +650,7 @@ func (rs *localRegion) setColumnValueToCtx(ctx *selectContext, h int64, row map[
|
||||
} else {
|
||||
data := row[colID]
|
||||
ft := distsql.FieldTypeFromPBColumn(col)
|
||||
// TODO: Should use session's TimeZone instead of UTC.
|
||||
datum, err := tablecodec.DecodeColumnValue(data, ft, time.UTC)
|
||||
datum, err := tablecodec.DecodeColumnValue(data, ft, ctx.eval.TimeZone)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user