diff --git a/domain/infosync/info.go b/domain/infosync/info.go index fc58783ff3108..8aec3c35275e8 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -856,7 +856,7 @@ func getServerInfo(id string, serverIDGetter func() uint64) *ServerInfo { failpoint.Inject("mockServerInfo", func(val failpoint.Value) { if val.(bool) { - info.StartTimestamp = 1282967700000 + info.StartTimestamp = 1282967700 info.Labels = map[string]string{ "foo": "bar", } diff --git a/domain/infosync/info_test.go b/domain/infosync/info_test.go index 001a106632230..bb0ded60e7e25 100644 --- a/domain/infosync/info_test.go +++ b/domain/infosync/info_test.go @@ -72,7 +72,7 @@ func TestTopology(t *testing.T) { topology, err := info.getTopologyFromEtcd(ctx) require.NoError(t, err) - require.Equal(t, int64(1282967700000), topology.StartTimestamp) + require.Equal(t, int64(1282967700), topology.StartTimestamp) v, ok := topology.Labels["foo"] require.True(t, ok) @@ -97,7 +97,7 @@ func TestTopology(t *testing.T) { dir := path.Dir(s) require.Equal(t, dir, topology.DeployPath) - require.Equal(t, int64(1282967700000), topology.StartTimestamp) + require.Equal(t, int64(1282967700), topology.StartTimestamp) require.Equal(t, info.getTopologyInfo(), *topology) // check ttl key diff --git a/server/conn.go b/server/conn.go index e9325a6dd97d6..ede5a12bf276d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -59,6 +59,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" @@ -1374,11 +1375,21 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { } func (cc *clientConn) writeStats(ctx context.Context) error { - msg := []byte("Uptime: 0 Threads: 0 Questions: 0 Slow queries: 0 Opens: 0 Flush tables: 0 Open tables: 0 Queries per second avg: 0.000") + var err error + var uptime int64 = 0 + info := serverInfo{} + info.ServerInfo, err = infosync.GetServerInfo() + if err != nil { + logutil.BgLogger().Error("Failed to get ServerInfo for uptime status", zap.Error(err)) + } else { + uptime = int64(time.Since(time.Unix(info.ServerInfo.StartTimestamp, 0)).Seconds()) + } + msg := []byte(fmt.Sprintf("Uptime: %d Threads: 0 Questions: 0 Slow queries: 0 Opens: 0 Flush tables: 0 Open tables: 0 Queries per second avg: 0.000", + uptime)) data := cc.alloc.AllocWithLen(4, len(msg)) data = append(data, msg...) - err := cc.writePacket(data) + err = cc.writePacket(data) if err != nil { return err } diff --git a/server/stat.go b/server/stat.go index 9725a7ec5e480..382a68e701ce0 100644 --- a/server/stat.go +++ b/server/stat.go @@ -16,7 +16,9 @@ package server import ( "crypto/x509" + "time" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" @@ -25,11 +27,13 @@ import ( var ( serverNotAfter = "Ssl_server_not_after" serverNotBefore = "Ssl_server_not_before" + upTime = "Uptime" ) var defaultStatus = map[string]*variable.StatusVal{ serverNotAfter: {Scope: variable.ScopeGlobal | variable.ScopeSession, Value: ""}, serverNotBefore: {Scope: variable.ScopeGlobal | variable.ScopeSession, Value: ""}, + upTime: {Scope: variable.ScopeGlobal, Value: 0}, } // GetScope gets the status variables scope. @@ -57,5 +61,15 @@ func (s *Server) Stats(vars *variable.SessionVars) (map[string]interface{}, erro } } } + + var err error + info := serverInfo{} + info.ServerInfo, err = infosync.GetServerInfo() + if err != nil { + logutil.BgLogger().Error("Failed to get ServerInfo for uptime status", zap.Error(err)) + } else { + m[upTime] = int64(time.Since(time.Unix(info.ServerInfo.StartTimestamp, 0)).Seconds()) + } + return m, nil } diff --git a/server/stat_test.go b/server/stat_test.go new file mode 100644 index 0000000000000..88fc2e0081a43 --- /dev/null +++ b/server/stat_test.go @@ -0,0 +1,61 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package server + +import ( + "context" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/mockstore" + "github.com/stretchr/testify/require" +) + +func TestUptime(t *testing.T) { + var err error + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/infosync/mockServerInfo", "return(true)")) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/domain/infosync/mockServerInfo") + require.NoError(t, err) + }() + + store, err := mockstore.NewMockStore() + require.NoError(t, err) + + dom, err := session.BootstrapSession(store) + defer func() { + dom.Close() + err := store.Close() + require.NoError(t, err) + }() + require.NoError(t, err) + + _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) + require.NoError(t, err) + + tidbdrv := NewTiDBDriver(store) + cfg := newTestConfig() + cfg.Socket = "" + server, err := NewServer(cfg, tidbdrv) + require.NoError(t, err) + + stats, err := server.Stats(nil) + require.NoError(t, err) + require.GreaterOrEqual(t, stats[upTime].(int64), int64(time.Since(time.Unix(1282967700, 0)).Seconds())) +}