diff --git a/executor/admin.go b/executor/admin.go index 572eb451adc4b..b980bef82d3f5 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" log "github.com/sirupsen/logrus" @@ -125,7 +126,7 @@ func (e *CheckIndexRangeExec) Open(ctx context.Context) error { func (e *CheckIndexRangeExec) buildDAGPB() (*tipb.DAGRequest, error) { dagReq := &tipb.DAGRequest{} dagReq.StartTs = e.ctx.Txn().StartTS() - dagReq.TimeZoneName, dagReq.TimeZoneOffset = zone(e.ctx) + dagReq.TimeZoneName, dagReq.TimeZoneOffset = timeutil.Zone(e.ctx.GetSessionVars().Location()) sc := e.ctx.GetSessionVars().StmtCtx dagReq.Flags = statementContextToFlags(sc) for i := range e.schema.Columns { @@ -223,7 +224,7 @@ func (e *RecoverIndexExec) constructLimitPB(count uint64) *tipb.Executor { func (e *RecoverIndexExec) buildDAGPB(txn kv.Transaction, limitCnt uint64) (*tipb.DAGRequest, error) { dagReq := &tipb.DAGRequest{} dagReq.StartTs = txn.StartTS() - dagReq.TimeZoneName, dagReq.TimeZoneOffset = zone(e.ctx) + dagReq.TimeZoneName, dagReq.TimeZoneOffset = timeutil.Zone(e.ctx.GetSessionVars().Location()) sc := e.ctx.GetSessionVars().StmtCtx dagReq.Flags = statementContextToFlags(sc) for i := range e.columns { @@ -651,7 +652,7 @@ func (e *CleanupIndexExec) Open(ctx context.Context) error { func (e *CleanupIndexExec) buildIdxDAGPB(txn kv.Transaction) (*tipb.DAGRequest, error) { dagReq := &tipb.DAGRequest{} dagReq.StartTs = txn.StartTS() - dagReq.TimeZoneName, dagReq.TimeZoneOffset = zone(e.ctx) + dagReq.TimeZoneName, dagReq.TimeZoneOffset = timeutil.Zone(e.ctx.GetSessionVars().Location()) sc := e.ctx.GetSessionVars().StmtCtx dagReq.Flags = statementContextToFlags(sc) for i := range e.idxCols { diff --git a/executor/builder.go b/executor/builder.go index 51b6c1773814a..eb5c3bcf8949a 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" "golang.org/x/net/context" @@ -1320,7 +1321,7 @@ func (b *executorBuilder) buildDelete(v *plan.Delete) Executor { } func (b *executorBuilder) buildAnalyzeIndexPushdown(task plan.AnalyzeIndexTask, maxNumBuckets uint64) *AnalyzeIndexExec { - _, offset := zone(b.ctx) + _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) e := &AnalyzeIndexExec{ ctx: b.ctx, physicalTableID: task.PhysicalTableID, @@ -1353,7 +1354,7 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plan.AnalyzeColumnsTa cols = append([]*model.ColumnInfo{task.PKInfo}, cols...) } - _, offset := zone(b.ctx) + _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) e := &AnalyzeColumnsExec{ ctx: b.ctx, physicalTableID: task.PhysicalTableID, @@ -1430,7 +1431,7 @@ func constructDistExec(sctx sessionctx.Context, plans []plan.PhysicalPlan) ([]*t func (b *executorBuilder) constructDAGReq(plans []plan.PhysicalPlan) (dagReq *tipb.DAGRequest, streaming bool, err error) { dagReq = &tipb.DAGRequest{} dagReq.StartTs = b.getStartTS() - dagReq.TimeZoneName, dagReq.TimeZoneOffset = zone(b.ctx) + dagReq.TimeZoneName, dagReq.TimeZoneOffset = timeutil.Zone(b.ctx.GetSessionVars().Location()) sc := b.ctx.GetSessionVars().StmtCtx dagReq.Flags = statementContextToFlags(sc) dagReq.Executors, streaming, err = constructDistExec(b.ctx, plans) diff --git a/executor/distsql.go b/executor/distsql.go index 115afa73694f6..e90cfd015551d 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -19,7 +19,6 @@ import ( "sort" "sync" "sync/atomic" - "time" "unsafe" "github.com/pingcap/tidb/distsql" @@ -117,22 +116,6 @@ func closeAll(objs ...Closeable) error { return errors.Trace(err) } -// zone returns the current timezone name and timezone offset in seconds. -// In compatible with MySQL, we change `Local` to `System`. -// TODO: Golang team plan to return system timezone name intead of -// returning `Local` when `loc` is `time.Local`. We need keep an eye on this. -func zone(sctx sessionctx.Context) (string, int64) { - loc := sctx.GetSessionVars().Location() - _, offset := time.Now().In(loc).Zone() - var name string - name = loc.String() - if name == "Local" { - name = "System" - } - - return name, int64(offset) -} - // statementContextToFlags converts StatementContext to tipb.SelectRequest.Flags. func statementContextToFlags(sc *stmtctx.StatementContext) uint64 { var flags uint64 diff --git a/executor/executor_test.go b/executor/executor_test.go index e45effd90f412..48e5f7935493d 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -26,6 +26,7 @@ import ( "time" gofail "github.com/etcd-io/gofail/runtime" + "github.com/golang/protobuf/proto" . "github.com/pingcap/check" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" @@ -58,6 +59,8 @@ import ( "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tidb/util/timeutil" + "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" "golang.org/x/net/context" ) @@ -2357,6 +2360,32 @@ func (s *testContextOptionSuite) TestCoprocessorPriority(c *C) { cli.mu.Unlock() } +func (s *testSuite) TestTimezonePushDown(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t (ts timestamp)") + defer tk.MustExec("drop table t") + tk.MustExec(`insert into t values ("2018-09-13 10:02:06")`) + + systemTZ := timeutil.SystemLocation() + c.Assert(systemTZ.String(), Not(Equals), "System") + ctx := context.Background() + count := 0 + ctx1 := context.WithValue(ctx, "CheckSelectRequestHook", func(req *kv.Request) { + count += 1 + dagReq := new(tipb.DAGRequest) + err := proto.Unmarshal(req.Data, dagReq) + c.Assert(err, IsNil) + c.Assert(dagReq.GetTimeZoneName(), Equals, systemTZ.String()) + }) + tk.Se.Execute(ctx1, `select * from t where ts = "2018-09-13 10:02:06"`) + + tk.MustExec(`set time_zone="System"`) + tk.Se.Execute(ctx1, `select * from t where ts = "2018-09-13 10:02:06"`) + + c.Assert(count, Equals, 2) // Make sure the hook function is called. +} + func (s *testSuite) TestNotFillCacheFlag(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 437094bce1d44..f2a46abb302eb 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tidb/util/timeutil" "github.com/pkg/errors" ) @@ -997,7 +998,7 @@ func (s *testEvaluatorSuite) TestSysDate(c *C) { fc := funcs[ast.Sysdate] ctx := mock.NewContext() - ctx.GetSessionVars().StmtCtx.TimeZone = time.Local + ctx.GetSessionVars().StmtCtx.TimeZone = timeutil.SystemLocation() timezones := []types.Datum{types.NewDatum(1234), types.NewDatum(0)} for _, timezone := range timezones { // sysdate() result is not affected by "timestamp" session variable. diff --git a/session/bootstrap.go b/session/bootstrap.go index dfa65909e98ae..62986e734e58f 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/timeutil" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -233,7 +234,10 @@ const ( bootstrappedVarTrue = "True" // The variable name in mysql.TiDB table. // It is used for getting the version of the TiDB server which bootstrapped the store. - tidbServerVersionVar = "tidb_server_version" // + tidbServerVersionVar = "tidb_server_version" + // The variable name in mysql.tidb table and it will be used when we want to know + // system timezone. + tidbSystemTZ = "system_tz" // Const for TiDB server version 2. version2 = 2 version3 = 3 @@ -257,6 +261,7 @@ const ( version21 = 21 version22 = 22 version23 = 23 + version24 = 24 ) func checkBootstrapped(s Session) (bool, error) { @@ -407,6 +412,10 @@ func upgrade(s Session) { upgradeToVer23(s) } + if ver < version24 { + upgradeToVer24(s) + } + updateBootstrapVer(s) _, err = s.Execute(context.Background(), "COMMIT") @@ -649,6 +658,18 @@ func upgradeToVer23(s Session) { doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `flag` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } +// writeSystemTZ writes system timezone info into mysql.tidb +func writeSystemTZ(s Session) { + sql := fmt.Sprintf(`INSERT HIGH_PRIORITY INTO %s.%s VALUES ("%s", "%s", "TiDB Global System Timezone.") ON DUPLICATE KEY UPDATE VARIABLE_VALUE="%s"`, + mysql.SystemDB, mysql.TiDBTable, tidbSystemTZ, timeutil.InferSystemTZ(), timeutil.InferSystemTZ()) + mustExecute(s, sql) +} + +// upgradeToVer24 initializes `System` timezone according to docs/design/2018-09-10-adding-tz-env.md +func upgradeToVer24(s Session) { + writeSystemTZ(s) +} + // updateBootstrapVer updates bootstrap version variable in mysql.TiDB table. func updateBootstrapVer(s Session) { // Update bootstrap version. @@ -732,6 +753,7 @@ func doDMLWorks(s Session) { mysql.SystemDB, mysql.TiDBTable, tidbServerVersionVar, currentBootstrapVersion) mustExecute(s, sql) + writeSystemTZ(s) _, err := s.Execute(context.Background(), "COMMIT") if err != nil { time.Sleep(1 * time.Second) diff --git a/session/session.go b/session/session.go index 4c47455187439..6ced52fe474cc 100644 --- a/session/session.go +++ b/session/session.go @@ -53,6 +53,7 @@ import ( "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/kvcache" + "github.com/pingcap/tidb/util/timeutil" binlog "github.com/pingcap/tipb/go-binlog" "github.com/pkg/errors" log "github.com/sirupsen/logrus" @@ -1068,6 +1069,20 @@ func CreateSession(store kv.Storage) (Session, error) { return s, nil } +// loadSystemTZ loads systemTZ from mysql.tidb +func loadSystemTZ(se *session) (string, error) { + sql := `select variable_value from mysql.tidb where variable_name = "system_tz"` + rss, errLoad := se.Execute(context.Background(), sql) + if errLoad != nil { + return "", errLoad + } + // the record of mysql.tidb under where condition: variable_name = "system_tz" should shall only be one. + defer rss[0].Close() + chk := rss[0].NewChunk() + rss[0].Next(context.Background(), chk) + return chk.GetRow(0).GetString(0), nil +} + // BootstrapSession runs the first time when the TiDB server start. func BootstrapSession(store kv.Storage) (*domain.Domain, error) { ver := getStoreBootstrapVersion(store) @@ -1081,11 +1096,20 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { if err != nil { return nil, errors.Trace(err) } + // get system tz from mysql.tidb + tz, err := loadSystemTZ(se) + if err != nil { + return nil, errors.Trace(err) + } + + timeutil.SetSystemTZ(tz) + dom := domain.GetDomain(se) err = dom.LoadPrivilegeLoop(se) if err != nil { return nil, errors.Trace(err) } + se1, err := createSession(store) if err != nil { return nil, errors.Trace(err) @@ -1180,7 +1204,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er const ( notBootstrapped = 0 - currentBootstrapVersion = 23 + currentBootstrapVersion = 24 ) func getStoreBootstrapVersion(store kv.Storage) int64 { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index b0454332fe625..b7896929d2484 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/timeutil" "github.com/pkg/errors" ) @@ -426,7 +427,7 @@ func (s *SessionVars) GetNextPreparedStmtID() uint32 { func (s *SessionVars) Location() *time.Location { loc := s.TimeZone if loc == nil { - loc = time.Local + loc = timeutil.SystemLocation() } return loc } diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 176e99ebacaa2..2cc8a1cf3b682 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/timeutil" "github.com/pkg/errors" ) @@ -350,8 +351,7 @@ func tidbOptInt64(opt string, defaultVal int64) int64 { func parseTimeZone(s string) (*time.Location, error) { if strings.EqualFold(s, "SYSTEM") { - // TODO: Support global time_zone variable, it should be set to global time_zone value. - return time.Local, nil + return timeutil.SystemLocation(), nil } loc, err := time.LoadLocation(s) diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index 2f2af9687e03a..0d874793e121b 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -15,9 +15,7 @@ package mocktikv import ( "bytes" - "fmt" "io" - "sync" "time" "github.com/golang/protobuf/proto" @@ -36,6 +34,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" mockpkg "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" "golang.org/x/net/context" @@ -45,50 +44,6 @@ import ( var dummySlice = make([]byte, 0) -// locCache is a simple map with lock. It stores all used timezone during the lifetime of tidb instance. -// Talked with Golang team about whether they can have some forms of cache policy available for programmer, -// they suggests that only programmers knows which one is best for their use case. -// For detail, please refer to: https://github.com/golang/go/issues/26106 -type locCache struct { - sync.RWMutex - // locMap stores locations used in past and can be retrieved by a timezone's name. - locMap map[string]*time.Location -} - -// init initializes `locCache`. -func init() { - LocCache = &locCache{} - LocCache.locMap = make(map[string]*time.Location) -} - -// LocCache is a simple cache policy to improve the performance of 'time.LoadLocation'. -var LocCache *locCache - -// getLoc first trying to load location from a cache map. If nothing found in such map, then call -// `time.LocadLocation` to get a timezone location. After trying both way, an error wil be returned -// if valid Location is not found. -func (lm *locCache) getLoc(name string) (*time.Location, error) { - if name == "System" { - name = "Local" - } - lm.RLock() - if v, ok := lm.locMap[name]; ok { - lm.RUnlock() - return v, nil - } - - if loc, err := time.LoadLocation(name); err == nil { - lm.RUnlock() - lm.Lock() - lm.locMap[name] = loc - lm.Unlock() - return loc, nil - } - - lm.RUnlock() - return nil, fmt.Errorf("invalid name for timezone %s", name) -} - type dagContext struct { dagReq *tipb.DAGRequest keyRanges []*coprocessor.KeyRange @@ -169,8 +124,9 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex // timezone offset in seconds east of UTC is used to constructed the timezone. func constructTimeZone(name string, offset int) (*time.Location, error) { if name != "" { - return LocCache.getLoc(name) + return timeutil.LoadLocation(name) } + return time.FixedZone("", offset), nil } diff --git a/util/timeutil/time.go b/util/timeutil/time.go new file mode 100644 index 0000000000000..4aa212e232172 --- /dev/null +++ b/util/timeutil/time.go @@ -0,0 +1,159 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package timeutil + +import ( + "errors" + "fmt" + "os" + "path/filepath" + "strings" + "sync" + "syscall" + "time" + + log "github.com/sirupsen/logrus" +) + +// init initializes `locCache`. +func init() { + // We need set systemTZ when it is in testing process. + if systemTZ == "" { + systemTZ = "System" + } + locCa = &locCache{} + locCa.locMap = make(map[string]*time.Location) +} + +// locCa is a simple cache policy to improve the performance of 'time.LoadLocation'. +var locCa *locCache + +// systemTZ is current TiDB's system timezone name. +var systemTZ string +var zoneSources = []string{ + "/usr/share/zoneinfo/", + "/usr/share/lib/zoneinfo/", + "/usr/lib/locale/TZ/", + // this is for macOS + "/var/db/timezone/zoneinfo", +} + +// locCache is a simple map with lock. It stores all used timezone during the lifetime of tidb instance. +// Talked with Golang team about whether they can have some forms of cache policy available for programmer, +// they suggests that only programmers knows which one is best for their use case. +// For detail, please refer to: https://github.com/golang/go/issues/26106 +type locCache struct { + sync.RWMutex + // locMap stores locations used in past and can be retrieved by a timezone's name. + locMap map[string]*time.Location +} + +// InferSystemTZ reads system timezone from `TZ`, the path of the soft link of `/etc/localtime`. If both of them are failed, system timezone will be set to `UTC`. +// It is exported because we need to use it during bootstap stage. And it should be only used at that stage. +func InferSystemTZ() string { + // consult $TZ to find the time zone to use. + // no $TZ means use the system default /etc/localtime. + // $TZ="" means use UTC. + // $TZ="foo" means use /usr/share/zoneinfo/foo. + tz, ok := syscall.Getenv("TZ") + switch { + case !ok: + path, err1 := filepath.EvalSymlinks("/etc/localtime") + if err1 == nil { + name, err2 := inferTZNameFromFileName(path) + if err2 == nil { + return name + } + log.Errorln(err2) + } + log.Errorln(err1) + case tz != "" && tz != "UTC": + for _, source := range zoneSources { + if _, err := os.Stat(source + tz); err == nil { + return tz + } + } + } + return "UTC" +} + +// inferTZNameFromFileName gets IANA timezone name from zoneinfo path. +// TODO: It will be refined later. This is just a quick fix. +func inferTZNameFromFileName(path string) (string, error) { + // phase1 only support read /etc/localtime which is a softlink to zoneinfo file + substr := "zoneinfo" + if idx := strings.Index(path, substr); idx != -1 { + return string(path[idx+len(substr)+1:]), nil + } + return "", errors.New(fmt.Sprintf("path %s is not supported", path)) +} + +// SystemLocation returns time.SystemLocation's IANA timezone location. It is TiDB's global timezone location. +func SystemLocation() *time.Location { + loc, err := LoadLocation(systemTZ) + if err != nil { + return time.Local + } + return loc +} + +// SetSystemTZ sets systemTZ by the value loaded from mysql.tidb. +func SetSystemTZ(name string) { + systemTZ = name +} + +// getLoc first trying to load location from a cache map. If nothing found in such map, then call +// `time.LoadLocation` to get a timezone location. After trying both way, an error will be returned +// if valid Location is not found. +func (lm *locCache) getLoc(name string) (*time.Location, error) { + if name == "System" { + return time.Local, nil + } + lm.RLock() + v, ok := lm.locMap[name] + lm.RUnlock() + if ok { + return v, nil + } + + if loc, err := time.LoadLocation(name); err == nil { + // assign value back to map + lm.Lock() + lm.locMap[name] = loc + lm.Unlock() + return loc, nil + } + + return nil, fmt.Errorf("invalid name for timezone %s", name) +} + +// LoadLocation loads time.Location by IANA timezone time. +func LoadLocation(name string) (*time.Location, error) { + return locCa.getLoc(name) +} + +// Zone returns the current timezone name and timezone offset in seconds. +// In compatible with MySQL, we change `SystemLocation` to `System`. +func Zone(loc *time.Location) (string, int64) { + _, offset := time.Now().In(loc).Zone() + var name string + name = loc.String() + // when we found name is "SystemLocation", we have no chice but push down + // "System" to tikv side. + if name == "Local" { + name = "System" + } + + return name, int64(offset) +} diff --git a/util/timeutil/time_test.go b/util/timeutil/time_test.go new file mode 100644 index 0000000000000..e23e42f73503d --- /dev/null +++ b/util/timeutil/time_test.go @@ -0,0 +1,60 @@ +// Copyright 2018 PingCAP, Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSES/QL-LICENSE file. + +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package timeutil + +import ( + "os" + "testing" + + . "github.com/pingcap/check" +) + +var _ = Suite(&testTimeSuite{}) + +func TestT(t *testing.T) { + TestingT(t) +} + +type testTimeSuite struct{} + +func (s *testTimeSuite) TestgetTZNameFromFileName(c *C) { + tz, err := inferTZNameFromFileName("/user/share/zoneinfo/Asia/Shanghai") + c.Assert(err, IsNil) + c.Assert(tz, Equals, "Asia/Shanghai") +} + +func (s *testTimeSuite) TestLocal(c *C) { + os.Setenv("TZ", "Asia/Shanghai") + systemTZ = InferSystemTZ() + loc := SystemLocation() + c.Assert(systemTZ, Equals, "Asia/Shanghai") + c.Assert(loc.String(), Equals, "Asia/Shanghai") + + os.Setenv("TZ", "UTC") + // reset systemTZ + systemTZ = InferSystemTZ() + loc = SystemLocation() + c.Assert(loc.String(), Equals, "UTC") + + os.Setenv("TZ", "") + // reset systemTZ + systemTZ = InferSystemTZ() + loc = SystemLocation() + c.Assert(loc.String(), Equals, "UTC") + os.Unsetenv("TZ") +}