tikv: reduce call of TiKVSendReqHistogram.WithLabelValues (#15297)
This commit is contained in:
@ -35,6 +35,7 @@ import (
|
||||
"github.com/pingcap/tidb/metrics"
|
||||
"github.com/pingcap/tidb/store/tikv/tikvrpc"
|
||||
"github.com/pingcap/tidb/util/logutil"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/backoff"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
@ -279,6 +280,30 @@ func (c *rpcClient) closeConns() {
|
||||
c.Unlock()
|
||||
}
|
||||
|
||||
var sendReqHistCache sync.Map
|
||||
|
||||
type sendReqHistCacheKey struct {
|
||||
tp tikvrpc.CmdType
|
||||
id uint64
|
||||
}
|
||||
|
||||
func (c *rpcClient) updateTiKVSendReqHistogram(req *tikvrpc.Request, start time.Time) {
|
||||
key := sendReqHistCacheKey{
|
||||
req.Type,
|
||||
req.Context.GetPeer().GetStoreId(),
|
||||
}
|
||||
|
||||
v, ok := sendReqHistCache.Load(key)
|
||||
if !ok {
|
||||
reqType := req.Type.String()
|
||||
storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10)
|
||||
v = metrics.TiKVSendReqHistogram.WithLabelValues(reqType, storeID)
|
||||
sendReqHistCache.Store(key, v)
|
||||
}
|
||||
|
||||
v.(prometheus.Observer).Observe(time.Since(start).Seconds())
|
||||
}
|
||||
|
||||
// SendRequest sends a Request to server and receives Response.
|
||||
func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) {
|
||||
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
|
||||
@ -288,11 +313,7 @@ func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R
|
||||
}
|
||||
|
||||
start := time.Now()
|
||||
reqType := req.Type.String()
|
||||
storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10)
|
||||
defer func() {
|
||||
metrics.TiKVSendReqHistogram.WithLabelValues(reqType, storeID).Observe(time.Since(start).Seconds())
|
||||
}()
|
||||
defer c.updateTiKVSendReqHistogram(req, start)
|
||||
|
||||
if atomic.CompareAndSwapUint32(&c.dieNotify, 1, 0) {
|
||||
c.recycleDieConnArray()
|
||||
@ -314,6 +335,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)
|
||||
metrics.GRPCConnTransientFailureCounter.WithLabelValues(addr, storeID).Inc()
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user