Support actively requesting update health feedback information by calling RPC to TiKV (#1287)

* Support actively requesting update health feedback information by calling RPC to TiKV

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Update kvproto; follow the protocol change; and add more comments

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Fix build

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Add integration tests

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* tune the frequency of active updating

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Update kvproto

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Fix test

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Revert changes to examples

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

* Remove unnecessary empty line

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>

---------

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>
Co-authored-by: MyonKeminta <MyonKeminta@users.noreply.github.com>
This commit is contained in:
MyonKeminta 2024-05-27 14:36:27 +08:00 committed by GitHub
parent 1c06c5cbf5
commit 31a8ddab19
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 262 additions and 56 deletions

2
go.mod
View File

@ -15,7 +15,7 @@ require (
github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c
github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989
github.com/pingcap/kvproto v0.0.0-20240227073058-929ab83f9754
github.com/pingcap/kvproto v0.0.0-20240513094934-d9297553c900
github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3
github.com/pkg/errors v0.9.1
github.com/prometheus/client_golang v1.18.0

4
go.sum
View File

@ -74,8 +74,8 @@ github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgW
github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew=
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E=
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw=
github.com/pingcap/kvproto v0.0.0-20240227073058-929ab83f9754 h1:nU9wDeMsID8EWawRQVdmRYcNhUrlI4TKogZhXleG4QQ=
github.com/pingcap/kvproto v0.0.0-20240227073058-929ab83f9754/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8=
github.com/pingcap/kvproto v0.0.0-20240513094934-d9297553c900 h1:snIM8DC846ufdlRclITACXfr1kvVIPU4cuQ6w3JVVY4=
github.com/pingcap/kvproto v0.0.0-20240513094934-d9297553c900/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8=
github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw=
github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=

View File

@ -6,7 +6,7 @@ require (
github.com/ninedraft/israce v0.0.3
github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f
github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c
github.com/pingcap/kvproto v0.0.0-20240227073058-929ab83f9754
github.com/pingcap/kvproto v0.0.0-20240513094934-d9297553c900
github.com/pingcap/tidb v1.1.0-beta.0.20240430081142-7481aa6d0b8b
github.com/pkg/errors v0.9.1
github.com/stretchr/testify v1.9.0

View File

@ -337,8 +337,8 @@ github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E=
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw=
github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w=
github.com/pingcap/kvproto v0.0.0-20240227073058-929ab83f9754 h1:nU9wDeMsID8EWawRQVdmRYcNhUrlI4TKogZhXleG4QQ=
github.com/pingcap/kvproto v0.0.0-20240227073058-929ab83f9754/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8=
github.com/pingcap/kvproto v0.0.0-20240513094934-d9297553c900 h1:snIM8DC846ufdlRclITACXfr1kvVIPU4cuQ6w3JVVY4=
github.com/pingcap/kvproto v0.0.0-20240513094934-d9297553c900/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8=
github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM=
github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d h1:y3EueKVfVykdpTyfUnQGqft0ud+xVFuCdp1XkVL0X1E=

View File

@ -0,0 +1,62 @@
// Copyright 2024 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 tikv_test
import (
"context"
"testing"
"time"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/tikvrpc"
)
func TestGetHealthFeedback(t *testing.T) {
if !*withTiKV {
return
}
tikvCluster := NewTestStore(t)
defer tikvCluster.Close()
// Find any TiKV node
store := tikvCluster.GetRegionCache().GetAllStores()[0]
for _, s := range tikvCluster.GetRegionCache().GetAllStores() {
if s.StoreType() == tikvrpc.TiKV {
store = s
}
}
require.NotNil(t, store)
client := tikvCluster.GetTiKVClient()
ctx := context.Background()
for i := 0; i < 3; i++ {
// In normal cases TiKV's slow score should be stable with value 1. Set it to any unstable value and check again
// to ensure the value is indeed received from TiKV.
store.GetHealthStatus().ResetTiKVServerSideSlowScoreForTest(50)
resp, err := client.SendRequest(ctx, store.GetAddr(), tikvrpc.NewRequest(tikvrpc.CmdGetHealthFeedback, &kvrpcpb.GetHealthFeedbackRequest{}), time.Second)
require.NoError(t, err)
getHealthFeedbackResp := resp.Resp.(*kvrpcpb.GetHealthFeedbackResponse)
require.NotNil(t, getHealthFeedbackResp)
require.NotEqual(t, uint64(0), getHealthFeedbackResp.GetHealthFeedback().GetFeedbackSeqNo())
require.Equal(t, int32(1), getHealthFeedbackResp.GetHealthFeedback().GetSlowScore())
require.Equal(t, store.StoreID(), getHealthFeedbackResp.GetHealthFeedback().GetStoreId())
// Updated in batch RPC stream.
require.Equal(t, int64(1), store.GetHealthStatus().GetHealthStatusDetail().TiKVSideSlowScore)
}
}

View File

@ -51,6 +51,7 @@ import (
"github.com/opentracing/opentracing-go"
"github.com/pingcap/kvproto/pkg/coprocessor"
"github.com/pingcap/kvproto/pkg/debugpb"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/mpp"
"github.com/pingcap/kvproto/pkg/tikvpb"
"github.com/pkg/errors"
@ -110,7 +111,7 @@ type Client interface {
// ClientEventListener is a listener to handle events produced by `Client`.
type ClientEventListener interface {
// OnHealthFeedback is called when `Client` receives a response that carries the HealthFeedback information.
OnHealthFeedback(feedback *tikvpb.HealthFeedback)
OnHealthFeedback(feedback *kvrpcpb.HealthFeedback)
}
// ClientExt is a client has extended interfaces.

View File

@ -44,6 +44,7 @@ import (
"sync/atomic"
"time"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/tikvpb"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
@ -783,7 +784,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport
}
}
func (c *batchCommandsClient) onHealthFeedback(feedback *tikvpb.HealthFeedback) {
func (c *batchCommandsClient) onHealthFeedback(feedback *kvrpcpb.HealthFeedback) {
if h := c.eventListener.Load(); h != nil {
(*h).OnHealthFeedback(feedback)
}

View File

@ -823,16 +823,16 @@ func TestPrioritySentLimit(t *testing.T) {
}
type testClientEventListener struct {
healthFeedbackCh chan *tikvpb.HealthFeedback
healthFeedbackCh chan *kvrpcpb.HealthFeedback
}
func newTestClientEventListener() *testClientEventListener {
return &testClientEventListener{
healthFeedbackCh: make(chan *tikvpb.HealthFeedback, 100),
healthFeedbackCh: make(chan *kvrpcpb.HealthFeedback, 100),
}
}
func (l *testClientEventListener) OnHealthFeedback(feedback *tikvpb.HealthFeedback) {
func (l *testClientEventListener) OnHealthFeedback(feedback *kvrpcpb.HealthFeedback) {
l.healthFeedbackCh <- feedback
}

View File

@ -99,7 +99,7 @@ func (s *MockServer) BatchCommands(ss tikvpb.Tikv_BatchCommandsServer) error {
err = ss.Send(&tikvpb.BatchCommandsResponse{
Responses: responses,
RequestIds: req.GetRequestIds(),
HealthFeedback: &tikvpb.HealthFeedback{
HealthFeedback: &kvrpcpb.HealthFeedback{
StoreId: 1,
FeedbackSeqNo: feedbackSeq,
SlowScore: 1,

View File

@ -55,7 +55,6 @@ import (
"github.com/opentracing/opentracing-go"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/kvproto/pkg/tikvpb"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/tikv/client-go/v2/config"
@ -631,6 +630,8 @@ type RegionCache struct {
codec apicodec.Codec
enableForwarding bool
requestHealthFeedbackCallback func(ctx context.Context, addr string) error
mu regionIndexMu
stores storeCache
@ -642,7 +643,8 @@ type RegionCache struct {
}
type regionCacheOptions struct {
noHealthTick bool
noHealthTick bool
requestHealthFeedbackCallback func(ctx context.Context, addr string) error
}
type RegionCacheOpt func(*regionCacheOptions)
@ -651,6 +653,12 @@ func RegionCacheNoHealthTick(o *regionCacheOptions) {
o.noHealthTick = true
}
func WithRequestHealthFeedbackCallback(callback func(ctx context.Context, addr string) error) RegionCacheOpt {
return func(options *regionCacheOptions) {
options.requestHealthFeedbackCallback = callback
}
}
// NewRegionCache creates a RegionCache.
func NewRegionCache(pdClient pd.Client, opt ...RegionCacheOpt) *RegionCache {
var options regionCacheOptions
@ -659,7 +667,8 @@ func NewRegionCache(pdClient pd.Client, opt ...RegionCacheOpt) *RegionCache {
}
c := &RegionCache{
pdClient: pdClient,
pdClient: pdClient,
requestHealthFeedbackCallback: options.requestHealthFeedbackCallback,
}
c.codec = apicodec.NewCodecV1(apicodec.ModeRaw)
@ -704,7 +713,7 @@ func NewRegionCache(pdClient pd.Client, opt ...RegionCacheOpt) *RegionCache {
return false
}, time.Duration(refreshStoreInterval/4)*time.Second, c.stores.getCheckStoreEvents())
if !options.noHealthTick {
c.bg.schedule(repeat(c.checkAndUpdateStoreHealthStatus), time.Duration(refreshStoreInterval/4)*time.Second)
c.bg.schedule(c.checkAndUpdateStoreHealthStatus, time.Duration(refreshStoreInterval/4)*time.Second)
}
c.bg.schedule(repeat(c.reportStoreReplicaFlows), time.Duration(refreshStoreInterval/2)*time.Second)
if refreshCacheInterval := config.GetGlobalConfig().RegionsRefreshInterval; refreshCacheInterval > 0 {
@ -2618,7 +2627,7 @@ func (r *Region) ContainsByEnd(key []byte) bool {
}
// checkAndUpdateStoreHealthStatus checks and updates health stats on each store.
func (c *RegionCache) checkAndUpdateStoreHealthStatus() {
func (c *RegionCache) checkAndUpdateStoreHealthStatus(ctx context.Context, now time.Time) bool {
defer func() {
r := recover()
if r != nil {
@ -2630,16 +2639,18 @@ func (c *RegionCache) checkAndUpdateStoreHealthStatus() {
}
}
}()
healthDetails := make(map[uint64]HealthStatusDetail)
now := time.Now()
var stores []*Store
c.stores.forEach(func(store *Store) {
store.healthStatus.tick(now)
healthDetails[store.storeID] = store.healthStatus.GetHealthStatusDetail()
stores = append(stores, store)
})
for store, details := range healthDetails {
metrics.TiKVStoreSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(float64(details.ClientSideSlowScore))
metrics.TiKVFeedbackSlowScoreGauge.WithLabelValues(strconv.FormatUint(store, 10)).Set(float64(details.TiKVSideSlowScore))
for _, store := range stores {
store.healthStatus.tick(ctx, now, store, c.requestHealthFeedbackCallback)
healthDetails := store.healthStatus.GetHealthStatusDetail()
metrics.TiKVStoreSlowScoreGauge.WithLabelValues(strconv.FormatUint(store.storeID, 10)).Set(float64(healthDetails.ClientSideSlowScore))
metrics.TiKVFeedbackSlowScoreGauge.WithLabelValues(strconv.FormatUint(store.storeID, 10)).Set(float64(healthDetails.TiKVSideSlowScore))
}
return false
}
// reportStoreReplicaFlows reports the statistics on the related replicaFlowsType.
@ -2662,7 +2673,7 @@ func contains(startKey, endKey, key []byte) bool {
(bytes.Compare(key, endKey) < 0 || len(endKey) == 0)
}
func (c *RegionCache) onHealthFeedback(feedback *tikvpb.HealthFeedback) {
func (c *RegionCache) onHealthFeedback(feedback *kvrpcpb.HealthFeedback) {
store, ok := c.stores.get(feedback.GetStoreId())
if !ok {
logutil.BgLogger().Info("dropped health feedback info due to unknown store id", zap.Uint64("storeID", feedback.GetStoreId()))
@ -2683,6 +2694,6 @@ type regionCacheClientEventListener struct {
}
// OnHealthFeedback implements the `client.ClientEventListener` interface.
func (l *regionCacheClientEventListener) OnHealthFeedback(feedback *tikvpb.HealthFeedback) {
func (l *regionCacheClientEventListener) OnHealthFeedback(feedback *kvrpcpb.HealthFeedback) {
l.c.onHealthFeedback(feedback)
}

View File

@ -49,14 +49,15 @@ import (
"github.com/gogo/protobuf/proto"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/errorpb"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/kvproto/pkg/tikvpb"
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
"github.com/tikv/client-go/v2/config/retry"
"github.com/tikv/client-go/v2/internal/apicodec"
"github.com/tikv/client-go/v2/internal/mockstore/mocktikv"
"github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/tikvrpc"
pd "github.com/tikv/pd/client"
uatomic "go.uber.org/atomic"
)
@ -2088,10 +2089,14 @@ func (s *testRegionCacheSuite) TestHealthCheckWithStoreReplace() {
}
func (s *testRegionCacheSuite) TestTiKVSideSlowScore() {
store := newStore(1, "", "", "", tikvrpc.TiKV, resolved, nil)
store.livenessState = uint32(reachable)
ctx := context.Background()
stats := newStoreHealthStatus(1)
s.LessOrEqual(stats.GetHealthStatusDetail().TiKVSideSlowScore, int64(1))
now := time.Now()
stats.tick(now)
stats.tick(ctx, now, store, nil)
s.LessOrEqual(stats.GetHealthStatusDetail().TiKVSideSlowScore, int64(1))
s.False(stats.tikvSideSlowScore.hasTiKVFeedback.Load())
s.False(stats.IsSlow())
@ -2108,22 +2113,26 @@ func (s *testRegionCacheSuite) TestTiKVSideSlowScore() {
s.True(stats.IsSlow())
now = now.Add(time.Minute * 2)
stats.tick(now)
stats.tick(ctx, now, store, nil)
s.Equal(int64(60), stats.GetHealthStatusDetail().TiKVSideSlowScore)
s.False(stats.IsSlow())
now = now.Add(time.Minute * 3)
stats.tick(now)
stats.tick(ctx, now, store, nil)
s.Equal(int64(1), stats.GetHealthStatusDetail().TiKVSideSlowScore)
s.False(stats.IsSlow())
now = now.Add(time.Minute)
stats.tick(now)
stats.tick(ctx, now, store, nil)
s.Equal(int64(1), stats.GetHealthStatusDetail().TiKVSideSlowScore)
s.False(stats.IsSlow())
}
func (s *testRegionCacheSuite) TestStoreHealthStatus() {
store := newStore(1, "", "", "", tikvrpc.TiKV, resolved, nil)
store.livenessState = uint32(reachable)
ctx := context.Background()
stats := newStoreHealthStatus(1)
now := time.Now()
s.False(stats.IsSlow())
@ -2131,7 +2140,7 @@ func (s *testRegionCacheSuite) TestStoreHealthStatus() {
for !stats.clientSideSlowScore.isSlow() {
stats.clientSideSlowScore.recordSlowScoreStat(time.Minute)
}
stats.tick(now)
stats.tick(ctx, now, store, nil)
s.True(stats.IsSlow())
s.Equal(int64(stats.clientSideSlowScore.getSlowScore()), stats.GetHealthStatusDetail().ClientSideSlowScore)
@ -2142,7 +2151,7 @@ func (s *testRegionCacheSuite) TestStoreHealthStatus() {
for stats.clientSideSlowScore.isSlow() {
stats.clientSideSlowScore.recordSlowScoreStat(time.Millisecond)
stats.tick(now)
stats.tick(ctx, now, store, nil)
}
s.True(stats.IsSlow())
s.Equal(int64(stats.clientSideSlowScore.getSlowScore()), stats.GetHealthStatusDetail().ClientSideSlowScore)
@ -2160,7 +2169,7 @@ func (s *testRegionCacheSuite) TestRegionCacheHandleHealthStatus() {
s.True(exists)
s.False(store1.healthStatus.IsSlow())
feedbackMsg := &tikvpb.HealthFeedback{
feedbackMsg := &kvrpcpb.HealthFeedback{
StoreId: s.store1,
FeedbackSeqNo: 1,
SlowScore: 100,
@ -2169,7 +2178,7 @@ func (s *testRegionCacheSuite) TestRegionCacheHandleHealthStatus() {
s.True(store1.healthStatus.IsSlow())
s.Equal(int64(100), store1.healthStatus.GetHealthStatusDetail().TiKVSideSlowScore)
feedbackMsg = &tikvpb.HealthFeedback{
feedbackMsg = &kvrpcpb.HealthFeedback{
StoreId: s.store1,
FeedbackSeqNo: 2,
SlowScore: 90,
@ -2178,7 +2187,7 @@ func (s *testRegionCacheSuite) TestRegionCacheHandleHealthStatus() {
s.cache.onHealthFeedback(feedbackMsg)
s.Equal(int64(100), store1.healthStatus.GetHealthStatusDetail().TiKVSideSlowScore)
feedbackMsg = &tikvpb.HealthFeedback{
feedbackMsg = &kvrpcpb.HealthFeedback{
StoreId: s.store1,
FeedbackSeqNo: 3,
SlowScore: 90,
@ -2187,7 +2196,7 @@ func (s *testRegionCacheSuite) TestRegionCacheHandleHealthStatus() {
s.cache.onHealthFeedback(feedbackMsg)
s.Equal(int64(90), store1.healthStatus.GetHealthStatusDetail().TiKVSideSlowScore)
feedbackMsg = &tikvpb.HealthFeedback{
feedbackMsg = &kvrpcpb.HealthFeedback{
StoreId: s.store1,
FeedbackSeqNo: 4,
SlowScore: 50,

View File

@ -546,6 +546,10 @@ func (s *mockTikvGrpcServer) KvBufferBatchGet(context.Context, *kvrpcpb.BufferBa
return nil, errors.New("unreachable")
}
func (s *mockTikvGrpcServer) GetHealthFeedback(ctx context.Context, request *kvrpcpb.GetHealthFeedbackRequest) (*kvrpcpb.GetHealthFeedbackResponse, error) {
return nil, errors.New("unreachable")
}
func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionForGrpcWhenCtxCanceled() {
// prepare a mock tikv grpc server
addr := "localhost:56341"

View File

@ -3101,7 +3101,7 @@ func (s *testReplicaSelectorSuite) resetStoreState() {
for _, store := range rc.getStore().stores {
store.loadStats.Store(nil)
store.healthStatus.clientSideSlowScore.resetSlowScore()
store.healthStatus.resetTiKVServerSideSlowScoreForTest()
store.healthStatus.ResetTiKVServerSideSlowScoreForTest(1)
store.healthStatus.updateSlowFlag()
atomic.StoreUint32(&store.livenessState, uint32(reachable))
store.setResolveState(resolved)

View File

@ -18,13 +18,14 @@ import (
"context"
"fmt"
"math"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/kvproto/pkg/tikvpb"
"github.com/pkg/errors"
"github.com/tikv/client-go/v2/config"
"github.com/tikv/client-go/v2/config/retry"
@ -260,6 +261,11 @@ func newUninitializedStore(id uint64) *Store {
}
}
// StoreType returns the type of the store.
func (s *Store) StoreType() tikvrpc.EndpointType {
return s.storeType
}
// IsTiFlash returns true if the storeType is TiFlash
func (s *Store) IsTiFlash() bool {
return s.storeType == tikvrpc.TiFlash
@ -324,6 +330,11 @@ func (s *Store) IsLabelsMatch(labels []*metapb.StoreLabel) bool {
return true
}
// GetHealthStatus returns the health status of the store. This is exported for test purpose.
func (s *Store) GetHealthStatus() *StoreHealthStatus {
return s.healthStatus
}
func isStoreContainLabel(labels []*metapb.StoreLabel, key string, val string) (res bool) {
for _, label := range labels {
if label.GetKey() == key && label.GetValue() == val {
@ -822,7 +833,7 @@ const (
tikvSlowScoreSlowThreshold int64 = 80
tikvSlowScoreUpdateInterval = time.Millisecond * 100
tikvSlowScoreUpdateFromPDInterval = time.Minute
tikvSlowScoreActiveUpdateInterval = time.Second * 15
)
type StoreHealthStatus struct {
@ -880,9 +891,10 @@ func (s *StoreHealthStatus) GetHealthStatusDetail() HealthStatusDetail {
// tick updates the health status that changes over time, such as slow score's decaying, etc. This function is expected
// to be called periodically.
func (s *StoreHealthStatus) tick(now time.Time) {
func (s *StoreHealthStatus) tick(ctx context.Context, now time.Time, store *Store, requestHealthFeedbackCallback func(ctx context.Context, addr string) error) {
metrics.TiKVHealthFeedbackOpsCounter.WithLabelValues(strconv.FormatUint(store.StoreID(), 10), "tick").Inc()
s.clientSideSlowScore.updateSlowScore()
s.updateTiKVServerSideSlowScoreOnTick(now)
s.updateTiKVServerSideSlowScoreOnTick(ctx, now, store, requestHealthFeedbackCallback)
s.updateSlowFlag()
}
@ -900,17 +912,55 @@ func (s *StoreHealthStatus) markAlreadySlow() {
// updateTiKVServerSideSlowScoreOnTick updates the slow score actively, which is expected to be a periodic job.
// It skips updating if the last update time didn't elapse long enough, or it's being updated concurrently.
func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick(now time.Time) {
func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick(ctx context.Context, now time.Time, store *Store, requestHealthFeedbackCallback func(ctx context.Context, addr string) error) {
if !s.tikvSideSlowScore.hasTiKVFeedback.Load() {
// Do nothing if no feedback has been received from this store yet.
return
}
lastUpdateTime := s.tikvSideSlowScore.lastUpdateTime.Load()
if lastUpdateTime == nil || now.Sub(*lastUpdateTime) < tikvSlowScoreUpdateFromPDInterval {
// If the first feedback is
// Skip tick if the store's slow score is 1, as it's likely to be a normal case that a health store is not being
// accessed.
if s.tikvSideSlowScore.score.Load() <= 1 {
return
}
needRefreshing := func() bool {
lastUpdateTime := s.tikvSideSlowScore.lastUpdateTime.Load()
if lastUpdateTime == nil {
// If the first hasn't been received yet, assume the store doesn't support feeding back and skip the tick.
return false
}
return now.Sub(*lastUpdateTime) >= tikvSlowScoreActiveUpdateInterval
}
if !needRefreshing() {
return
}
// If not updated for too long, try to explicitly fetch it from TiKV.
// Note that this can't be done while holding the mutex, because the updating is done by the client when receiving
// the response (in the same way as handling the feedback information pushed from TiKV), which needs acquiring the
// mutex.
if requestHealthFeedbackCallback != nil && store.getLivenessState() == reachable {
addr := store.GetAddr()
if len(addr) == 0 {
logutil.Logger(ctx).Warn("skip actively request health feedback info from store due to unknown addr", zap.Uint64("storeID", store.StoreID()))
} else {
metrics.TiKVHealthFeedbackOpsCounter.WithLabelValues(strconv.FormatUint(store.StoreID(), 10), "active_update").Inc()
err := requestHealthFeedbackCallback(ctx, store.GetAddr())
if err != nil {
metrics.TiKVHealthFeedbackOpsCounter.WithLabelValues(strconv.FormatUint(store.StoreID(), 10), "active_update_err").Inc()
logutil.Logger(ctx).Warn("actively request health feedback info from store got error", zap.Uint64("storeID", store.StoreID()), zap.Error(err))
}
}
// Continue if active updating is unsuccessful.
if !needRefreshing() {
return
}
}
if !s.tikvSideSlowScore.TryLock() {
// It must be being updated concurrently.
return
@ -918,16 +968,13 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScoreOnTick(now time.Time) {
defer s.tikvSideSlowScore.Unlock()
// Reload update time as it might be updated concurrently before acquiring mutex
lastUpdateTime = s.tikvSideSlowScore.lastUpdateTime.Load()
lastUpdateTime := s.tikvSideSlowScore.lastUpdateTime.Load()
elapsed := now.Sub(*lastUpdateTime)
if elapsed < tikvSlowScoreUpdateFromPDInterval {
if elapsed < tikvSlowScoreActiveUpdateInterval {
return
}
// TODO: Try to get store status from PD here. But it's not mandatory.
// Don't forget to update tests if getting slow score from PD is implemented here.
// If updating from PD is not successful: decay the slow score.
// If requesting from TiKV is not successful: decay the slow score.
score := s.tikvSideSlowScore.score.Load()
if score < 1 {
return
@ -949,6 +996,20 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime
lastScore := s.tikvSideSlowScore.score.Load()
if lastScore == score {
// It's still needed to update the lastUpdateTime to tell whether the slow score is not being updated for too
// long (so that it's needed to explicitly get the slow score).
// from TiKV.
// But it can be safely skipped if the score is 1 (as explicit getting slow score won't be performed in this
// case). And note that it should be updated within mutex.
if score > 1 {
// Skip if not locked as it's being updated concurrently.
if s.tikvSideSlowScore.TryLock() {
newUpdateTime := new(time.Time)
*newUpdateTime = currTime
s.tikvSideSlowScore.lastUpdateTime.Store(newUpdateTime)
s.tikvSideSlowScore.Unlock()
}
}
return
}
@ -980,9 +1041,12 @@ func (s *StoreHealthStatus) updateTiKVServerSideSlowScore(score int64, currTime
s.tikvSideSlowScore.lastUpdateTime.Store(newUpdateTime)
}
func (s *StoreHealthStatus) resetTiKVServerSideSlowScoreForTest() {
// ResetTiKVServerSideSlowScoreForTest resets the TiKV-side slow score information and make it expired so that the
// next update can be effective. A new score should be passed to the function. For a store that's running normally
// without any sign of being slow, the value should be 1.
func (s *StoreHealthStatus) ResetTiKVServerSideSlowScoreForTest(score int64) {
s.setTiKVSlowScoreLastUpdateTimeForTest(time.Now().Add(-time.Hour * 2))
s.updateTiKVServerSideSlowScore(1, time.Now().Add(-time.Hour))
s.updateTiKVServerSideSlowScore(score, time.Now().Add(-time.Hour))
}
func (s *StoreHealthStatus) updateSlowFlag() {
@ -1002,7 +1066,7 @@ func (s *StoreHealthStatus) setTiKVSlowScoreLastUpdateTimeForTest(lastUpdateTime
s.tikvSideSlowScore.lastUpdateTime.Store(&lastUpdateTime)
}
func (s *Store) recordHealthFeedback(feedback *tikvpb.HealthFeedback) {
func (s *Store) recordHealthFeedback(feedback *kvrpcpb.HealthFeedback) {
// Note that the `FeedbackSeqNo` field of `HealthFeedback` is not used yet. It's a monotonic value that can help
// to drop out-of-order feedback messages. But it's not checked for now since it's not very necessary to receive
// only a slow score. It's prepared for possible use in the future.

View File

@ -103,6 +103,7 @@ var (
TiKVAggressiveLockedKeysCounter *prometheus.CounterVec
TiKVStoreSlowScoreGauge *prometheus.GaugeVec
TiKVFeedbackSlowScoreGauge *prometheus.GaugeVec
TiKVHealthFeedbackOpsCounter *prometheus.CounterVec
TiKVPreferLeaderFlowsGauge *prometheus.GaugeVec
TiKVStaleReadCounter *prometheus.CounterVec
TiKVStaleReadReqCounter *prometheus.CounterVec
@ -726,6 +727,15 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) {
ConstLabels: constLabels,
}, []string{LblStore})
TiKVHealthFeedbackOpsCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "health_feedback_ops_counter",
Help: "Counter of operations about TiKV health feedback",
ConstLabels: constLabels,
}, []string{LblScope, LblType})
TiKVPreferLeaderFlowsGauge = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: namespace,
@ -868,6 +878,7 @@ func RegisterMetrics() {
prometheus.MustRegister(TiKVAggressiveLockedKeysCounter)
prometheus.MustRegister(TiKVStoreSlowScoreGauge)
prometheus.MustRegister(TiKVFeedbackSlowScoreGauge)
prometheus.MustRegister(TiKVHealthFeedbackOpsCounter)
prometheus.MustRegister(TiKVPreferLeaderFlowsGauge)
prometheus.MustRegister(TiKVStaleReadCounter)
prometheus.MustRegister(TiKVStaleReadReqCounter)

View File

@ -235,6 +235,28 @@ func loadOption(store *KVStore, opt ...Option) {
}
}
const getHealthFeedbackTimeout = time.Second * 2
func requestHealthFeedbackFromKVClient(ctx context.Context, addr string, tikvClient Client) error {
// When batch RPC is enabled (`MaxBatchSize` > 0), a `GetHealthFeedback` RPC call will cause TiKV also sending the
// health feedback information in via the `BatchCommandsResponse`, which will be handled by the batch client.
// Therefore the same information carried in the response don't need to be handled in this case. And as we're
// currently not supporting health feedback mechanism without enabling batch RPC, we do not use the information
// carried in the `resp` here.
resp, err := tikvClient.SendRequest(ctx, addr, tikvrpc.NewRequest(tikvrpc.CmdGetHealthFeedback, &kvrpcpb.GetHealthFeedbackRequest{}), getHealthFeedbackTimeout)
if err != nil {
return err
}
regionErr, err := resp.GetRegionError()
if err != nil {
return err
}
if regionErr != nil {
return errors.Errorf("requested health feedback from store but received region error: %s", regionErr.String())
}
return nil
}
// NewKVStore creates a new TiKV store instance.
func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Client, opt ...Option) (*KVStore, error) {
o, err := oracles.NewPdOracle(pdClient, defaultOracleUpdateInterval)
@ -242,7 +264,9 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Cl
return nil, err
}
ctx, cancel := context.WithCancel(context.Background())
regionCache := locate.NewRegionCache(pdClient)
regionCache := locate.NewRegionCache(pdClient, locate.WithRequestHealthFeedbackCallback(func(ctx context.Context, addr string) error {
return requestHealthFeedbackFromKVClient(ctx, addr, tikvclient)
}))
store := &KVStore{
clusterID: pdClient.GetClusterID(context.TODO()),
uuid: uuid,

View File

@ -99,6 +99,8 @@ const (
CmdStoreSafeTS
CmdLockWaitInfo
CmdGetHealthFeedback
CmdCop CmdType = 512 + iota
CmdCopStream
CmdBatchCop
@ -217,6 +219,8 @@ func (t CmdType) String() string {
return "StoreSafeTS"
case CmdLockWaitInfo:
return "LockWaitInfo"
case CmdGetHealthFeedback:
return "GetHealthFeedback"
case CmdFlashbackToVersion:
return "FlashbackToVersion"
case CmdPrepareFlashbackToVersion:
@ -559,6 +563,11 @@ func (req *Request) LockWaitInfo() *kvrpcpb.GetLockWaitInfoRequest {
return req.Req.(*kvrpcpb.GetLockWaitInfoRequest)
}
// GetHealthFeedback returns GetHealthFeedbackRequest in request.
func (req *Request) GetHealthFeedback() *kvrpcpb.GetHealthFeedbackRequest {
return req.Req.(*kvrpcpb.GetHealthFeedbackRequest)
}
// FlashbackToVersion returns FlashbackToVersionRequest in request.
func (req *Request) FlashbackToVersion() *kvrpcpb.FlashbackToVersionRequest {
return req.Req.(*kvrpcpb.FlashbackToVersionRequest)
@ -642,6 +651,8 @@ func (req *Request) ToBatchCommandsRequest() *tikvpb.BatchCommandsRequest_Reques
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Flush{Flush: req.Flush()}}
case CmdBufferBatchGet:
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_BufferBatchGet{BufferBatchGet: req.BufferBatchGet()}}
case CmdGetHealthFeedback:
return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_GetHealthFeedback{GetHealthFeedback: req.GetHealthFeedback()}}
}
return nil
}
@ -717,6 +728,8 @@ func FromBatchCommandsResponse(res *tikvpb.BatchCommandsResponse_Response) (*Res
return &Response{Resp: res.Flush}, nil
case *tikvpb.BatchCommandsResponse_Response_BufferBatchGet:
return &Response{Resp: res.BufferBatchGet}, nil
case *tikvpb.BatchCommandsResponse_Response_GetHealthFeedback:
return &Response{Resp: res.GetHealthFeedback}, nil
}
panic("unreachable")
}
@ -950,6 +963,10 @@ func GenRegionErrorResp(req *Request, e *errorpb.Error) (*Response, error) {
p = &kvrpcpb.BufferBatchGetResponse{
RegionError: e,
}
case CmdGetHealthFeedback:
p = &kvrpcpb.GetHealthFeedbackResponse{
RegionError: e,
}
default:
return nil, errors.Errorf("invalid request type %v", req.Type)
}
@ -1124,6 +1141,8 @@ func CallRPC(ctx context.Context, client tikvpb.TikvClient, req *Request) (*Resp
resp.Resp, err = client.KvFlush(ctx, req.Flush())
case CmdBufferBatchGet:
resp.Resp, err = client.KvBufferBatchGet(ctx, req.BufferBatchGet())
case CmdGetHealthFeedback:
resp.Resp, err = client.GetHealthFeedback(ctx, req.GetHealthFeedback())
default:
return nil, errors.Errorf("invalid request type: %v", req.Type)
}