From 0007b78ba9b14f26050d1419c581bbb87ab81dcd Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Wed, 8 May 2024 15:47:15 +0800 Subject: [PATCH] ttl: use static expression context to caculate the ttl expire time --- pkg/ddl/BUILD.bazel | 1 + pkg/ddl/ttl.go | 24 ++---- pkg/ttl/cache/BUILD.bazel | 5 +- pkg/ttl/cache/table.go | 119 ++++++++++++++--------------- pkg/ttl/cache/table_test.go | 144 +++++++++++++++++------------------- 5 files changed, 132 insertions(+), 161 deletions(-) diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index df643deea7e08..f4afaeb40affc 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -128,6 +128,7 @@ go_library( "//pkg/table/tables", "//pkg/tablecodec", "//pkg/tidb-binlog/pump_client", + "//pkg/ttl/cache", "//pkg/types", "//pkg/types/parser_driver", "//pkg/util", diff --git a/pkg/ddl/ttl.go b/pkg/ddl/ttl.go index 7b6b1c4a7ec04..eefe3919cc581 100644 --- a/pkg/ddl/ttl.go +++ b/pkg/ddl/ttl.go @@ -15,19 +15,18 @@ package ddl import ( - "fmt" "strings" + "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/meta" - "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/dbterror" ) @@ -97,7 +96,7 @@ func onTTLInfoChange(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err er } func checkTTLInfoValid(ctx sessionctx.Context, schema model.CIStr, tblInfo *model.TableInfo) error { - if err := checkTTLIntervalExpr(ctx.GetExprCtx(), tblInfo.TTLInfo); err != nil { + if err := checkTTLIntervalExpr(tblInfo.TTLInfo); err != nil { return err } @@ -108,20 +107,9 @@ func checkTTLInfoValid(ctx sessionctx.Context, schema model.CIStr, tblInfo *mode return checkTTLInfoColumnType(tblInfo) } -func checkTTLIntervalExpr(ctx expression.BuildContext, ttlInfo *model.TTLInfo) error { - // FIXME: use a better way to validate the interval expression in ttl - var nowAddIntervalExpr ast.ExprNode - - unit := ast.TimeUnitType(ttlInfo.IntervalTimeUnit) - expr := fmt.Sprintf("select NOW() + INTERVAL %s %s", ttlInfo.IntervalExprStr, unit.String()) - stmts, _, err := parser.New().ParseSQL(expr) - if err != nil { - // FIXME: the error information can be wrong, as it could indicate an unknown position to user. - return errors.Trace(err) - } - nowAddIntervalExpr = stmts[0].(*ast.SelectStmt).Fields.Fields[0].Expr - _, err = expression.EvalSimpleAst(ctx, nowAddIntervalExpr) - return err +func checkTTLIntervalExpr(ttlInfo *model.TTLInfo) error { + _, err := cache.EvalExpireTime(time.Now(), ttlInfo.IntervalExprStr, ast.TimeUnitType(ttlInfo.IntervalTimeUnit)) + return errors.Trace(err) } func checkTTLInfoColumnType(tblInfo *model.TableInfo) error { diff --git a/pkg/ttl/cache/BUILD.bazel b/pkg/ttl/cache/BUILD.bazel index 5725cd26f53d4..5fe03a4d0f063 100644 --- a/pkg/ttl/cache/BUILD.bazel +++ b/pkg/ttl/cache/BUILD.bazel @@ -12,6 +12,8 @@ go_library( importpath = "github.com/pingcap/tidb/pkg/ttl/cache", visibility = ["//visibility:public"], deps = [ + "//pkg/expression", + "//pkg/expression/contextstatic", "//pkg/infoschema", "//pkg/kv", "//pkg/parser/ast", @@ -48,10 +50,11 @@ go_test( ], embed = [":cache"], flaky = True, - shard_count = 16, + shard_count = 17, deps = [ "//pkg/infoschema", "//pkg/kv", + "//pkg/parser/ast", "//pkg/parser/model", "//pkg/server", "//pkg/session", diff --git a/pkg/ttl/cache/table.go b/pkg/ttl/cache/table.go index 2a8d46b55f96a..bb6b99123104c 100644 --- a/pkg/ttl/cache/table.go +++ b/pkg/ttl/cache/table.go @@ -23,6 +23,8 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/expression/contextstatic" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" @@ -193,88 +195,44 @@ func SetMockExpireTime(ctx context.Context, tm time.Time) context.Context { } // EvalExpireTime returns the expired time. -// It uses the global timezone to compute the expired time. -// Then we'll reset the returned expired time to the same timezone as the input `now`. -func (t *PhysicalTable) EvalExpireTime(ctx context.Context, se session.Session, - now time.Time) (expire time.Time, err error) { - if intest.InTest { - if tm, ok := ctx.Value(mockExpireTimeKey).(time.Time); ok { - return tm, err - } - } - - expireExpr := t.TTLInfo.IntervalExprStr - unit := ast.TimeUnitType(t.TTLInfo.IntervalTimeUnit) - - // Use the global time zone to compute expire time. - // Different timezones may have different results event with the same "now" time and TTL expression. - // Consider a TTL setting with the expiration `INTERVAL 1 MONTH`. - // If the current timezone is `Asia/Shanghai` and now is `2021-03-01 00:00:00 +0800` - // the expired time should be `2021-02-01 00:00:00 +0800`, corresponding to UTC time `2021-01-31 16:00:00 UTC`. - // But if we use the `UTC` time zone, the current time is `2021-02-28 16:00:00 UTC`, - // and the expired time should be `2021-01-28 16:00:00 UTC` that is not the same the previous one. - globalTz, err := se.GlobalTimeZone(ctx) - if err != nil { - return time.Time{}, err - } - - var rows []chunk.Row - - // We should set the session time zone to UTC because the next SQLs should be executed in the UTC timezone. - // The session time zone should be reverted to the original one after the SQLs are executed. - rows, err = se.ExecuteSQL(ctx, "SELECT @@time_zone") - if err != nil { - return - } - - originalTZ := rows[0].GetString(0) - if _, err = se.ExecuteSQL(ctx, "SET @@time_zone='UTC'"); err != nil { - return - } - - defer func() { - _, restoreErr := se.ExecuteSQL(ctx, "SET @@time_zone=%?", originalTZ) - if err == nil { - err = restoreErr - } - }() - +func EvalExpireTime(now time.Time, interval string, unit ast.TimeUnitType) (time.Time, error) { // Firstly, we should use the UTC time zone to compute the expired time to avoid time shift caused by DST. // The start time should be a time with the same datetime string as `now` but it is in the UTC timezone. // For example, if global timezone is `Asia/Shanghai` with a string format `2020-01-01 08:00:00 +0800`. // The startTime should be in timezone `UTC` and have a string format `2020-01-01 08:00:00 +0000` which is not the // same as the original one (`2020-01-01 00:00:00 +0000` in UTC actually). - nowInGlobalTZ := now.In(globalTz) - startTime := time.Date( - nowInGlobalTZ.Year(), nowInGlobalTZ.Month(), nowInGlobalTZ.Day(), - nowInGlobalTZ.Hour(), nowInGlobalTZ.Minute(), nowInGlobalTZ.Second(), - nowInGlobalTZ.Nanosecond(), time.UTC, + start := time.Date( + now.Year(), now.Month(), now.Day(), + now.Hour(), now.Minute(), now.Second(), + now.Nanosecond(), time.UTC, ) - rows, err = se.ExecuteSQL( - ctx, - // FROM_UNIXTIME does not support negative value, so we use `FROM_UNIXTIME(0) + INTERVAL ` - // to present current time - fmt.Sprintf("SELECT FROM_UNIXTIME(0) + INTERVAL %d MICROSECOND - INTERVAL %s %s", - startTime.UnixMicro(), - expireExpr, - unit.String(), + exprCtx := contextstatic.NewStaticExprContext() + // we need to set the location to UTC to make sure the time is in the same timezone as the start time. + intest.Assert(exprCtx.GetEvalCtx().Location() == time.UTC) + expr, err := expression.ParseSimpleExpr( + exprCtx, + fmt.Sprintf("FROM_UNIXTIME(0) + INTERVAL %d MICROSECOND - INTERVAL %s %s", + start.UnixMicro(), interval, unit.String(), ), ) + if err != nil { + return time.Time{}, err + } + tm, _, err := expr.EvalTime(exprCtx.GetEvalCtx(), chunk.Row{}) if err != nil { - return + return time.Time{}, err } - tm, err := rows[0].GetTime(0).GoTime(time.UTC) + end, err := tm.GoTime(time.UTC) if err != nil { - return + return time.Time{}, err } // Then we should add the duration between the time get from the previous SQL and the start time to the now time. - expiredTime := nowInGlobalTZ. - In(now.Location()). - Add(tm.Sub(startTime)). + expiredTime := now. + Add(end.Sub(start)). // Truncate to second to make sure the precision is always the same with the one stored in a table to avoid some // comparing problems in testing. Truncate(time.Second) @@ -282,6 +240,37 @@ func (t *PhysicalTable) EvalExpireTime(ctx context.Context, se session.Session, return expiredTime, nil } +// EvalExpireTime returns the expired time for the current time. +// It uses the global timezone in session to evaluation the context +// and the return time is in the same timezone of now argument. +func (t *PhysicalTable) EvalExpireTime(ctx context.Context, se session.Session, + now time.Time) (time.Time, error) { + if intest.InTest { + if tm, ok := ctx.Value(mockExpireTimeKey).(time.Time); ok { + return tm, nil + } + } + + // Use the global time zone to compute expire time. + // Different timezones may have different results event with the same "now" time and TTL expression. + // Consider a TTL setting with the expiration `INTERVAL 1 MONTH`. + // If the current timezone is `Asia/Shanghai` and now is `2021-03-01 00:00:00 +0800` + // the expired time should be `2021-02-01 00:00:00 +0800`, corresponding to UTC time `2021-01-31 16:00:00 UTC`. + // But if we use the `UTC` time zone, the current time is `2021-02-28 16:00:00 UTC`, + // and the expired time should be `2021-01-28 16:00:00 UTC` that is not the same the previous one. + globalTz, err := se.GlobalTimeZone(ctx) + if err != nil { + return time.Time{}, err + } + + expire, err := EvalExpireTime(now.In(globalTz), t.TTLInfo.IntervalExprStr, ast.TimeUnitType(t.TTLInfo.IntervalTimeUnit)) + if err != nil { + return time.Time{}, err + } + + return expire.In(now.Location()), nil +} + // SplitScanRanges split ranges for TTL scan func (t *PhysicalTable) SplitScanRanges(ctx context.Context, store kv.Storage, splitCnt int) ([]ScanRange, error) { if len(t.KeyColumns) < 1 || splitCnt <= 1 { diff --git a/pkg/ttl/cache/table_test.go b/pkg/ttl/cache/table_test.go index 2a9cc36443fcc..87ea2bb1dbc4f 100644 --- a/pkg/ttl/cache/table_test.go +++ b/pkg/ttl/cache/table_test.go @@ -20,6 +20,7 @@ import ( "testing" "time" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/ttl/cache" @@ -162,141 +163,130 @@ func TestNewTTLTable(t *testing.T) { } } -func TestEvalTTLExpireTime(t *testing.T) { +func TestTableEvalTTLExpireTime(t *testing.T) { store, do := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - tk.MustExec("create table test.t(a int, t datetime) ttl = `t` + interval 1 day") - tk.MustExec("create table test.t2(a int, t datetime) ttl = `t` + interval 3 month") tk.MustExec("set @@time_zone='Asia/Tokyo'") + tk.MustExec("create table test.t(a int, t datetime) ttl = `t` + interval 1 month") tb, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() ttlTbl, err := cache.NewPhysicalTable(model.NewCIStr("test"), tblInfo, model.NewCIStr("")) require.NoError(t, err) + se := session.NewSession(tk.Session(), tk.Session(), nil) + // the global timezone set to +02:00 + tz1 := time.FixedZone("", 2*3600) + _, err = se.ExecuteSQL(context.TODO(), "SET @@global.time_zone = '+02:00'") + require.NoError(t, err) + // the timezone of now argument is set to -02:00 + tz2 := time.FixedZone("-02:00", -2*3600) + now, err := time.ParseInLocation(time.DateTime, "1999-02-28 23:00:00", tz2) + require.NoError(t, err) + tm, err := ttlTbl.EvalExpireTime(context.TODO(), se, now) + require.NoError(t, err) + // The expired time should be calculated according to the global time zone + require.Equal(t, "1999-02-01 03:00:00", tm.In(tz1).Format(time.DateTime)) + // The location of the expired time should be the same with the input argument `now` + require.Same(t, tz2, tm.Location()) + + // should support a string format interval + tk.MustExec("create table test.t2(a int, t datetime) ttl = `t` + interval '1:3' hour_minute") tb2, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) require.NoError(t, err) tblInfo2 := tb2.Meta() ttlTbl2, err := cache.NewPhysicalTable(model.NewCIStr("test"), tblInfo2, model.NewCIStr("")) require.NoError(t, err) - - se := session.NewSession(tk.Session(), tk.Session(), nil) - - now := time.UnixMilli(0) - tz1, err := time.LoadLocation("Asia/Shanghai") + now, err = time.ParseInLocation(time.DateTime, "2020-01-01 15:00:00", tz1) require.NoError(t, err) - tz2, err := time.LoadLocation("Europe/Berlin") + tm, err = ttlTbl2.EvalExpireTime(context.TODO(), se, now) require.NoError(t, err) + require.Equal(t, "2020-01-01 13:57:00", tm.Format(time.DateTime)) + require.Same(t, tz1, tm.Location()) - _, err = se.ExecuteSQL(context.TODO(), "SET @@global.time_zone = 'Asia/Shanghai'") - require.NoError(t, err) - tm, err := ttlTbl.EvalExpireTime(context.TODO(), se, now) - require.NoError(t, err) - require.Equal(t, now.Add(-time.Hour*24).Unix(), tm.Unix()) - require.Equal(t, "1969-12-31 08:00:00", tm.In(tz1).Format(time.DateTime)) - require.Same(t, now.Location(), tm.Location()) + // session time zone should keep unchanged + tk.MustQuery("select @@time_zone").Check(testkit.Rows("Asia/Tokyo")) +} - _, err = se.ExecuteSQL(context.TODO(), "SET @@global.time_zone = 'Europe/Berlin'") +func TestEvalTTLExpireTime(t *testing.T) { + tzShanghai, err := time.LoadLocation("Asia/Shanghai") require.NoError(t, err) - tm, err = ttlTbl.EvalExpireTime(context.TODO(), se, now) + tzBerlin, err := time.LoadLocation("Europe/Berlin") require.NoError(t, err) - require.Equal(t, now.Add(-time.Hour*24).Unix(), tm.Unix()) - require.Equal(t, "1969-12-31 01:00:00", tm.In(tz2).Format(time.DateTime)) - require.Same(t, now.Location(), tm.Location()) - _, err = se.ExecuteSQL(context.TODO(), "SET @@global.time_zone = 'Asia/Shanghai'") + tm, err := cache.EvalExpireTime(time.UnixMilli(0).In(tzShanghai), "1", ast.TimeUnitDay) require.NoError(t, err) - tm, err = ttlTbl2.EvalExpireTime(context.TODO(), se, now) + require.Equal(t, time.UnixMilli(0).Add(-time.Hour*24).Unix(), tm.Unix()) + require.Equal(t, "1969-12-31 08:00:00", tm.Format(time.DateTime)) + require.Same(t, tzShanghai, tm.Location()) + + tm, err = cache.EvalExpireTime(time.UnixMilli(0).In(tzBerlin), "1", ast.TimeUnitDay) require.NoError(t, err) - require.Equal(t, "1969-10-01 08:00:00", tm.In(tz1).Format(time.DateTime)) - require.Same(t, now.Location(), tm.Location()) + require.Equal(t, time.UnixMilli(0).Add(-time.Hour*24).Unix(), tm.Unix()) + require.Equal(t, "1969-12-31 01:00:00", tm.In(tzBerlin).Format(time.DateTime)) + require.Same(t, tzBerlin, tm.Location()) - _, err = se.ExecuteSQL(context.TODO(), "SET @@global.time_zone = 'Europe/Berlin'") + tm, err = cache.EvalExpireTime(time.UnixMilli(0).In(tzShanghai), "3", ast.TimeUnitMonth) require.NoError(t, err) - tm, err = ttlTbl2.EvalExpireTime(context.TODO(), se, now) + require.Equal(t, "1969-10-01 08:00:00", tm.In(tzShanghai).Format(time.DateTime)) + require.Same(t, tzShanghai, tm.Location()) + + tm, err = cache.EvalExpireTime(time.UnixMilli(0).In(tzBerlin), "3", ast.TimeUnitMonth) require.NoError(t, err) - require.Equal(t, "1969-10-01 01:00:00", tm.In(tz2).Format(time.DateTime)) - require.Same(t, now.Location(), tm.Location()) + require.Equal(t, "1969-10-01 01:00:00", tm.In(tzBerlin).Format(time.DateTime)) + require.Same(t, tzBerlin, tm.Location()) // test cases for daylight saving time. // When local standard time was about to reach Sunday, 10 March 2024, 02:00:00 clocks were turned forward 1 hour to // Sunday, 10 March 2024, 03:00:00 local daylight time instead. - tz3, err := time.LoadLocation("America/Los_Angeles") - require.NoError(t, err) - now, err = time.ParseInLocation(time.DateTime, "2024-03-11 19:49:59", tz3) - require.NoError(t, err) - _, err = se.ExecuteSQL(context.TODO(), "SET @@global.time_zone = 'America/Los_Angeles'") + tzLosAngeles, err := time.LoadLocation("America/Los_Angeles") require.NoError(t, err) - tk.MustExec("create table test.t3(a int, t datetime) ttl = `t` + interval 90 minute") - tb3, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t3")) + now, err := time.ParseInLocation(time.DateTime, "2024-03-11 19:49:59", tzLosAngeles) require.NoError(t, err) - tblInfo3 := tb3.Meta() - ttlTbl3, err := cache.NewPhysicalTable(model.NewCIStr("test"), tblInfo3, model.NewCIStr("")) + tm, err = cache.EvalExpireTime(now, "90", ast.TimeUnitMinute) require.NoError(t, err) - require.NoError(t, err) - tm, err = ttlTbl3.EvalExpireTime(context.TODO(), se, now) - require.NoError(t, err) - require.Equal(t, "2024-03-11 18:19:59", tm.In(tz3).Format(time.DateTime)) - require.Same(t, now.Location(), tm.Location()) + require.Equal(t, "2024-03-11 18:19:59", tm.Format(time.DateTime)) + require.Same(t, tzLosAngeles, tm.Location()) // across day light-saving time - now, err = time.ParseInLocation(time.DateTime, "2024-03-10 03:01:00", tz3) + now, err = time.ParseInLocation(time.DateTime, "2024-03-10 03:01:00", tzLosAngeles) require.NoError(t, err) - tm, err = ttlTbl3.EvalExpireTime(context.TODO(), se, now) + tm, err = cache.EvalExpireTime(now, "90", ast.TimeUnitMinute) require.NoError(t, err) - require.Equal(t, "2024-03-10 00:31:00", tm.In(tz3).Format(time.DateTime)) - require.Same(t, now.Location(), tm.Location()) + require.Equal(t, "2024-03-10 00:31:00", tm.Format(time.DateTime)) + require.Same(t, tzLosAngeles, tm.Location()) - now, err = time.ParseInLocation(time.DateTime, "2024-03-10 04:01:00", tz3) + now, err = time.ParseInLocation(time.DateTime, "2024-03-10 04:01:00", tzLosAngeles) require.NoError(t, err) - tm, err = ttlTbl3.EvalExpireTime(context.TODO(), se, now) + tm, err = cache.EvalExpireTime(now, "90", ast.TimeUnitMinute) require.NoError(t, err) - require.Equal(t, "2024-03-10 01:31:00", tm.In(tz3).Format(time.DateTime)) - require.Same(t, now.Location(), tm.Location()) + require.Equal(t, "2024-03-10 01:31:00", tm.Format(time.DateTime)) + require.Same(t, tzLosAngeles, tm.Location()) - now, err = time.ParseInLocation(time.DateTime, "2024-11-03 03:00:00", tz3) + now, err = time.ParseInLocation(time.DateTime, "2024-11-03 03:00:00", tzLosAngeles) require.NoError(t, err) - tm, err = ttlTbl3.EvalExpireTime(context.TODO(), se, now) + tm, err = cache.EvalExpireTime(now, "90", ast.TimeUnitMinute) require.NoError(t, err) - require.Equal(t, "2024-11-03 01:30:00", tm.In(tz3).Format(time.DateTime)) - require.Same(t, now.Location(), tm.Location()) + require.Equal(t, "2024-11-03 01:30:00", tm.Format(time.DateTime)) + require.Same(t, tzLosAngeles, tm.Location()) // 2024-11-03 01:30:00 in America/Los_Angeles has two related time points: // 2024-11-03 01:30:00 -0700 PDT // 2024-11-03 01:30:00 -0800 PST // We must use the earlier one to avoid deleting some unexpected rows. require.Equal(t, int64(5400), now.Unix()-tm.Unix()) - // we should use global time zone to calculate the expired time - _, err = se.ExecuteSQL(context.TODO(), "SET @@global.time_zone = 'Asia/Shanghai'") - require.NoError(t, err) - now, err = time.ParseInLocation(time.DateTime, "1999-02-28 16:00:00", time.UTC) - require.NoError(t, err) - tm, err = ttlTbl2.EvalExpireTime(context.TODO(), se, now) - require.NoError(t, err) - require.Equal(t, "1998-12-01 00:00:00", tm.In(tz1).Format(time.DateTime)) - require.Same(t, time.UTC, tm.Location()) - // time should be truncated to second to make the result simple now, err = time.ParseInLocation("2006-01-02 15:04:05.000000", "2023-01-02 15:00:01.986542", time.UTC) require.NoError(t, err) - tm, err = ttlTbl.EvalExpireTime(context.TODO(), se, now) + tm, err = cache.EvalExpireTime(now, "1", ast.TimeUnitDay) require.NoError(t, err) require.Equal(t, "2023-01-01 15:00:01.000000", tm.Format("2006-01-02 15:04:05.000000")) require.Same(t, time.UTC, tm.Location()) // test for string interval format - tk.MustExec("create table test.t4(a int, t datetime) ttl = `t` + interval '1:3' hour_minute") - tb4, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t4")) - require.NoError(t, err) - tblInfo4 := tb4.Meta() - ttlTbl4, err := cache.NewPhysicalTable(model.NewCIStr("test"), tblInfo4, model.NewCIStr("")) - require.NoError(t, err) - tm, err = ttlTbl4.EvalExpireTime(context.TODO(), se, time.Unix(0, 0).In(tz2)) + tm, err = cache.EvalExpireTime(time.Unix(0, 0).In(tzBerlin), "'1:3'", ast.TimeUnitHourMinute) require.NoError(t, err) require.Equal(t, "1969-12-31 22:57:00", tm.In(time.UTC).Format(time.DateTime)) - require.Same(t, tz2, tm.Location()) - - // session time zone should keep unchanged - tk.MustQuery("select @@time_zone").Check(testkit.Rows("Asia/Tokyo")) + require.Same(t, tzBerlin, tm.Location()) }