Skip to content

Commit

Permalink
expression: check timezone when encoding timestamp datum (#10303)
Browse files Browse the repository at this point in the history
  • Loading branch information
eurekaka authored and qw4990 committed Apr 30, 2019
1 parent e90804f commit e8d9a3f
Show file tree
Hide file tree
Showing 4 changed files with 43 additions and 19 deletions.
2 changes: 1 addition & 1 deletion expression/distsql_builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -557,7 +557,7 @@ func convertTime(data []byte, ftPB *tipb.FieldType, tz *time.Location) (*Constan
if err != nil {
return nil, err
}
if ft.Tp == mysql.TypeTimestamp && !t.IsZero() {
if ft.Tp == mysql.TypeTimestamp && tz != time.UTC {
err = t.ConvertTimeZone(time.UTC, tz)
if err != nil {
return nil, err
Expand Down
8 changes: 3 additions & 5 deletions expression/expr_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ func (pc PbConverter) conOrCorColToPBExpr(expr Expression) *tipb.Expr {
logutil.Logger(context.Background()).Error("eval constant or correlated column", zap.String("expression", expr.ExplainInfo()), zap.Error(err))
return nil
}
tp, val, ok := pc.encodeDatum(d)
tp, val, ok := pc.encodeDatum(ft, d)
if !ok {
return nil
}
Expand All @@ -117,7 +117,7 @@ func (pc PbConverter) conOrCorColToPBExpr(expr Expression) *tipb.Expr {
return &tipb.Expr{Tp: tp, Val: val, FieldType: ToPBFieldType(ft)}
}

func (pc *PbConverter) encodeDatum(d types.Datum) (tipb.ExprType, []byte, bool) {
func (pc *PbConverter) encodeDatum(ft *types.FieldType, d types.Datum) (tipb.ExprType, []byte, bool) {
var (
tp tipb.ExprType
val []byte
Expand Down Expand Up @@ -157,13 +157,11 @@ func (pc *PbConverter) encodeDatum(d types.Datum) (tipb.ExprType, []byte, bool)
case types.KindMysqlTime:
if pc.client.IsRequestTypeSupported(kv.ReqTypeDAG, int64(tipb.ExprType_MysqlTime)) {
tp = tipb.ExprType_MysqlTime
t := d.GetMysqlTime()
v, err := t.ToPackedUint()
val, err := codec.EncodeMySQLTime(pc.sc, d, ft.Tp, nil)
if err != nil {
logutil.Logger(context.Background()).Error("encode mysql time", zap.Error(err))
return tp, nil, false
}
val = codec.EncodeUint(nil, v)
return tp, val, true
}
return tp, nil, false
Expand Down
14 changes: 14 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4235,3 +4235,17 @@ where
datediff(b.date8, date(from_unixtime(a.starttime))) >= 0`
tk.MustQuery(q)
}

func (s *testIntegrationSuite) TestTimestampDatumEncode(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec(`drop table if exists t;`)
tk.MustExec(`create table t (a bigint primary key, b timestamp)`)
tk.MustExec(`insert into t values (1, "2019-04-29 11:56:12")`)
tk.MustQuery(`explain select * from t where b = (select max(b) from t)`).Check(testkit.Rows(
"TableReader_43 10.00 root data:Selection_42",
"└─Selection_42 10.00 cop eq(test.t.b, 2019-04-29 11:56:12)",
" └─TableScan_41 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo",
))
tk.MustQuery(`select * from t where b = (select max(b) from t)`).Check(testkit.Rows(`1 2019-04-29 11:56:12`))
}
38 changes: 25 additions & 13 deletions util/codec/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,21 +67,10 @@ func encode(sc *stmtctx.StatementContext, b []byte, vals []types.Datum, comparab
b = encodeBytes(b, vals[i].GetBytes(), comparable)
case types.KindMysqlTime:
b = append(b, uintFlag)
t := vals[i].GetMysqlTime()
// Encoding timestamp need to consider timezone.
// If it's not in UTC, transform to UTC first.
if t.Type == mysql.TypeTimestamp && sc.TimeZone != time.UTC {
err = t.ConvertTimeZone(sc.TimeZone, time.UTC)
if err != nil {
return nil, errors.Trace(err)
}
}
var v uint64
v, err = t.ToPackedUint()
b, err = EncodeMySQLTime(sc, vals[i], mysql.TypeUnspecified, b)
if err != nil {
return nil, errors.Trace(err)
return nil, err
}
b = EncodeUint(b, v)
case types.KindMysqlDuration:
// duration may have negative value, so we cannot use String to encode directly.
b = append(b, durationFlag)
Expand Down Expand Up @@ -134,6 +123,29 @@ func encode(sc *stmtctx.StatementContext, b []byte, vals []types.Datum, comparab
return b, errors.Trace(err)
}

// EncodeMySQLTime encodes datum of `KindMysqlTime` to []byte.
func EncodeMySQLTime(sc *stmtctx.StatementContext, d types.Datum, tp byte, b []byte) (_ []byte, err error) {
t := d.GetMysqlTime()
// Encoding timestamp need to consider timezone. If it's not in UTC, transform to UTC first.
// This is compatible with `PBToExpr > convertTime`, and coprocessor assumes the passed timestamp is in UTC as well.
if tp == mysql.TypeUnspecified {
tp = t.Type
}
if tp == mysql.TypeTimestamp && sc.TimeZone != time.UTC {
err = t.ConvertTimeZone(sc.TimeZone, time.UTC)
if err != nil {
return nil, err
}
}
var v uint64
v, err = t.ToPackedUint()
if err != nil {
return nil, err
}
b = EncodeUint(b, v)
return b, nil
}

func encodeBytes(b []byte, v []byte, comparable bool) []byte {
if comparable {
b = append(b, bytesFlag)
Expand Down

0 comments on commit e8d9a3f

Please sign in to comment.