Skip to content

Commit

Permalink
server: Add uptime status var and statistics
Browse files Browse the repository at this point in the history
  • Loading branch information
dveeden committed Dec 21, 2021
1 parent 06c1e8c commit 4af54fa
Show file tree
Hide file tree
Showing 5 changed files with 91 additions and 5 deletions.
2 changes: 1 addition & 1 deletion domain/infosync/info.go
Original file line number Diff line number Diff line change
Expand Up @@ -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",
}
Expand Down
4 changes: 2 additions & 2 deletions domain/infosync/info_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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
Expand Down
15 changes: 13 additions & 2 deletions server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
}
Expand Down
14 changes: 14 additions & 0 deletions server/stat.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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.
Expand Down Expand Up @@ -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
}
61 changes: 61 additions & 0 deletions server/stat_test.go
Original file line number Diff line number Diff line change
@@ -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()))
}

0 comments on commit 4af54fa

Please sign in to comment.