Skip to content

Commit

Permalink
metrics, ddl, server: Add metrics for ddl and server (pingcap#5840)
Browse files Browse the repository at this point in the history
DDL: is owner
Server events: start/stop/kill/hang
  • Loading branch information
shenli authored Feb 9, 2018
1 parent b05bbc5 commit eb71063
Show file tree
Hide file tree
Showing 6 changed files with 46 additions and 6 deletions.
4 changes: 3 additions & 1 deletion ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -252,7 +252,6 @@ func newDDL(ctx goctx.Context, etcdCli *clientv3.Client, store kv.Storage,
if hook == nil {
hook = &BaseCallback{}
}

id := uuid.NewV4().String()
ctx, cancelFunc := goctx.WithCancel(ctx)
var manager owner.Manager
Expand Down Expand Up @@ -291,7 +290,9 @@ func newDDL(ctx goctx.Context, etcdCli *clientv3.Client, store kv.Storage,

d.start(ctx)
variable.RegisterStatistics(d)

log.Infof("[ddl] start DDL:%s", d.uuid)
metrics.DDLCounter.WithLabelValues(metrics.CreateDDL).Inc()
return d
}

Expand All @@ -315,6 +316,7 @@ func (d *ddl) start(ctx goctx.Context) {
terror.Log(errors.Trace(err))
d.wait.Add(1)
go d.onDDLWorker()
metrics.DDLCounter.WithLabelValues(metrics.CreateDDLWorker).Inc()
}

// For every start, we will send a fake job to let worker
Expand Down
6 changes: 3 additions & 3 deletions ddl/ddl_db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -332,9 +332,9 @@ func (s *testStateChangeSuite) TestShowIndex(c *C) {
}
switch job.SchemaState {
case model.StateDeleteOnly, model.StateWriteOnly, model.StateWriteReorganization:
result, err := s.execQuery(tk, showIndexSQL)
if err != nil {
checkErr = err
result, err1 := s.execQuery(tk, showIndexSQL)
if err1 != nil {
checkErr = err1
break
}
checkErr = checkResult(result, testkit.Rows("t 0 PRIMARY 1 c1 A 0 <nil> <nil> BTREE "))
Expand Down
3 changes: 3 additions & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,9 @@ func cleanNotify(ch chan struct{}) {
func (d *ddl) isOwner() bool {
isOwner := d.ownerManager.IsOwner()
log.Debugf("[ddl] it's the job owner %v, self id %s", isOwner, d.uuid)
if isOwner {
metrics.DDLCounter.WithLabelValues(metrics.IsDDLOwner).Inc()
}
return isOwner
}

Expand Down
12 changes: 12 additions & 0 deletions metrics/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,17 @@ var (
Help: "Bucketed histogram of processing time (s) of ddl worker operations",
Buckets: prometheus.ExponentialBuckets(0.001, 2, 20),
}, []string{"op", "result_state"})

CreateDDL = "create_ddl"
CreateDDLWorker = "create_ddl_worker"
IsDDLOwner = "is_ddl_owner"
DDLCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "tidb",
Subsystem: "ddl",
Name: "ddl_counter",
Help: "Counter of creating ddl/worker and isowner.",
}, []string{"type"})
)

func init() {
Expand All @@ -99,4 +110,5 @@ func init() {
prometheus.MustRegister(UpdateSelfVersionHistogram)
prometheus.MustRegister(OwnerHandleSyncerHistogram)
prometheus.MustRegister(DDLWorkerHistogram)
prometheus.MustRegister(DDLCounter)
}
16 changes: 16 additions & 0 deletions metrics/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,21 @@ var (
Name: "critical_error",
Help: "Counter of critical errors.",
})

EventStart = "server_start"
EventGracefulDown = "server_graceful_shutdown"
// Eventkill occurs when the server.Kill() function is called.
EventKill = "server_kill"
// EventHang occurs when server meet some critical error. It will close the listening port and hang for ever.
EventHang = "server_hang"
EventClose = "server_close"
ServerEventCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "tidb",
Subsystem: "server",
Name: "server_event",
Help: "Counter of tidb-server event.",
}, []string{"type"})
)

func init() {
Expand All @@ -71,6 +86,7 @@ func init() {
prometheus.MustRegister(ConnGauge)
prometheus.MustRegister(ExecuteErrorCounter)
prometheus.MustRegister(CriticalErrorCounter)
prometheus.MustRegister(ServerEventCounter)
}

// ExecuteErrorToLabel converts an execute error to label.
Expand Down
11 changes: 9 additions & 2 deletions server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ type Server struct {
clients map[uint32]*clientConn
capability uint32

// When a critical error occurred, we don't want to exit the process, because there may be
// stopListenerCh is used when a critical error occurred, we don't want to exit the process, because there may be
// a supervisor automatically restart it, then new client connection will be created, but we can't server it.
// So we just stop the listener and store to force clients to chose other TiDB servers.
stopListenerCh chan struct{}
Expand Down Expand Up @@ -226,6 +226,8 @@ func (s *Server) loadTLSCertificates() {

// Run runs the server.
func (s *Server) Run() error {
metrics.ServerEventCounter.WithLabelValues(metrics.EventStart).Inc()

// Start HTTP API to report tidb info such as TPS.
if s.cfg.Status.ReportStatus {
s.startStatusHTTP()
Expand Down Expand Up @@ -259,6 +261,7 @@ func (s *Server) Run() error {
terror.Log(errors.Trace(err))
s.listener = nil
for {
metrics.ServerEventCounter.WithLabelValues(metrics.EventHang).Inc()
log.Errorf("listener stopped, waiting for manual kill.")
time.Sleep(time.Minute)
}
Expand All @@ -283,6 +286,7 @@ func (s *Server) Close() {
terror.Log(errors.Trace(err))
s.listener = nil
}
metrics.ServerEventCounter.WithLabelValues(metrics.EventClose).Inc()
}

// onConn runs in its own goroutine, handles queries from this connection.
Expand Down Expand Up @@ -328,6 +332,8 @@ func (s *Server) ShowProcessList() []util.ProcessInfo {
func (s *Server) Kill(connectionID uint64, query bool) {
s.rwlock.Lock()
defer s.rwlock.Unlock()
log.Infof("[server] Kill connectionID %d, query %t]", connectionID, query)
metrics.ServerEventCounter.WithLabelValues(metrics.EventKill).Inc()

conn, ok := s.clients[uint32(connectionID)]
if !ok {
Expand All @@ -350,7 +356,8 @@ func (s *Server) Kill(connectionID uint64, query bool) {

// GracefulDown waits all clients to close.
func (s *Server) GracefulDown() {
log.Info("graceful shutdown.")
log.Info("[server] graceful shutdown.")
metrics.ServerEventCounter.WithLabelValues(metrics.EventGracefulDown).Inc()

count := s.ConnectionCount()
for i := 0; count > 0; i++ {
Expand Down

0 comments on commit eb71063

Please sign in to comment.