Make the txn safe point polling support falling back to old version of PD (#1727)

closes tikv/client-go#1728

Signed-off-by: MyonKeminta <MyonKeminta@users.noreply.github.com>
This commit is contained in:
MyonKeminta 2025-07-30 16:44:49 +08:00 committed by GitHub
parent 24b88f0f87
commit 58e9bb4b6d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
9 changed files with 589 additions and 41 deletions

View File

@ -0,0 +1,333 @@
package tikv_test
import (
"bytes"
"context"
"encoding/json"
"fmt"
"io"
"net/http"
"strings"
"testing"
"time"
"github.com/google/uuid"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/keyspacepb"
"github.com/pingcap/tidb/pkg/keyspace"
"github.com/stretchr/testify/suite"
"github.com/tikv/client-go/v2/config"
"github.com/tikv/client-go/v2/tikv"
pd "github.com/tikv/pd/client"
pdgc "github.com/tikv/pd/client/clients/gc"
"github.com/tikv/pd/client/constants"
"github.com/tikv/pd/client/pkg/caller"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
type hookedGCStatesClient struct {
inner pdgc.GCStatesClient
getGCStatesHook func(inner pdgc.GCStatesClient, ctx context.Context) (pdgc.GCState, error)
}
func (c *hookedGCStatesClient) SetGCBarrier(ctx context.Context, barrierID string, barrierTS uint64, ttl time.Duration) (*pdgc.GCBarrierInfo, error) {
return c.inner.SetGCBarrier(ctx, barrierID, barrierTS, ttl)
}
func (c *hookedGCStatesClient) DeleteGCBarrier(ctx context.Context, barrierID string) (*pdgc.GCBarrierInfo, error) {
return c.inner.DeleteGCBarrier(ctx, barrierID)
}
func (c *hookedGCStatesClient) GetGCState(ctx context.Context) (pdgc.GCState, error) {
if c.getGCStatesHook != nil {
return c.getGCStatesHook(c.inner, ctx)
}
return c.inner.GetGCState(ctx)
}
func hookGCStatesClientForStore(store *tikv.StoreProbe, getGCStatesHook func(inner pdgc.GCStatesClient, ctx context.Context) (pdgc.GCState, error)) {
store.ReplaceGCStatesClient(&hookedGCStatesClient{
inner: store.GetGCStatesClient(),
getGCStatesHook: getGCStatesHook,
})
}
var _ = pdgc.GCStatesClient(&hookedGCStatesClient{})
func TestGCWithTiKVSuite(t *testing.T) {
suite.Run(t, new(testGCWithTiKVSuite))
}
type testGCWithTiKVSuite struct {
suite.Suite
globalPDCli pd.Client
addrs []string
pdClis []*tikv.CodecPDClient
stores []*tikv.StoreProbe
keyspaces []*keyspacepb.KeyspaceMeta
}
func (s *testGCWithTiKVSuite) SetupTest() {
if !*withTiKV {
s.T().Skip("Cannot run without real tikv cluster")
}
s.addrs = strings.Split(*pdAddrs, ",")
var err error
s.globalPDCli, err = pd.NewClient(caller.TestComponent, s.addrs, pd.SecurityOption{})
s.Require().NoError(err)
}
func (s *testGCWithTiKVSuite) TearDownTest() {
re := s.Require()
for _, store := range s.stores {
re.NoError(store.Close())
}
if s.globalPDCli != nil {
s.globalPDCli.Close()
}
for _, pdCli := range s.pdClis {
pdCli.Close()
}
for _, keyspaceMeta := range s.keyspaces {
if keyspaceMeta == nil {
s.dropKeyspace(keyspaceMeta)
}
}
}
func (s *testGCWithTiKVSuite) createClient(keyspaceMeta *keyspacepb.KeyspaceMeta) (*tikv.CodecPDClient, *tikv.StoreProbe) {
re := s.Require()
var pdCli *tikv.CodecPDClient
if keyspaceMeta == nil {
inner, err := pd.NewClient(caller.TestComponent, s.addrs, pd.SecurityOption{})
re.NoError(err)
pdCli = tikv.NewCodecPDClient(tikv.ModeTxn, inner)
} else {
inner, err := pd.NewClientWithKeyspace(context.Background(), caller.TestComponent, keyspaceMeta.GetId(), s.addrs, pd.SecurityOption{})
re.NoError(err)
pdCli, err = tikv.NewCodecPDClientWithKeyspace(tikv.ModeTxn, inner, keyspaceMeta.GetName())
re.NoError(err)
}
tlsConfig, err := (&config.Security{}).ToTLSConfig()
re.NoError(err)
var spkv tikv.SafePointKV
if keyspaceMeta == nil {
spkv, err = tikv.NewEtcdSafePointKV(s.addrs, tlsConfig)
re.NoError(err)
} else {
spkv, err = tikv.NewEtcdSafePointKV(s.addrs, tlsConfig, tikv.WithPrefix(keyspace.MakeKeyspaceEtcdNamespace(pdCli.GetCodec())))
re.NoError(err)
}
store, err := tikv.NewKVStore("test-store", pdCli, spkv, tikv.NewRPCClient(tikv.WithCodec(pdCli.GetCodec())))
return pdCli, &tikv.StoreProbe{KVStore: store}
}
func (s *testGCWithTiKVSuite) createKeyspace(name string, keyspaceLevelGC bool) *keyspacepb.KeyspaceMeta {
re := s.Require()
gcManagementType := "unified"
if keyspaceLevelGC {
gcManagementType = "keyspace_level"
}
req := struct {
Name string `json:"name"`
Config map[string]string `json:"config"`
}{
Name: name,
Config: map[string]string{"gc_management_type": gcManagementType},
}
reqJson, err := json.Marshal(req)
re.NoError(err)
resp, err := http.Post(fmt.Sprintf("%s/pd/api/v2/keyspaces", s.addrs[0]), "application/json", bytes.NewBuffer(reqJson))
re.NoError(err)
defer resp.Body.Close()
respBody := new(strings.Builder)
_, err = io.Copy(respBody, resp.Body)
re.NoError(err)
re.Equal(http.StatusOK, resp.StatusCode, "Failed to create keyspace %s, response: %s", name, respBody.String())
meta, err := s.globalPDCli.LoadKeyspace(context.Background(), name)
re.NoError(err)
// Avoid goroutine leak in the test.
http.DefaultClient.CloseIdleConnections()
return meta
}
type storeKeyspaceType int
const (
storeNullKeyspace storeKeyspaceType = iota
storeKeyspaceLevelGCKeyspace
storeUnifiedGCKeyspace
)
func (s *testGCWithTiKVSuite) dropKeyspace(keyspaceMeta *keyspacepb.KeyspaceMeta) {
re := s.Require()
// Nil might be used to represent the null keyspace.
if keyspaceMeta != nil {
_, err := s.globalPDCli.UpdateKeyspaceState(context.Background(), keyspaceMeta.Id, keyspacepb.KeyspaceState_ARCHIVED)
re.NoError(err)
}
}
func genKeyspaceName() string {
return uuid.New().String()
}
func (s *testGCWithTiKVSuite) prepareClients(storeKeyspaceTypes ...storeKeyspaceType) {
s.pdClis = make([]*tikv.CodecPDClient, 0, len(storeKeyspaceTypes))
s.stores = make([]*tikv.StoreProbe, 0, len(storeKeyspaceTypes))
s.keyspaces = make([]*keyspacepb.KeyspaceMeta, 0, len(storeKeyspaceTypes))
for _, t := range storeKeyspaceTypes {
var keyspaceMeta *keyspacepb.KeyspaceMeta
switch t {
case storeNullKeyspace:
// represents by nil keyspace meta
case storeKeyspaceLevelGCKeyspace:
keyspaceMeta = s.createKeyspace(genKeyspaceName(), true)
case storeUnifiedGCKeyspace:
keyspaceMeta = s.createKeyspace(genKeyspaceName(), false)
}
pdCli, store := s.createClient(keyspaceMeta)
s.keyspaces = append(s.keyspaces, keyspaceMeta)
s.pdClis = append(s.pdClis, pdCli)
s.stores = append(s.stores, store)
}
}
func (s *testGCWithTiKVSuite) TestLoadTxnSafePointFallback() {
re := s.Require()
re.NoError(failpoint.Enable("tikvclient/noBuiltInTxnSafePointUpdater", "return"))
defer func() {
re.NoError(failpoint.Disable("tikvclient/noBuiltInTxnSafePointUpdater"))
}()
s.prepareClients(storeNullKeyspace, storeUnifiedGCKeyspace, storeKeyspaceLevelGCKeyspace)
ctx := context.Background()
// The new keyspaces always have zero txn safe point, while the null keyspace may have larger txn safe point due
// to other tests that has been run on the cluster. The following test will use values from the null keyspace's
// txn safe point.
state, err := s.pdClis[0].GetGCStatesClient(constants.NullKeyspaceID).GetGCState(ctx)
re.NoError(err)
base := state.TxnSafePoint
callCounter := 0
hook := func(inner pdgc.GCStatesClient, ctx context.Context) (pdgc.GCState, error) {
callCounter += 1
return pdgc.GCState{}, status.Errorf(codes.Unimplemented, "simulated unimplemented error")
}
for _, store := range s.stores {
hookGCStatesClientForStore(store, hook)
_, err = store.GetGCStatesClient().GetGCState(ctx)
re.Error(err)
re.Equal(codes.Unimplemented, status.Code(err))
}
re.Equal(len(s.stores), callCounter)
callCounter = 0
// Updating the null keyspace. The new value is visible by the null keyspace and keyspaces using unified GC,
// but not by keyspaces using keyspace-level GC.
res, err := s.pdClis[0].GetGCInternalController(constants.NullKeyspaceID).AdvanceTxnSafePoint(ctx, base+1)
re.NoError(err)
re.Equal(base+1, res.NewTxnSafePoint)
// Call LoadTxnSafePoint twice, so that by checking the callCounter, we chan ensure the new API is called at most
// once (for each KVStore) and won't be called again since the first falling back.
// For store[0] (null keyspace) and store[1] (unified GC)
for i := 0; i < 2; i++ {
for _, store := range s.stores[0:2] {
txnSafePoint, err := store.LoadTxnSafePoint(ctx)
re.NoError(err)
re.Equal(base+1, txnSafePoint)
}
// For store[2] (keyspace level GC): Invisible to the new txn safe point.
txnSafePoint, err := s.stores[2].LoadTxnSafePoint(ctx)
re.NoError(err)
re.Less(txnSafePoint, base+1)
}
// Updating the keyspace with keyspace level GC. The effect is visible only in the same keyspace.
res, err = s.pdClis[2].GetGCInternalController(s.keyspaces[2].GetId()).AdvanceTxnSafePoint(ctx, base+2)
re.NoError(err)
re.Equal(base+2, res.NewTxnSafePoint)
for i := 0; i < 2; i++ {
txnSafePoint, err := s.stores[2].LoadTxnSafePoint(ctx)
re.NoError(err)
re.Equal(base+2, txnSafePoint)
for _, store := range s.stores[0:2] {
txnSafePoint, err := store.LoadTxnSafePoint(ctx)
re.NoError(err)
re.Less(txnSafePoint, base+2)
}
}
// Each store tries `GetGCState` only once.
re.Equal(len(s.stores), callCounter)
}
func (s *testGCWithTiKVSuite) TestCompatibleTxnSafePointLoaderValueParsing() {
re := s.Require()
re.NoError(failpoint.Enable("tikvclient/noBuiltInTxnSafePointUpdater", "return"))
defer func() {
re.NoError(failpoint.Disable("tikvclient/noBuiltInTxnSafePointUpdater"))
}()
s.prepareClients(storeNullKeyspace)
store := s.stores[0]
hook := func(inner pdgc.GCStatesClient, ctx context.Context) (pdgc.GCState, error) {
return pdgc.GCState{}, status.Errorf(codes.Unimplemented, "simulated unimplemented error")
}
hookGCStatesClientForStore(store, hook)
// Load txn safe point once to enter fallback state.
ctx := context.Background()
_, err := store.LoadTxnSafePoint(ctx)
re.NoError(err)
fpname := "tikvclient/compatibleTxnSafePointLoaderEtcdGetResult"
re.NoError(failpoint.Enable(fpname, `return("empty")`))
defer func() {
re.NoError(failpoint.Disable(fpname))
}()
txnSafePoint, err := store.LoadTxnSafePoint(ctx)
re.NoError(err)
re.Equal(uint64(0), txnSafePoint)
re.NoError(failpoint.Enable(fpname, `return("value:")`))
txnSafePoint, err = store.LoadTxnSafePoint(ctx)
re.NoError(err)
re.Equal(uint64(0), txnSafePoint)
re.NoError(failpoint.Enable(fpname, `return("value:1")`))
txnSafePoint, err = store.LoadTxnSafePoint(ctx)
re.NoError(err)
re.Equal(uint64(1), txnSafePoint)
re.NoError(failpoint.Enable(fpname, `return("value:abcde")`))
txnSafePoint, err = store.LoadTxnSafePoint(ctx)
re.Error(err)
re.NoError(failpoint.Enable(fpname, `return("value:459579321342754819")`))
txnSafePoint, err = store.LoadTxnSafePoint(ctx)
re.NoError(err)
re.Equal(uint64(459579321342754819), txnSafePoint)
re.NoError(failpoint.Enable(fpname, `return("value:459579321342754819abcdefg")`))
txnSafePoint, err = store.LoadTxnSafePoint(ctx)
re.Error(err)
re.NoError(failpoint.Enable(fpname, `return("value:18446744073709551615")`))
txnSafePoint, err = store.LoadTxnSafePoint(ctx)
re.NoError(err)
re.Equal(uint64(18446744073709551615), txnSafePoint)
// Out of range of uint64
re.NoError(failpoint.Enable(fpname, `return("value:18446744073709551616")`))
txnSafePoint, err = store.LoadTxnSafePoint(ctx)
re.Error(err)
}

View File

@ -85,9 +85,9 @@ func mymakeKeys(rowNum int, prefix string) [][]byte {
func (s *testSafePointSuite) waitUntilErrorPlugIn(t uint64) {
for {
s.store.SaveSafePoint(t + 10)
s.store.SaveSafePointToSafePointKV(t + 10)
cachedTime := time.Now()
newSafePoint, err := s.store.LoadSafePoint()
newSafePoint, err := s.store.LoadSafePointFromSafePointKV()
if err == nil {
s.store.UpdateTxnSafePointCache(newSafePoint, cachedTime)
break

View File

@ -7,6 +7,7 @@ import (
"github.com/pingcap/kvproto/pkg/keyspacepb"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/tikv/client-go/v2/tikvrpc"
"github.com/tikv/pd/client/constants"
)
type (
@ -25,7 +26,7 @@ const (
const (
// NullspaceID is a special keyspace id that represents no keyspace exist.
NullspaceID KeyspaceID = 0xffffffff
NullspaceID = KeyspaceID(constants.NullKeyspaceID)
)
// ParseKeyspaceID retrieves the keyspaceID from the given keyspace-encoded key.

View File

@ -54,7 +54,7 @@ var (
TiKVRawkvCmdHistogram *prometheus.HistogramVec
TiKVRawkvSizeHistogram *prometheus.HistogramVec
TiKVTxnRegionsNumHistogram *prometheus.HistogramVec
TiKVLoadSafepointCounter *prometheus.CounterVec
TiKVLoadTxnSafePointCounter *prometheus.CounterVec
TiKVSecondaryLockCleanupFailureCounter *prometheus.CounterVec
TiKVRegionCacheCounter *prometheus.CounterVec
TiKVLoadRegionCounter *prometheus.CounterVec
@ -275,7 +275,7 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) {
ConstLabels: constLabels,
}, []string{LblType, LblScope})
TiKVLoadSafepointCounter = prometheus.NewCounterVec(
TiKVLoadTxnSafePointCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: namespace,
Subsystem: subsystem,
@ -922,7 +922,7 @@ func RegisterMetrics() {
prometheus.MustRegister(TiKVRawkvCmdHistogram)
prometheus.MustRegister(TiKVRawkvSizeHistogram)
prometheus.MustRegister(TiKVTxnRegionsNumHistogram)
prometheus.MustRegister(TiKVLoadSafepointCounter)
prometheus.MustRegister(TiKVLoadTxnSafePointCounter)
prometheus.MustRegister(TiKVSecondaryLockCleanupFailureCounter)
prometheus.MustRegister(TiKVRegionCacheCounter)
prometheus.MustRegister(TiKVLoadRegionCounter)

View File

@ -0,0 +1,139 @@
package tikv
import (
"context"
"crypto/tls"
"fmt"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/pkg/errors"
"github.com/tikv/client-go/v2/internal/apicodec"
"github.com/tikv/client-go/v2/util"
pd "github.com/tikv/pd/client"
etcdmvccpb "go.etcd.io/etcd/api/v3/mvccpb"
clientv3 "go.etcd.io/etcd/client/v3"
)
const (
unifiedTxnSafePointPath = "/tidb/store/gcworker/saved_safe_point"
keyspaceLevelTxnSafePointPath = "/keyspaces/tidb/%d/tidb/store/gcworker/saved_safe_point"
)
// compatibleTxnSafePointLoader is used to load txn safe point from etcd for old versions where the GetGCState API
// is not yet supported.
//
// The same functionality can be provided by EtcdSafePointKV. However, EtcdSafePointKV determines whether the key needs
// to be prefixed. Meanwhile, when client-go's user creates KVStore, it needs manually create EtcdSafePointKV first.
// This means, whether the etcd client is correctly prefixed depends on whether the user creates it correctly.
// Considering that TiDB is not the only user of client-go, this was a bad design, and we cannot assume the caller
// can easily make it correct (e.g., whether the txn safe point is stored in prefixed key depends on whether
// keyspace level GC is enabled for a keyspace). To minimize the risk of misusing, we rewrote a special util for loading
// it. The EtcdSafePointKV, on the contrary, will be deprecated in the future.
type compatibleTxnSafePointLoader struct {
mu sync.Mutex
etcdCli atomic.Pointer[clientv3.Client]
endpoints []string
tlsConfig *tls.Config
codec apicodec.Codec
}
func newCompatibleTxnSafePointLoader(codec apicodec.Codec, etcdEndpoints []string, tlsConfig *tls.Config) *compatibleTxnSafePointLoader {
return &compatibleTxnSafePointLoader{
endpoints: etcdEndpoints,
tlsConfig: tlsConfig,
codec: codec,
}
}
func (l *compatibleTxnSafePointLoader) getEtcdCli() (*clientv3.Client, error) {
l.mu.Lock()
defer l.mu.Unlock()
cli := l.etcdCli.Load()
// It may be concurrently initialized by other goroutine before we successfully acquire the mutex
if cli != nil {
return cli, nil
}
cli, err := createEtcdKV(l.endpoints, l.tlsConfig)
if err != nil {
return nil, errors.WithStack(err)
}
// Note that the atomic operation is still necessary though the function is protected by the mutex. It's because
// that the first attempt to load the pointer (at the beginning of loadTxnSafePoint) is not in the mutex's range,
// and it can be concurrent with the writing here, which is still unsafe.
l.etcdCli.Store(cli)
return cli, nil
}
func (*compatibleTxnSafePointLoader) etcdGet(ctx context.Context, cli *clientv3.Client, key string) (*clientv3.GetResponse, error) {
if val, err := util.EvalFailpoint("compatibleTxnSafePointLoaderEtcdGetResult"); err == nil {
str, ok := val.(string)
if !ok {
panic("invalid failpoint value for compatibleTxnSafePointLoaderEtcdGetResult, string is expected")
}
if str == "empty" {
return &clientv3.GetResponse{}, nil
}
if strings.HasPrefix(str, "value:") {
value := strings.TrimPrefix(str, "value:")
return &clientv3.GetResponse{
Kvs: []*etcdmvccpb.KeyValue{{Key: []byte(key), Value: []byte(value)}},
Count: 1,
}, nil
}
panic("invalid failpoint value for compatibleTxnSafePointLoaderEtcdGetResult, invalid format")
}
return cli.Get(ctx, key)
}
func (l *compatibleTxnSafePointLoader) loadTxnSafePoint(ctx context.Context) (uint64, error) {
cli := l.etcdCli.Load()
var err error
if cli == nil {
// Lazy create the etcd client.
cli, err = l.getEtcdCli()
if err != nil {
return 0, errors.WithStack(err)
}
}
key := unifiedTxnSafePointPath
keyspaceMeta := l.codec.GetKeyspaceMeta()
if pd.IsKeyspaceUsingKeyspaceLevelGC(keyspaceMeta) {
key = fmt.Sprintf(keyspaceLevelTxnSafePointPath, keyspaceMeta.Id)
}
// Follow the same implementation as the EtcdSafePointKV by setting the timeout 5 seconds.
ctx, cancel := context.WithTimeout(ctx, time.Second*5)
defer cancel()
resp, err := l.etcdGet(ctx, cli, key)
if err != nil {
return 0, errors.WithStack(err)
}
if len(resp.Kvs) == 0 {
return 0, nil
}
str := string(resp.Kvs[0].Value)
if len(str) == 0 {
return 0, nil
}
value, err := strconv.ParseUint(str, 10, 64)
if err != nil {
return 0, errors.WithStack(err)
}
return value, nil
}
func (l *compatibleTxnSafePointLoader) Close() error {
cli := l.etcdCli.Load()
if cli != nil {
return cli.Close()
}
return nil
}

View File

@ -58,10 +58,6 @@ type Storage interface {
// GetSafePointKV gets the SafePointKV.
GetSafePointKV() SafePointKV
// UpdateTxnSafePointCache updates the cached txn safe point, which is used for safety check of data access
// operations to prevent accessing GC-ed inconsistent data.
UpdateTxnSafePointCache(txnSafePoint uint64, now time.Time)
// SetOracle sets the Oracle.
SetOracle(oracle oracle.Oracle)

View File

@ -67,6 +67,7 @@ import (
"github.com/tikv/client-go/v2/txnkv/txnsnapshot"
"github.com/tikv/client-go/v2/util"
pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/clients/gc"
pdhttp "github.com/tikv/pd/client/http"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/pkg/caller"
@ -75,7 +76,9 @@ import (
atomicutil "go.uber.org/atomic"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/keepalive"
grpcStatus "google.golang.org/grpc/status"
)
const (
@ -135,11 +138,14 @@ type KVStore struct {
kv SafePointKV
gcStatesClient gc.GCStatesClient
gcStateCacheMu struct {
sync.RWMutex
cachedTxnSafePoint uint64
lastCacheTime time.Time
}
gcStatesAPIUnavailable atomic.Bool
compatibleTxnSafePointLoader *compatibleTxnSafePointLoader
// storeID -> safeTS, stored as map[uint64]uint64
// safeTS here will be used during the Stale Read process,
@ -165,9 +171,9 @@ func (s *KVStore) Go(f func()) error {
return s.gP.Run(f)
}
// UpdateTxnSafePointCache updates the cached txn safe point, which is used for safety check of data access
// updateTxnSafePointCache updates the cached txn safe point, which is used for safety check of data access
// operations to prevent accessing GC-ed inconsistent data.
func (s *KVStore) UpdateTxnSafePointCache(txnSafePoint uint64, now time.Time) {
func (s *KVStore) updateTxnSafePointCache(txnSafePoint uint64, now time.Time) {
s.gcStateCacheMu.Lock()
defer s.gcStateCacheMu.Unlock()
@ -175,6 +181,39 @@ func (s *KVStore) UpdateTxnSafePointCache(txnSafePoint uint64, now time.Time) {
s.gcStateCacheMu.cachedTxnSafePoint = txnSafePoint
}
func (s *KVStore) loadTxnSafePointInCompatibleMode(ctx context.Context) (uint64, error) {
res, err := s.compatibleTxnSafePointLoader.loadTxnSafePoint(ctx)
if err != nil {
logutil.BgLogger().Error("failed to load current txn safe point from PD in compatible mode", zap.Error(err))
metrics.TiKVLoadTxnSafePointCounter.WithLabelValues("fail_compatible").Inc()
return 0, errors.WithStack(err)
}
metrics.TiKVLoadTxnSafePointCounter.WithLabelValues("ok_compatible").Inc()
return res, nil
}
func (s *KVStore) loadTxnSafePoint(ctx context.Context) (uint64, error) {
if s.gcStatesAPIUnavailable.Load() {
// Print in debug level to avoid being too verbose. A warning log will be printed when the first fallback occurs.
logutil.Logger(ctx).Debug("GC states API is not available, which is possibly caused by cluster version < 9.0. Txn safe point updating will be fallen back to direct etcd reading.")
return s.loadTxnSafePointInCompatibleMode(ctx)
}
gcStates, err := s.gcStatesClient.GetGCState(ctx)
if err != nil {
if status, ok := grpcStatus.FromError(err); ok && status.Code() == codes.Unimplemented {
logutil.Logger(ctx).Warn("GC states API is not available, which is possibly caused by cluster version < 9.0. Txn safe point updating will be fallen back to direct etcd reading.", zap.Error(err))
s.gcStatesAPIUnavailable.Store(true)
return s.loadTxnSafePointInCompatibleMode(ctx)
}
logutil.Logger(ctx).Error("failed to load current txn safe point from PD", zap.Error(err))
metrics.TiKVLoadTxnSafePointCounter.WithLabelValues("fail").Inc()
return 0, err
}
metrics.TiKVLoadTxnSafePointCounter.WithLabelValues("ok").Inc()
return gcStates.TxnSafePoint, nil
}
// CheckVisibility checks if it is safe to read using given ts.
func (s *KVStore) CheckVisibility(startTS uint64) error {
s.gcStateCacheMu.RLock()
@ -294,25 +333,28 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Cl
}))
}
regionCache := locate.NewRegionCache(pdClient, opts...)
codec := pdClient.(*CodecPDClient).GetCodec()
etcdAddrs, etcdTlsCfg := spkv.extractConnectionInfo()
store := &KVStore{
clusterID: pdClient.GetClusterID(context.TODO()),
uuid: uuid,
oracle: o,
pdClient: pdClient.WithCallerComponent("kv-store"),
regionCache: regionCache,
kv: spkv,
replicaReadSeed: rand.Uint32(),
ctx: ctx,
cancel: cancel,
gP: NewSpool(128, 10*time.Second),
clusterID: pdClient.GetClusterID(context.TODO()),
uuid: uuid,
oracle: o,
pdClient: pdClient.WithCallerComponent("kv-store"),
regionCache: regionCache,
kv: spkv,
gcStatesClient: pdClient.GetGCStatesClient(uint32(codec.GetKeyspaceID())),
compatibleTxnSafePointLoader: newCompatibleTxnSafePointLoader(codec, etcdAddrs, etcdTlsCfg),
replicaReadSeed: rand.Uint32(),
ctx: ctx,
cancel: cancel,
gP: NewSpool(128, 10*time.Second),
}
keyspaceID := pdClient.(*CodecPDClient).GetCodec().GetKeyspaceID()
gcStates, err := pdClient.GetGCStatesClient(uint32(keyspaceID)).GetGCState(context.Background())
txnSafePoint, err := store.loadTxnSafePoint(context.Background())
if err != nil {
return nil, errors.WithStack(err)
}
store.UpdateTxnSafePointCache(gcStates.TxnSafePoint, time.Now())
store.updateTxnSafePointCache(txnSafePoint, time.Now())
store.clientMu.client = client.NewReqCollapse(client.NewInterceptedClient(tikvclient))
store.clientMu.client.SetEventListener(regionCache.GetClientEventListener())
@ -367,19 +409,18 @@ func (s *KVStore) IsLatchEnabled() bool {
func (s *KVStore) runTxnSafePointUpdater() {
defer s.wg.Done()
if _, e := util.EvalFailpoint("noBuiltInTxnSafePointUpdater"); e == nil {
return
}
d := pollTxnSafePointInterval
gcStatesClient := s.pdClient.GetGCStatesClient(uint32(s.getCodec().GetKeyspaceID()))
for {
select {
case now := <-time.After(d):
gcStates, err := gcStatesClient.GetGCState(context.Background())
txnSafePoint, err := s.loadTxnSafePoint(context.Background())
if err == nil {
metrics.TiKVLoadSafepointCounter.WithLabelValues("ok").Inc()
s.UpdateTxnSafePointCache(gcStates.TxnSafePoint, now)
s.updateTxnSafePointCache(txnSafePoint, now)
d = pollTxnSafePointInterval
} else {
metrics.TiKVLoadSafepointCounter.WithLabelValues("fail").Inc()
logutil.BgLogger().Error("fail to load txn safe point from pd", zap.Error(err))
d = pollTxnSafePointQuickRepeatInterval
}
case <-s.ctx.Done():
@ -388,10 +429,6 @@ func (s *KVStore) runTxnSafePointUpdater() {
}
}
func (s *KVStore) getCodec() Codec {
return s.pdClient.(*CodecPDClient).GetCodec()
}
// Begin a global transaction.
func (s *KVStore) Begin(opts ...TxnOption) (txn *transaction.KVTxn, err error) {
options := &transaction.TxnOptions{}
@ -469,6 +506,9 @@ func (s *KVStore) Close() error {
if err := s.kv.Close(); err != nil {
return err
}
if err := s.compatibleTxnSafePointLoader.Close(); err != nil {
return err
}
return nil
}

View File

@ -70,6 +70,8 @@ type SafePointKV interface {
Get(k string) (string, error)
GetWithPrefix(k string) ([]*mvccpb.KeyValue, error)
Close() error
extractConnectionInfo() (endpoints []string, tlsConfig *tls.Config)
}
// MockSafePointKV implements SafePointKV at mock test
@ -119,6 +121,10 @@ func (w *MockSafePointKV) Close() error {
return nil
}
func (w *MockSafePointKV) extractConnectionInfo() (endpoints []string, tlsConfig *tls.Config) {
return nil, nil
}
// option represents safePoint kv configuration.
type option struct {
prefix string
@ -135,7 +141,9 @@ func WithPrefix(prefix string) SafePointKVOpt {
// EtcdSafePointKV implements SafePointKV at runtime
type EtcdSafePointKV struct {
cli *clientv3.Client
addrs []string
tlsConfig *tls.Config
cli *clientv3.Client
}
// NewEtcdSafePointKV creates an instance of EtcdSafePointKV
@ -155,7 +163,11 @@ func NewEtcdSafePointKV(addrs []string, tlsConfig *tls.Config, opts ...SafePoint
etcdCli.Watcher = namespace.NewWatcher(etcdCli.Watcher, opt.prefix)
etcdCli.Lease = namespace.NewLease(etcdCli.Lease, opt.prefix)
}
return &EtcdSafePointKV{cli: etcdCli}, nil
return &EtcdSafePointKV{
addrs: addrs,
tlsConfig: tlsConfig,
cli: etcdCli,
}, nil
}
// Put implements the Put method for SafePointKV
@ -196,6 +208,11 @@ func (w *EtcdSafePointKV) Close() error {
return errors.WithStack(w.cli.Close())
}
func (w *EtcdSafePointKV) extractConnectionInfo() (endpoints []string, tlsConfig *tls.Config) {
return w.addrs, w.tlsConfig
}
// Deprecated: Do not use
func saveSafePoint(kv SafePointKV, t uint64) error {
s := strconv.FormatUint(t, 10)
err := kv.Put(GcSavedSafePoint, s)

View File

@ -46,6 +46,8 @@ import (
"github.com/tikv/client-go/v2/txnkv/txnlock"
"github.com/tikv/client-go/v2/txnkv/txnsnapshot"
pd "github.com/tikv/pd/client"
pdgc "github.com/tikv/pd/client/clients/gc"
clientv3 "go.etcd.io/etcd/client/v3"
)
// StoreProbe wraps KVStore and exposes internal states for testing purpose.
@ -91,16 +93,20 @@ func (s StoreProbe) SendTxnHeartbeat(ctx context.Context, key []byte, startTS ui
return newTTL, err
}
// LoadSafePoint from safepoint kv.
func (s StoreProbe) LoadSafePoint() (uint64, error) {
// LoadSafePointFromSafePointKV from safepoint kv.
func (s StoreProbe) LoadSafePointFromSafePointKV() (uint64, error) {
return loadSafePoint(s.GetSafePointKV())
}
// SaveSafePoint saves safepoint to kv.
func (s StoreProbe) SaveSafePoint(v uint64) error {
// SaveSafePointToSafePointKV saves safepoint to kv.
func (s StoreProbe) SaveSafePointToSafePointKV(v uint64) error {
return saveSafePoint(s.GetSafePointKV(), v)
}
func (s StoreProbe) LoadTxnSafePoint(ctx context.Context) (uint64, error) {
return s.KVStore.loadTxnSafePoint(ctx)
}
// SetRegionCacheStore is used to set a store in region cache, for testing only
func (s StoreProbe) SetRegionCacheStore(id uint64, storeType tikvrpc.EndpointType, state uint64, labels []*metapb.StoreLabel) {
s.regionCache.SetRegionCacheStore(id, "", "", storeType, state, labels)
@ -152,6 +158,22 @@ outerLoop:
return result, nil
}
func (s StoreProbe) UpdateTxnSafePointCache(txnSafePoint uint64, now time.Time) {
s.KVStore.updateTxnSafePointCache(txnSafePoint, now)
}
func (s StoreProbe) GetGCStatesClient() pdgc.GCStatesClient {
return s.gcStatesClient
}
func (s StoreProbe) ReplaceGCStatesClient(c pdgc.GCStatesClient) {
s.KVStore.gcStatesClient = c
}
func (s StoreProbe) GetCompatibleTxnSafePointLoaderUnderlyingEtcdClient() *clientv3.Client {
return s.KVStore.compatibleTxnSafePointLoader.etcdCli.Load()
}
// LockResolverProbe wraps a LockResolver and exposes internal stats for testing purpose.
type LockResolverProbe struct {
*txnlock.LockResolverProbe