Skip to content

Commit

Permalink
*: write system timezone into mysql.tidb in bootstrap stage. (#7638)
Browse files Browse the repository at this point in the history
  • Loading branch information
zhexuany authored and shenli committed Sep 14, 2018
1 parent a7f4687 commit 2944195
Show file tree
Hide file tree
Showing 12 changed files with 313 additions and 76 deletions.
7 changes: 4 additions & 3 deletions executor/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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 {
Expand Down
7 changes: 4 additions & 3 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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)
Expand Down
17 changes: 0 additions & 17 deletions executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"sort"
"sync"
"sync/atomic"
"time"
"unsafe"

"github.com/pingcap/tidb/distsql"
Expand Down Expand Up @@ -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
Expand Down
29 changes: 29 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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"
)
Expand Down Expand Up @@ -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")
Expand Down
3 changes: 2 additions & 1 deletion expression/builtin_time_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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.
Expand Down
24 changes: 23 additions & 1 deletion session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand All @@ -257,6 +261,7 @@ const (
version21 = 21
version22 = 22
version23 = 23
version24 = 24
)

func checkBootstrapped(s Session) (bool, error) {
Expand Down Expand Up @@ -407,6 +412,10 @@ func upgrade(s Session) {
upgradeToVer23(s)
}

if ver < version24 {
upgradeToVer24(s)
}

updateBootstrapVer(s)
_, err = s.Execute(context.Background(), "COMMIT")

Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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)
Expand Down
26 changes: 25 additions & 1 deletion session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
Expand All @@ -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)
Expand Down Expand Up @@ -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 {
Expand Down
3 changes: 2 additions & 1 deletion sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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
}
Expand Down
4 changes: 2 additions & 2 deletions sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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)
Expand Down
Loading

0 comments on commit 2944195

Please sign in to comment.