Skip to content

Commit

Permalink
store/tikv: remove tidb/metrics from store/tikv (pingcap#23302)
Browse files Browse the repository at this point in the history
  • Loading branch information
aierui authored Mar 18, 2021
1 parent a3e8617 commit d370409
Show file tree
Hide file tree
Showing 5 changed files with 29 additions and 35 deletions.
27 changes: 0 additions & 27 deletions metrics/gprc.go

This file was deleted.

7 changes: 5 additions & 2 deletions metrics/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,9 +46,12 @@ const (
opSucc = "ok"
opFailed = "err"

TiDB = "tidb"
LabelScope = "scope"
ScopeGlobal = "global"
ScopeSession = "session"
Server = "server"
TiKVClient = "tikvclient"
)

// RetLabel returns "ok" when err == nil and "err" when err != nil.
Expand Down Expand Up @@ -139,14 +142,14 @@ func RegisterMetrics() {
prometheus.MustRegister(TotalCopProcHistogram)
prometheus.MustRegister(TotalCopWaitHistogram)
prometheus.MustRegister(HandleSchemaValidate)
prometheus.MustRegister(GRPCConnTransientFailureCounter)
prometheus.MustRegister(MaxProcs)
prometheus.MustRegister(GOGC)
prometheus.MustRegister(ConnIdleDurationHistogram)
prometheus.MustRegister(ServerInfo)
prometheus.MustRegister(TokenGauge)
prometheus.MustRegister(ConfigStatus)

tikvmetrics.InitMetrics("tidb", "tikvclient")
tikvmetrics.InitMetrics(TiDB, TiKVClient)
tikvmetrics.RegisterMetrics()
tikvmetrics.TiKVPanicCounter = PanicCounter // reset tidb metrics for tikv metrics
}
3 changes: 1 addition & 2 deletions store/tikv/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@ import (
"github.com/pingcap/kvproto/pkg/tikvpb"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/kv"
tidbmetrics "github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/store/tikv/config"
"github.com/pingcap/tidb/store/tikv/logutil"
"github.com/pingcap/tidb/store/tikv/metrics"
Expand Down Expand Up @@ -384,7 +383,7 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R
clientConn := connArray.Get()
if state := clientConn.GetState(); state == connectivity.TransientFailure {
storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10)
tidbmetrics.GRPCConnTransientFailureCounter.WithLabelValues(addr, storeID).Inc()
metrics.TiKVGRPCConnTransientFailureCounter.WithLabelValues(addr, storeID).Inc()
}

if req.IsDebugReq() {
Expand Down
7 changes: 3 additions & 4 deletions store/tikv/client_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/tikvpb"
"github.com/pingcap/parser/terror"
tidbmetrics "github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/store/tikv/config"
"github.com/pingcap/tidb/store/tikv/logutil"
"github.com/pingcap/tidb/store/tikv/metrics"
Expand Down Expand Up @@ -249,7 +248,7 @@ func (c *batchCommandsClient) send(request *tikvpb.BatchCommandsRequest, entries
func (c *batchCommandsClient) recv() (resp *tikvpb.BatchCommandsResponse, err error) {
defer func() {
if r := recover(); r != nil {
tidbmetrics.PanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
logutil.BgLogger().Error("batchCommandsClient.recv panic",
zap.Reflect("r", r),
zap.Stack("stack"))
Expand Down Expand Up @@ -331,7 +330,7 @@ func (c *batchCommandsClient) reCreateStreamingClientOnce(perr error) error {
func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransportLayerLoad *uint64) {
defer func() {
if r := recover(); r != nil {
tidbmetrics.PanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
logutil.BgLogger().Error("batchRecvLoop",
zap.Reflect("r", r),
zap.Stack("stack"))
Expand Down Expand Up @@ -448,7 +447,7 @@ func resetRequests(requests []*tikvpb.BatchCommandsRequest_Request) []*tikvpb.Ba
func (a *batchConn) batchSendLoop(cfg config.TiKVClient) {
defer func() {
if r := recover(); r != nil {
tidbmetrics.PanicCounter.WithLabelValues(metrics.LabelBatchSendLoop).Inc()
metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchSendLoop).Inc()
logutil.BgLogger().Error("batchSendLoop",
zap.Reflect("r", r),
zap.Stack("stack"))
Expand Down
20 changes: 20 additions & 0 deletions store/tikv/metrics/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,8 @@ var (
TiKVAsyncCommitTxnCounter *prometheus.CounterVec
TiKVOnePCTxnCounter *prometheus.CounterVec
TiKVStoreLimitErrorCounter *prometheus.CounterVec
TiKVGRPCConnTransientFailureCounter *prometheus.CounterVec
TiKVPanicCounter *prometheus.CounterVec
)

// Label constants.
Expand Down Expand Up @@ -355,6 +357,22 @@ func initMetrics(namespace, subsystem string) {
Help: "store token is up to the limit, probably because one of the stores is the hotspot or unavailable",
}, []string{LblAddress, LblStore})

TiKVGRPCConnTransientFailureCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "connection_transient_failure_count",
Help: "Counter of gRPC connection transient failure",
}, []string{LblAddress, LblStore})

TiKVPanicCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "panic_total",
Help: "Counter of panic.",
}, []string{LblType})

initShortcuts()
}

Expand Down Expand Up @@ -404,4 +422,6 @@ func RegisterMetrics() {
prometheus.MustRegister(TiKVAsyncCommitTxnCounter)
prometheus.MustRegister(TiKVOnePCTxnCounter)
prometheus.MustRegister(TiKVStoreLimitErrorCounter)
prometheus.MustRegister(TiKVGRPCConnTransientFailureCounter)
prometheus.MustRegister(TiKVPanicCounter)
}

0 comments on commit d370409

Please sign in to comment.