diff --git a/.build.ps1 b/.build.ps1 index 6f29acab3cc15..bcd506c49f874 100644 --- a/.build.ps1 +++ b/.build.ps1 @@ -287,14 +287,6 @@ task ExplainTest -If (-not ((Get-Content cmd\explaintest\r\explain.result -Raw) } } -# Synopsis: Check dependency. -task CheckDep { - $list = go list -json github.com/pingcap/tidb/store/tikv | ConvertFrom-Json - if ($list.Imports | Where-Object { Select-String -Pattern '^github.com/pingcap/parser$' -InputObject $_ }) { - throw 'incorrect import of github.com/pingcap/parser' - } -} - # Synopsis: Run unit tests. task GoTest BuildFailPoint, { Enable-FailPoint @@ -343,14 +335,6 @@ task GoLeakTest BuildFailPoint, { $env:TZ = $Task.Data.tz } -# Synopsis: Run some tests with real TiKV. -task TiKVIntegrationTest BuildFailPoint, { - Enable-FailPoint - { & $GO test -p $P github.com/pingcap/tidb/store/tikv -with-tikv=true } -} -Done { - Disable-FailPoint -} - # Synopsis: Ensure generated code is up to date. task GoGenerate { exec { & $GO generate ./... } @@ -361,7 +345,7 @@ task GoGenerate { } # Synopsis: Run common tests. -task Test ExplainTest, CheckDep, GoTest, GoGenerate +task Test ExplainTest, GoTest, GoGenerate # Synopsis: Check and Test. task Dev Check, Test diff --git a/Makefile b/Makefile index d09ed06f2c528..6451340aed4e3 100644 --- a/Makefile +++ b/Makefile @@ -110,7 +110,7 @@ test: test_part_1 test_part_2 test_part_1: checklist explaintest -test_part_2: checkdep gotest gogenerate +test_part_2: gotest gogenerate explaintest: server_check @cd cmd/explaintest && ./run-tests.sh -s ../../bin/tidb-server @@ -153,10 +153,6 @@ leak: failpoint-enable $(GOTEST) -tags leak $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) -tikv_integration_test: failpoint-enable - $(GOTEST) ./store/tikv/. -with-tikv=true || { $(FAILPOINT_DISABLE); exit 1; } - @$(FAILPOINT_DISABLE) - server: ifeq ($(TARGET), "") CGO_ENABLED=1 $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server tidb-server/main.go @@ -208,9 +204,6 @@ failpoint-disable: tools/bin/failpoint-ctl # Restoring gofail failpoints... @$(FAILPOINT_DISABLE) -checkdep: - $(GO) list -f '{{ join .Imports "\n" }}' github.com/pingcap/tidb/store/tikv | grep ^github.com/pingcap/parser$$ || exit 0; exit 1 - tools/bin/megacheck: tools/check/go.mod cd tools/check; \ $(GO) build -o ../bin/megacheck honnef.co/go/tools/cmd/megacheck diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go deleted file mode 100644 index ec1557ff79ed5..0000000000000 --- a/store/tikv/2pc.go +++ /dev/null @@ -1,1781 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - "encoding/hex" - "math" - "math/rand" - "strings" - "sync" - "sync/atomic" - "time" - "unsafe" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/config" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/unionstore" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" - zap "go.uber.org/zap" -) - -// If the duration of a single request exceeds the slowRequestThreshold, a warning log will be logged. -const slowRequestThreshold = time.Minute - -type twoPhaseCommitAction interface { - handleSingleBatch(*twoPhaseCommitter, *Backoffer, batchMutations) error - tiKVTxnRegionsNumHistogram() prometheus.Observer - String() string -} - -// Global variable set by config file. -var ( - ManagedLockTTL uint64 = 20000 // 20s -) - -// twoPhaseCommitter executes a two-phase commit protocol. -type twoPhaseCommitter struct { - store *KVStore - txn *KVTxn - startTS uint64 - mutations *memBufferMutations - lockTTL uint64 - commitTS uint64 - priority kvrpcpb.CommandPri - sessionID uint64 // sessionID is used for log. - cleanWg sync.WaitGroup - detail unsafe.Pointer - txnSize int - hasNoNeedCommitKeys bool - - primaryKey []byte - forUpdateTS uint64 - - mu struct { - sync.RWMutex - undeterminedErr error // undeterminedErr saves the rpc error we encounter when commit primary key. - committed bool - } - syncLog bool - // For pessimistic transaction - isPessimistic bool - isFirstLock bool - // regionTxnSize stores the number of keys involved in each region - regionTxnSize map[uint64]int - // Used by pessimistic transaction and large transaction. - ttlManager - - testingKnobs struct { - acAfterCommitPrimary chan struct{} - bkAfterCommitPrimary chan struct{} - noFallBack bool - } - - useAsyncCommit uint32 - minCommitTS uint64 - maxCommitTS uint64 - prewriteStarted bool - prewriteCancelled uint32 - prewriteFailed uint32 - useOnePC uint32 - onePCCommitTS uint64 - - hasTriedAsyncCommit bool - hasTriedOnePC bool - - // doingAmend means the amend prewrite is ongoing. - doingAmend bool - - binlog BinlogExecutor - - resourceGroupTag []byte -} - -type memBufferMutations struct { - storage *unionstore.MemDB - handles []unionstore.MemKeyHandle -} - -func newMemBufferMutations(sizeHint int, storage *unionstore.MemDB) *memBufferMutations { - return &memBufferMutations{ - handles: make([]unionstore.MemKeyHandle, 0, sizeHint), - storage: storage, - } -} - -func (m *memBufferMutations) Len() int { - return len(m.handles) -} - -func (m *memBufferMutations) GetKey(i int) []byte { - return m.storage.GetKeyByHandle(m.handles[i]) -} - -func (m *memBufferMutations) GetKeys() [][]byte { - ret := make([][]byte, m.Len()) - for i := range ret { - ret[i] = m.GetKey(i) - } - return ret -} - -func (m *memBufferMutations) GetValue(i int) []byte { - v, _ := m.storage.GetValueByHandle(m.handles[i]) - return v -} - -func (m *memBufferMutations) GetOp(i int) kvrpcpb.Op { - return kvrpcpb.Op(m.handles[i].UserData >> 1) -} - -func (m *memBufferMutations) IsPessimisticLock(i int) bool { - return m.handles[i].UserData&1 != 0 -} - -func (m *memBufferMutations) Slice(from, to int) CommitterMutations { - return &memBufferMutations{ - handles: m.handles[from:to], - storage: m.storage, - } -} - -func (m *memBufferMutations) Push(op kvrpcpb.Op, isPessimisticLock bool, handle unionstore.MemKeyHandle) { - aux := uint16(op) << 1 - if isPessimisticLock { - aux |= 1 - } - handle.UserData = aux - m.handles = append(m.handles, handle) -} - -// CommitterMutations contains the mutations to be submitted. -type CommitterMutations interface { - Len() int - GetKey(i int) []byte - GetKeys() [][]byte - GetOp(i int) kvrpcpb.Op - GetValue(i int) []byte - IsPessimisticLock(i int) bool - Slice(from, to int) CommitterMutations -} - -// PlainMutations contains transaction operations. -type PlainMutations struct { - ops []kvrpcpb.Op - keys [][]byte - values [][]byte - isPessimisticLock []bool -} - -// NewPlainMutations creates a PlainMutations object with sizeHint reserved. -func NewPlainMutations(sizeHint int) PlainMutations { - return PlainMutations{ - ops: make([]kvrpcpb.Op, 0, sizeHint), - keys: make([][]byte, 0, sizeHint), - values: make([][]byte, 0, sizeHint), - isPessimisticLock: make([]bool, 0, sizeHint), - } -} - -// Slice return a sub mutations in range [from, to). -func (c *PlainMutations) Slice(from, to int) CommitterMutations { - var res PlainMutations - res.keys = c.keys[from:to] - if c.ops != nil { - res.ops = c.ops[from:to] - } - if c.values != nil { - res.values = c.values[from:to] - } - if c.isPessimisticLock != nil { - res.isPessimisticLock = c.isPessimisticLock[from:to] - } - return &res -} - -// Push another mutation into mutations. -func (c *PlainMutations) Push(op kvrpcpb.Op, key []byte, value []byte, isPessimisticLock bool) { - c.ops = append(c.ops, op) - c.keys = append(c.keys, key) - c.values = append(c.values, value) - c.isPessimisticLock = append(c.isPessimisticLock, isPessimisticLock) -} - -// Len returns the count of mutations. -func (c *PlainMutations) Len() int { - return len(c.keys) -} - -// GetKey returns the key at index. -func (c *PlainMutations) GetKey(i int) []byte { - return c.keys[i] -} - -// GetKeys returns the keys. -func (c *PlainMutations) GetKeys() [][]byte { - return c.keys -} - -// GetOps returns the key ops. -func (c *PlainMutations) GetOps() []kvrpcpb.Op { - return c.ops -} - -// GetValues returns the key values. -func (c *PlainMutations) GetValues() [][]byte { - return c.values -} - -// GetPessimisticFlags returns the key pessimistic flags. -func (c *PlainMutations) GetPessimisticFlags() []bool { - return c.isPessimisticLock -} - -// GetOp returns the key op at index. -func (c *PlainMutations) GetOp(i int) kvrpcpb.Op { - return c.ops[i] -} - -// GetValue returns the key value at index. -func (c *PlainMutations) GetValue(i int) []byte { - if len(c.values) <= i { - return nil - } - return c.values[i] -} - -// IsPessimisticLock returns the key pessimistic flag at index. -func (c *PlainMutations) IsPessimisticLock(i int) bool { - return c.isPessimisticLock[i] -} - -// PlainMutation represents a single transaction operation. -type PlainMutation struct { - KeyOp kvrpcpb.Op - Key []byte - Value []byte - IsPessimisticLock bool -} - -// MergeMutations append input mutations into current mutations. -func (c *PlainMutations) MergeMutations(mutations PlainMutations) { - c.ops = append(c.ops, mutations.ops...) - c.keys = append(c.keys, mutations.keys...) - c.values = append(c.values, mutations.values...) - c.isPessimisticLock = append(c.isPessimisticLock, mutations.isPessimisticLock...) -} - -// AppendMutation merges a single Mutation into the current mutations. -func (c *PlainMutations) AppendMutation(mutation PlainMutation) { - c.ops = append(c.ops, mutation.KeyOp) - c.keys = append(c.keys, mutation.Key) - c.values = append(c.values, mutation.Value) - c.isPessimisticLock = append(c.isPessimisticLock, mutation.IsPessimisticLock) -} - -// newTwoPhaseCommitter creates a twoPhaseCommitter. -func newTwoPhaseCommitter(txn *KVTxn, sessionID uint64) (*twoPhaseCommitter, error) { - return &twoPhaseCommitter{ - store: txn.store, - txn: txn, - startTS: txn.StartTS(), - sessionID: sessionID, - regionTxnSize: map[uint64]int{}, - ttlManager: ttlManager{ - ch: make(chan struct{}), - }, - isPessimistic: txn.IsPessimistic(), - binlog: txn.binlog, - }, nil -} - -func (c *twoPhaseCommitter) extractKeyExistsErr(err *tikverr.ErrKeyExist) error { - if !c.txn.us.HasPresumeKeyNotExists(err.GetKey()) { - return errors.Errorf("session %d, existErr for key:%s should not be nil", c.sessionID, err.GetKey()) - } - return errors.Trace(err) -} - -// KVFilter is a filter that filters out unnecessary KV pairs. -type KVFilter interface { - // IsUnnecessaryKeyValue returns whether this KV pair should be committed. - IsUnnecessaryKeyValue(key, value []byte, flags kv.KeyFlags) bool -} - -func (c *twoPhaseCommitter) initKeysAndMutations() error { - var size, putCnt, delCnt, lockCnt, checkCnt int - - txn := c.txn - memBuf := txn.GetMemBuffer() - sizeHint := txn.us.GetMemBuffer().Len() - c.mutations = newMemBufferMutations(sizeHint, memBuf) - c.isPessimistic = txn.IsPessimistic() - filter := txn.kvFilter - - var err error - for it := memBuf.IterWithFlags(nil, nil); it.Valid(); err = it.Next() { - _ = err - key := it.Key() - flags := it.Flags() - var value []byte - var op kvrpcpb.Op - - if !it.HasValue() { - if !flags.HasLocked() { - continue - } - op = kvrpcpb.Op_Lock - lockCnt++ - } else { - value = it.Value() - if len(value) > 0 { - isUnnecessaryKV := filter != nil && filter.IsUnnecessaryKeyValue(key, value, flags) - if isUnnecessaryKV { - if !flags.HasLocked() { - continue - } - // If the key was locked before, we should prewrite the lock even if - // the KV needn't be committed according to the filter. Otherwise, we - // were forgetting removing pessimistic locks added before. - op = kvrpcpb.Op_Lock - lockCnt++ - } else { - op = kvrpcpb.Op_Put - if flags.HasPresumeKeyNotExists() { - op = kvrpcpb.Op_Insert - } - putCnt++ - } - } else { - if !txn.IsPessimistic() && flags.HasPresumeKeyNotExists() { - // delete-your-writes keys in optimistic txn need check not exists in prewrite-phase - // due to `Op_CheckNotExists` doesn't prewrite lock, so mark those keys should not be used in commit-phase. - op = kvrpcpb.Op_CheckNotExists - checkCnt++ - memBuf.UpdateFlags(key, kv.SetPrewriteOnly) - } else { - // normal delete keys in optimistic txn can be delete without not exists checking - // delete-your-writes keys in pessimistic txn can ensure must be no exists so can directly delete them - op = kvrpcpb.Op_Del - delCnt++ - } - } - } - - var isPessimistic bool - if flags.HasLocked() { - isPessimistic = c.isPessimistic - } - c.mutations.Push(op, isPessimistic, it.Handle()) - size += len(key) + len(value) - - if len(c.primaryKey) == 0 && op != kvrpcpb.Op_CheckNotExists { - c.primaryKey = key - } - } - - if c.mutations.Len() == 0 { - return nil - } - c.txnSize = size - - const logEntryCount = 10000 - const logSize = 4 * 1024 * 1024 // 4MB - if c.mutations.Len() > logEntryCount || size > logSize { - logutil.BgLogger().Info("[BIG_TXN]", - zap.Uint64("session", c.sessionID), - zap.String("key sample", kv.StrKey(c.mutations.GetKey(0))), - zap.Int("size", size), - zap.Int("keys", c.mutations.Len()), - zap.Int("puts", putCnt), - zap.Int("dels", delCnt), - zap.Int("locks", lockCnt), - zap.Int("checks", checkCnt), - zap.Uint64("txnStartTS", txn.startTS)) - } - - // Sanity check for startTS. - if txn.StartTS() == math.MaxUint64 { - err = errors.Errorf("try to commit with invalid txnStartTS: %d", txn.StartTS()) - logutil.BgLogger().Error("commit failed", - zap.Uint64("session", c.sessionID), - zap.Error(err)) - return errors.Trace(err) - } - - commitDetail := &util.CommitDetails{WriteSize: size, WriteKeys: c.mutations.Len()} - metrics.TiKVTxnWriteKVCountHistogram.Observe(float64(commitDetail.WriteKeys)) - metrics.TiKVTxnWriteSizeHistogram.Observe(float64(commitDetail.WriteSize)) - c.hasNoNeedCommitKeys = checkCnt > 0 - c.lockTTL = txnLockTTL(txn.startTime, size) - c.priority = txn.priority.ToPB() - c.syncLog = txn.syncLog - c.resourceGroupTag = txn.resourceGroupTag - c.setDetail(commitDetail) - return nil -} - -func (c *twoPhaseCommitter) primary() []byte { - if len(c.primaryKey) == 0 { - return c.mutations.GetKey(0) - } - return c.primaryKey -} - -// asyncSecondaries returns all keys that must be checked in the recovery phase of an async commit. -func (c *twoPhaseCommitter) asyncSecondaries() [][]byte { - secondaries := make([][]byte, 0, c.mutations.Len()) - for i := 0; i < c.mutations.Len(); i++ { - k := c.mutations.GetKey(i) - if bytes.Equal(k, c.primary()) || c.mutations.GetOp(i) == kvrpcpb.Op_CheckNotExists { - continue - } - secondaries = append(secondaries, k) - } - return secondaries -} - -const bytesPerMiB = 1024 * 1024 - -func txnLockTTL(startTime time.Time, txnSize int) uint64 { - // Increase lockTTL for large transactions. - // The formula is `ttl = ttlFactor * sqrt(sizeInMiB)`. - // When writeSize is less than 256KB, the base ttl is defaultTTL (3s); - // When writeSize is 1MiB, 4MiB, or 10MiB, ttl is 6s, 12s, 20s correspondingly; - lockTTL := defaultLockTTL - if txnSize >= txnCommitBatchSize { - sizeMiB := float64(txnSize) / bytesPerMiB - lockTTL = uint64(float64(ttlFactor) * math.Sqrt(sizeMiB)) - if lockTTL < defaultLockTTL { - lockTTL = defaultLockTTL - } - if lockTTL > ManagedLockTTL { - lockTTL = ManagedLockTTL - } - } - - // Increase lockTTL by the transaction's read time. - // When resolving a lock, we compare current ts and startTS+lockTTL to decide whether to clean up. If a txn - // takes a long time to read, increasing its TTL will help to prevent it from been aborted soon after prewrite. - elapsed := time.Since(startTime) / time.Millisecond - return lockTTL + uint64(elapsed) -} - -var preSplitDetectThreshold uint32 = 100000 -var preSplitSizeThreshold uint32 = 32 << 20 - -// doActionOnMutations groups keys into primary batch and secondary batches, if primary batch exists in the key, -// it does action on primary batch first, then on secondary batches. If action is commit, secondary batches -// is done in background goroutine. -func (c *twoPhaseCommitter) doActionOnMutations(bo *Backoffer, action twoPhaseCommitAction, mutations CommitterMutations) error { - if mutations.Len() == 0 { - return nil - } - groups, err := c.groupMutations(bo, mutations) - if err != nil { - return errors.Trace(err) - } - - // This is redundant since `doActionOnGroupMutations` will still split groups into batches and - // check the number of batches. However we don't want the check fail after any code changes. - c.checkOnePCFallBack(action, len(groups)) - - return c.doActionOnGroupMutations(bo, action, groups) -} - -type groupedMutations struct { - region RegionVerID - mutations CommitterMutations -} - -// groupSortedMutationsByRegion separates keys into groups by their belonging Regions. -func groupSortedMutationsByRegion(c *RegionCache, bo *retry.Backoffer, m CommitterMutations) ([]groupedMutations, error) { - var ( - groups []groupedMutations - lastLoc *KeyLocation - ) - lastUpperBound := 0 - for i := 0; i < m.Len(); i++ { - if lastLoc == nil || !lastLoc.Contains(m.GetKey(i)) { - if lastLoc != nil { - groups = append(groups, groupedMutations{ - region: lastLoc.Region, - mutations: m.Slice(lastUpperBound, i), - }) - lastUpperBound = i - } - var err error - lastLoc, err = c.LocateKey(bo, m.GetKey(i)) - if err != nil { - return nil, errors.Trace(err) - } - } - } - if lastLoc != nil { - groups = append(groups, groupedMutations{ - region: lastLoc.Region, - mutations: m.Slice(lastUpperBound, m.Len()), - }) - } - return groups, nil -} - -// groupMutations groups mutations by region, then checks for any large groups and in that case pre-splits the region. -func (c *twoPhaseCommitter) groupMutations(bo *Backoffer, mutations CommitterMutations) ([]groupedMutations, error) { - groups, err := groupSortedMutationsByRegion(c.store.regionCache, bo, mutations) - if err != nil { - return nil, errors.Trace(err) - } - - // Pre-split regions to avoid too much write workload into a single region. - // In the large transaction case, this operation is important to avoid TiKV 'server is busy' error. - var didPreSplit bool - preSplitDetectThresholdVal := atomic.LoadUint32(&preSplitDetectThreshold) - for _, group := range groups { - if uint32(group.mutations.Len()) >= preSplitDetectThresholdVal { - logutil.BgLogger().Info("2PC detect large amount of mutations on a single region", - zap.Uint64("region", group.region.GetID()), - zap.Int("mutations count", group.mutations.Len())) - if c.store.preSplitRegion(bo.GetCtx(), group) { - didPreSplit = true - } - } - } - // Reload region cache again. - if didPreSplit { - groups, err = groupSortedMutationsByRegion(c.store.regionCache, bo, mutations) - if err != nil { - return nil, errors.Trace(err) - } - } - - return groups, nil -} - -// doActionOnGroupedMutations splits groups into batches (there is one group per region, and potentially many batches per group, but all mutations -// in a batch will belong to the same region). -func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPhaseCommitAction, groups []groupedMutations) error { - action.tiKVTxnRegionsNumHistogram().Observe(float64(len(groups))) - - var sizeFunc = c.keySize - - switch act := action.(type) { - case actionPrewrite: - // Do not update regionTxnSize on retries. They are not used when building a PrewriteRequest. - if !act.retry { - for _, group := range groups { - c.regionTxnSize[group.region.GetID()] = group.mutations.Len() - } - } - sizeFunc = c.keyValueSize - atomic.AddInt32(&c.getDetail().PrewriteRegionNum, int32(len(groups))) - case actionPessimisticLock: - if act.LockCtx.Stats != nil { - act.LockCtx.Stats.RegionNum = int32(len(groups)) - } - } - - batchBuilder := newBatched(c.primary()) - for _, group := range groups { - batchBuilder.appendBatchMutationsBySize(group.region, group.mutations, sizeFunc, txnCommitBatchSize) - } - firstIsPrimary := batchBuilder.setPrimary() - - actionCommit, actionIsCommit := action.(actionCommit) - _, actionIsCleanup := action.(actionCleanup) - _, actionIsPessimiticLock := action.(actionPessimisticLock) - - c.checkOnePCFallBack(action, len(batchBuilder.allBatches())) - - var err error - if val, err := util.EvalFailpoint("skipKeyReturnOK"); err == nil { - valStr, ok := val.(string) - if ok && c.sessionID > 0 { - if firstIsPrimary && actionIsPessimiticLock { - logutil.Logger(bo.GetCtx()).Warn("pessimisticLock failpoint", zap.String("valStr", valStr)) - switch valStr { - case "pessimisticLockSkipPrimary": - err = c.doActionOnBatches(bo, action, batchBuilder.allBatches()) - return err - case "pessimisticLockSkipSecondary": - err = c.doActionOnBatches(bo, action, batchBuilder.primaryBatch()) - return err - } - } - } - } - if _, err := util.EvalFailpoint("pessimisticRollbackDoNth"); err == nil { - _, actionIsPessimisticRollback := action.(actionPessimisticRollback) - if actionIsPessimisticRollback && c.sessionID > 0 { - logutil.Logger(bo.GetCtx()).Warn("pessimisticRollbackDoNth failpoint") - return nil - } - } - - if firstIsPrimary && - ((actionIsCommit && !c.isAsyncCommit()) || actionIsCleanup || actionIsPessimiticLock) { - // primary should be committed(not async commit)/cleanup/pessimistically locked first - err = c.doActionOnBatches(bo, action, batchBuilder.primaryBatch()) - if err != nil { - return errors.Trace(err) - } - if actionIsCommit && c.testingKnobs.bkAfterCommitPrimary != nil && c.testingKnobs.acAfterCommitPrimary != nil { - c.testingKnobs.acAfterCommitPrimary <- struct{}{} - <-c.testingKnobs.bkAfterCommitPrimary - } - batchBuilder.forgetPrimary() - } - // Already spawned a goroutine for async commit transaction. - if actionIsCommit && !actionCommit.retry && !c.isAsyncCommit() { - secondaryBo := retry.NewBackofferWithVars(context.Background(), CommitSecondaryMaxBackoff, c.txn.vars) - go func() { - if c.sessionID > 0 { - if v, err := util.EvalFailpoint("beforeCommitSecondaries"); err == nil { - if s, ok := v.(string); !ok { - logutil.Logger(bo.GetCtx()).Info("[failpoint] sleep 2s before commit secondary keys", - zap.Uint64("sessionID", c.sessionID), zap.Uint64("txnStartTS", c.startTS), zap.Uint64("txnCommitTS", c.commitTS)) - time.Sleep(2 * time.Second) - } else if s == "skip" { - logutil.Logger(bo.GetCtx()).Info("[failpoint] injected skip committing secondaries", - zap.Uint64("sessionID", c.sessionID), zap.Uint64("txnStartTS", c.startTS), zap.Uint64("txnCommitTS", c.commitTS)) - return - } - } - } - - e := c.doActionOnBatches(secondaryBo, action, batchBuilder.allBatches()) - if e != nil { - logutil.BgLogger().Debug("2PC async doActionOnBatches", - zap.Uint64("session", c.sessionID), - zap.Stringer("action type", action), - zap.Error(e)) - metrics.SecondaryLockCleanupFailureCounterCommit.Inc() - } - }() - } else { - err = c.doActionOnBatches(bo, action, batchBuilder.allBatches()) - } - return errors.Trace(err) -} - -// doActionOnBatches does action to batches in parallel. -func (c *twoPhaseCommitter) doActionOnBatches(bo *Backoffer, action twoPhaseCommitAction, batches []batchMutations) error { - if len(batches) == 0 { - return nil - } - - noNeedFork := len(batches) == 1 - if !noNeedFork { - if ac, ok := action.(actionCommit); ok && ac.retry { - noNeedFork = true - } - } - if noNeedFork { - for _, b := range batches { - e := action.handleSingleBatch(c, bo, b) - if e != nil { - logutil.BgLogger().Debug("2PC doActionOnBatches failed", - zap.Uint64("session", c.sessionID), - zap.Stringer("action type", action), - zap.Error(e), - zap.Uint64("txnStartTS", c.startTS)) - return errors.Trace(e) - } - } - return nil - } - rateLim := len(batches) - // Set rateLim here for the large transaction. - // If the rate limit is too high, tikv will report service is busy. - // If the rate limit is too low, we can't full utilize the tikv's throughput. - // TODO: Find a self-adaptive way to control the rate limit here. - if rateLim > config.GetGlobalConfig().CommitterConcurrency { - rateLim = config.GetGlobalConfig().CommitterConcurrency - } - batchExecutor := newBatchExecutor(rateLim, c, action, bo) - err := batchExecutor.process(batches) - return errors.Trace(err) -} - -func (c *twoPhaseCommitter) keyValueSize(key, value []byte) int { - return len(key) + len(value) -} - -func (c *twoPhaseCommitter) keySize(key, value []byte) int { - return len(key) -} - -type ttlManagerState uint32 - -const ( - stateUninitialized ttlManagerState = iota - stateRunning - stateClosed -) - -type ttlManager struct { - state ttlManagerState - ch chan struct{} - lockCtx *kv.LockCtx -} - -func (tm *ttlManager) run(c *twoPhaseCommitter, lockCtx *kv.LockCtx) { - // Run only once. - if !atomic.CompareAndSwapUint32((*uint32)(&tm.state), uint32(stateUninitialized), uint32(stateRunning)) { - return - } - tm.lockCtx = lockCtx - noKeepAlive := false - if _, err := util.EvalFailpoint("doNotKeepAlive"); err == nil { - noKeepAlive = true - } - - if !noKeepAlive { - go tm.keepAlive(c) - } -} - -func (tm *ttlManager) close() { - if !atomic.CompareAndSwapUint32((*uint32)(&tm.state), uint32(stateRunning), uint32(stateClosed)) { - return - } - close(tm.ch) -} - -const keepAliveMaxBackoff = 20000 // 20 seconds -const pessimisticLockMaxBackoff = 600000 // 10 minutes -const maxConsecutiveFailure = 10 - -func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { - // Ticker is set to 1/2 of the ManagedLockTTL. - ticker := time.NewTicker(time.Duration(atomic.LoadUint64(&ManagedLockTTL)) * time.Millisecond / 2) - defer ticker.Stop() - keepFail := 0 - for { - select { - case <-tm.ch: - return - case <-ticker.C: - // If kill signal is received, the ttlManager should exit. - if tm.lockCtx != nil && tm.lockCtx.Killed != nil && atomic.LoadUint32(tm.lockCtx.Killed) != 0 { - return - } - bo := retry.NewBackofferWithVars(context.Background(), keepAliveMaxBackoff, c.txn.vars) - now, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) - if err != nil { - logutil.Logger(bo.GetCtx()).Warn("keepAlive get tso fail", - zap.Error(err)) - return - } - - uptime := uint64(oracle.ExtractPhysical(now) - oracle.ExtractPhysical(c.startTS)) - if uptime > config.GetGlobalConfig().MaxTxnTTL { - // Checks maximum lifetime for the ttlManager, so when something goes wrong - // the key will not be locked forever. - logutil.Logger(bo.GetCtx()).Info("ttlManager live up to its lifetime", - zap.Uint64("txnStartTS", c.startTS), - zap.Uint64("uptime", uptime), - zap.Uint64("maxTxnTTL", config.GetGlobalConfig().MaxTxnTTL)) - metrics.TiKVTTLLifeTimeReachCounter.Inc() - // the pessimistic locks may expire if the ttl manager has timed out, set `LockExpired` flag - // so that this transaction could only commit or rollback with no more statement executions - if c.isPessimistic && tm.lockCtx != nil && tm.lockCtx.LockExpired != nil { - atomic.StoreUint32(tm.lockCtx.LockExpired, 1) - } - return - } - - newTTL := uptime + atomic.LoadUint64(&ManagedLockTTL) - logutil.Logger(bo.GetCtx()).Info("send TxnHeartBeat", - zap.Uint64("startTS", c.startTS), zap.Uint64("newTTL", newTTL)) - startTime := time.Now() - _, stopHeartBeat, err := sendTxnHeartBeat(bo, c.store, c.primary(), c.startTS, newTTL) - if err != nil { - keepFail++ - metrics.TxnHeartBeatHistogramError.Observe(time.Since(startTime).Seconds()) - logutil.Logger(bo.GetCtx()).Debug("send TxnHeartBeat failed", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - if stopHeartBeat || keepFail > maxConsecutiveFailure { - logutil.Logger(bo.GetCtx()).Warn("stop TxnHeartBeat", - zap.Error(err), - zap.Int("consecutiveFailure", keepFail), - zap.Uint64("txnStartTS", c.startTS)) - return - } - continue - } - keepFail = 0 - metrics.TxnHeartBeatHistogramOK.Observe(time.Since(startTime).Seconds()) - } - } -} - -func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, ttl uint64) (newTTL uint64, stopHeartBeat bool, err error) { - req := tikvrpc.NewRequest(tikvrpc.CmdTxnHeartBeat, &kvrpcpb.TxnHeartBeatRequest{ - PrimaryLock: primary, - StartVersion: startTS, - AdviseLockTtl: ttl, - }) - for { - loc, err := store.GetRegionCache().LocateKey(bo, primary) - if err != nil { - return 0, false, errors.Trace(err) - } - resp, err := store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) - if err != nil { - return 0, false, errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return 0, false, errors.Trace(err) - } - if regionErr != nil { - // For other region error and the fake region error, backoff because - // there's something wrong. - // For the real EpochNotMatch error, don't backoff. - if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return 0, false, errors.Trace(err) - } - } - continue - } - if resp.Resp == nil { - return 0, false, errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.TxnHeartBeatResponse) - if keyErr := cmdResp.GetError(); keyErr != nil { - return 0, true, errors.Errorf("txn %d heartbeat fail, primary key = %v, err = %s", startTS, hex.EncodeToString(primary), extractKeyErr(keyErr)) - } - return cmdResp.GetLockTtl(), false, nil - } -} - -// checkAsyncCommit checks if async commit protocol is available for current transaction commit, true is returned if possible. -func (c *twoPhaseCommitter) checkAsyncCommit() bool { - // Disable async commit in local transactions - if c.txn.GetScope() != oracle.GlobalTxnScope { - return false - } - - asyncCommitCfg := config.GetGlobalConfig().TiKVClient.AsyncCommit - // TODO the keys limit need more tests, this value makes the unit test pass by now. - // Async commit is not compatible with Binlog because of the non unique timestamp issue. - if c.sessionID > 0 && c.txn.enableAsyncCommit && - uint(c.mutations.Len()) <= asyncCommitCfg.KeysLimit && - !c.shouldWriteBinlog() { - totalKeySize := uint64(0) - for i := 0; i < c.mutations.Len(); i++ { - totalKeySize += uint64(len(c.mutations.GetKey(i))) - if totalKeySize > asyncCommitCfg.TotalKeySizeLimit { - return false - } - } - return true - } - return false -} - -// checkOnePC checks if 1PC protocol is available for current transaction. -func (c *twoPhaseCommitter) checkOnePC() bool { - // Disable 1PC in local transactions - if c.txn.GetScope() != oracle.GlobalTxnScope { - return false - } - - return c.sessionID > 0 && !c.shouldWriteBinlog() && c.txn.enable1PC -} - -func (c *twoPhaseCommitter) needLinearizability() bool { - return !c.txn.causalConsistency -} - -func (c *twoPhaseCommitter) isAsyncCommit() bool { - return atomic.LoadUint32(&c.useAsyncCommit) > 0 -} - -func (c *twoPhaseCommitter) setAsyncCommit(val bool) { - if val { - atomic.StoreUint32(&c.useAsyncCommit, 1) - } else { - atomic.StoreUint32(&c.useAsyncCommit, 0) - } -} - -func (c *twoPhaseCommitter) isOnePC() bool { - return atomic.LoadUint32(&c.useOnePC) > 0 -} - -func (c *twoPhaseCommitter) setOnePC(val bool) { - if val { - atomic.StoreUint32(&c.useOnePC, 1) - } else { - atomic.StoreUint32(&c.useOnePC, 0) - } -} - -func (c *twoPhaseCommitter) checkOnePCFallBack(action twoPhaseCommitAction, batchCount int) { - if _, ok := action.(actionPrewrite); ok { - if batchCount > 1 { - c.setOnePC(false) - } - } -} - -const ( - cleanupMaxBackoff = 20000 - tsoMaxBackoff = 15000 -) - -// VeryLongMaxBackoff is the max sleep time of transaction commit. -var VeryLongMaxBackoff = uint64(600000) // 10mins - -func (c *twoPhaseCommitter) cleanup(ctx context.Context) { - c.cleanWg.Add(1) - go func() { - if _, err := util.EvalFailpoint("commitFailedSkipCleanup"); err == nil { - logutil.Logger(ctx).Info("[failpoint] injected skip cleanup secondaries on failure", - zap.Uint64("txnStartTS", c.startTS)) - c.cleanWg.Done() - return - } - - cleanupKeysCtx := context.WithValue(context.Background(), retry.TxnStartKey, ctx.Value(retry.TxnStartKey)) - var err error - if !c.isOnePC() { - err = c.cleanupMutations(retry.NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) - } else if c.isPessimistic { - err = c.pessimisticRollbackMutations(retry.NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) - } - - if err != nil { - metrics.SecondaryLockCleanupFailureCounterRollback.Inc() - logutil.Logger(ctx).Info("2PC cleanup failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS), - zap.Bool("isPessimistic", c.isPessimistic), zap.Bool("isOnePC", c.isOnePC())) - } else { - logutil.Logger(ctx).Debug("2PC clean up done", - zap.Uint64("txnStartTS", c.startTS), zap.Bool("isPessimistic", c.isPessimistic), - zap.Bool("isOnePC", c.isOnePC())) - } - c.cleanWg.Done() - }() -} - -// execute executes the two-phase commit protocol. -func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { - var binlogSkipped bool - defer func() { - if c.isOnePC() { - // The error means the 1PC transaction failed. - if err != nil { - if c.getUndeterminedErr() == nil { - c.cleanup(ctx) - } - metrics.OnePCTxnCounterError.Inc() - } else { - metrics.OnePCTxnCounterOk.Inc() - } - } else if c.isAsyncCommit() { - // The error means the async commit should not succeed. - if err != nil { - if c.getUndeterminedErr() == nil { - c.cleanup(ctx) - } - metrics.AsyncCommitTxnCounterError.Inc() - } else { - metrics.AsyncCommitTxnCounterOk.Inc() - } - } else { - // Always clean up all written keys if the txn does not commit. - c.mu.RLock() - committed := c.mu.committed - undetermined := c.mu.undeterminedErr != nil - c.mu.RUnlock() - if !committed && !undetermined { - c.cleanup(ctx) - metrics.TwoPCTxnCounterError.Inc() - } else { - metrics.TwoPCTxnCounterOk.Inc() - } - c.txn.commitTS = c.commitTS - if binlogSkipped { - c.binlog.Skip() - return - } - if !c.shouldWriteBinlog() { - return - } - if err != nil { - c.binlog.Commit(ctx, 0) - } else { - c.binlog.Commit(ctx, int64(c.commitTS)) - } - } - }() - - commitTSMayBeCalculated := false - // Check async commit is available or not. - if c.checkAsyncCommit() { - commitTSMayBeCalculated = true - c.setAsyncCommit(true) - c.hasTriedAsyncCommit = true - } - // Check if 1PC is enabled. - if c.checkOnePC() { - commitTSMayBeCalculated = true - c.setOnePC(true) - c.hasTriedOnePC = true - } - - // TODO(youjiali1995): It's better to use different maxSleep for different operations - // and distinguish permanent errors from temporary errors, for example: - // - If all PDs are down, all requests to PD will fail due to network error. - // The maxSleep should't be very long in this case. - // - If the region isn't found in PD, it's possible the reason is write-stall. - // The maxSleep can be long in this case. - bo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&VeryLongMaxBackoff)), c.txn.vars) - - // If we want to use async commit or 1PC and also want linearizability across - // all nodes, we have to make sure the commit TS of this transaction is greater - // than the snapshot TS of all existent readers. So we get a new timestamp - // from PD and plus one as our MinCommitTS. - if commitTSMayBeCalculated && c.needLinearizability() { - util.EvalFailpoint("getMinCommitTSFromTSO") - latestTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) - // If we fail to get a timestamp from PD, we just propagate the failure - // instead of falling back to the normal 2PC because a normal 2PC will - // also be likely to fail due to the same timestamp issue. - if err != nil { - return errors.Trace(err) - } - // Plus 1 to avoid producing the same commit TS with previously committed transactions - c.minCommitTS = latestTS + 1 - } - // Calculate maxCommitTS if necessary - if commitTSMayBeCalculated { - if err = c.calculateMaxCommitTS(ctx); err != nil { - return errors.Trace(err) - } - } - - if c.sessionID > 0 { - util.EvalFailpoint("beforePrewrite") - } - - c.prewriteStarted = true - var binlogChan <-chan BinlogWriteResult - if c.shouldWriteBinlog() { - binlogChan = c.binlog.Prewrite(ctx, c.primary()) - } - - start := time.Now() - err = c.prewriteMutations(bo, c.mutations) - - // Return an undetermined error only if we don't know the transaction fails. - // If it fails due to a write conflict or a already existed unique key, we - // needn't return an undetermined error even if such an error is set. - if atomic.LoadUint32(&c.prewriteFailed) == 1 { - c.setUndeterminedErr(nil) - } - if err != nil { - if undeterminedErr := c.getUndeterminedErr(); undeterminedErr != nil { - logutil.Logger(ctx).Error("2PC commit result undetermined", - zap.Error(err), - zap.NamedError("rpcErr", undeterminedErr), - zap.Uint64("txnStartTS", c.startTS)) - return errors.Trace(terror.ErrResultUndetermined) - } - } - - commitDetail := c.getDetail() - commitDetail.PrewriteTime = time.Since(start) - if bo.GetTotalSleep() > 0 { - boSleep := int64(bo.GetTotalSleep()) * int64(time.Millisecond) - commitDetail.Mu.Lock() - if boSleep > commitDetail.Mu.CommitBackoffTime { - commitDetail.Mu.CommitBackoffTime = boSleep - commitDetail.Mu.BackoffTypes = bo.GetTypes() - } - commitDetail.Mu.Unlock() - } - - if binlogChan != nil { - startWaitBinlog := time.Now() - binlogWriteResult := <-binlogChan - commitDetail.WaitPrewriteBinlogTime = time.Since(startWaitBinlog) - if binlogWriteResult != nil { - binlogSkipped = binlogWriteResult.Skipped() - binlogErr := binlogWriteResult.GetError() - if binlogErr != nil { - return binlogErr - } - } - } - if err != nil { - logutil.Logger(ctx).Debug("2PC failed on prewrite", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - return errors.Trace(err) - } - - // strip check_not_exists keys that no need to commit. - c.stripNoNeedCommitKeys() - - var commitTS uint64 - - if c.isOnePC() { - if c.onePCCommitTS == 0 { - err = errors.Errorf("session %d invalid onePCCommitTS for 1PC protocol after prewrite, startTS=%v", c.sessionID, c.startTS) - return errors.Trace(err) - } - c.commitTS = c.onePCCommitTS - c.txn.commitTS = c.commitTS - logutil.Logger(ctx).Debug("1PC protocol is used to commit this txn", - zap.Uint64("startTS", c.startTS), zap.Uint64("commitTS", c.commitTS), - zap.Uint64("session", c.sessionID)) - return nil - } - - if c.onePCCommitTS != 0 { - logutil.Logger(ctx).Fatal("non 1PC transaction committed in 1PC", - zap.Uint64("session", c.sessionID), zap.Uint64("startTS", c.startTS)) - } - - if c.isAsyncCommit() { - if c.minCommitTS == 0 { - err = errors.Errorf("session %d invalid minCommitTS for async commit protocol after prewrite, startTS=%v", c.sessionID, c.startTS) - return errors.Trace(err) - } - commitTS = c.minCommitTS - } else { - start = time.Now() - logutil.Event(ctx, "start get commit ts") - commitTS, err = c.store.getTimestampWithRetry(retry.NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) - if err != nil { - logutil.Logger(ctx).Warn("2PC get commitTS failed", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - return errors.Trace(err) - } - commitDetail.GetCommitTsTime = time.Since(start) - logutil.Event(ctx, "finish get commit ts") - logutil.SetTag(ctx, "commitTs", commitTS) - } - - if !c.isAsyncCommit() { - tryAmend := c.isPessimistic && c.sessionID > 0 && c.txn.schemaAmender != nil - if !tryAmend { - _, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, false) - if err != nil { - return errors.Trace(err) - } - } else { - relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, true) - if err != nil { - return errors.Trace(err) - } - if memAmended { - // Get new commitTS and check schema valid again. - newCommitTS, err := c.getCommitTS(ctx, commitDetail) - if err != nil { - return errors.Trace(err) - } - // If schema check failed between commitTS and newCommitTs, report schema change error. - _, _, err = c.checkSchemaValid(ctx, newCommitTS, relatedSchemaChange.LatestInfoSchema, false) - if err != nil { - logutil.Logger(ctx).Info("schema check after amend failed, it means the schema version changed again", - zap.Uint64("startTS", c.startTS), - zap.Uint64("amendTS", commitTS), - zap.Int64("amendedSchemaVersion", relatedSchemaChange.LatestInfoSchema.SchemaMetaVersion()), - zap.Uint64("newCommitTS", newCommitTS)) - return errors.Trace(err) - } - commitTS = newCommitTS - } - } - } - atomic.StoreUint64(&c.commitTS, commitTS) - - if c.store.oracle.IsExpired(c.startTS, MaxTxnTimeUse, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) { - err = errors.Errorf("session %d txn takes too much time, txnStartTS: %d, comm: %d", - c.sessionID, c.startTS, c.commitTS) - return err - } - - if c.sessionID > 0 { - if val, err := util.EvalFailpoint("beforeCommit"); err == nil { - // Pass multiple instructions in one string, delimited by commas, to trigger multiple behaviors, like - // `return("delay,fail")`. Then they will be executed sequentially at once. - if v, ok := val.(string); ok { - for _, action := range strings.Split(v, ",") { - // Async commit transactions cannot return error here, since it's already successful. - if action == "fail" && !c.isAsyncCommit() { - logutil.Logger(ctx).Info("[failpoint] injected failure before commit", zap.Uint64("txnStartTS", c.startTS)) - return errors.New("injected failure before commit") - } else if action == "delay" { - duration := time.Duration(rand.Int63n(int64(time.Second) * 5)) - logutil.Logger(ctx).Info("[failpoint] injected delay before commit", - zap.Uint64("txnStartTS", c.startTS), zap.Duration("duration", duration)) - time.Sleep(duration) - } - } - } - } - } - - if c.isAsyncCommit() { - // For async commit protocol, the commit is considered success here. - c.txn.commitTS = c.commitTS - logutil.Logger(ctx).Debug("2PC will use async commit protocol to commit this txn", - zap.Uint64("startTS", c.startTS), zap.Uint64("commitTS", c.commitTS), - zap.Uint64("sessionID", c.sessionID)) - go func() { - if _, err := util.EvalFailpoint("asyncCommitDoNothing"); err == nil { - return - } - commitBo := retry.NewBackofferWithVars(ctx, CommitSecondaryMaxBackoff, c.txn.vars) - err := c.commitMutations(commitBo, c.mutations) - if err != nil { - logutil.Logger(ctx).Warn("2PC async commit failed", zap.Uint64("sessionID", c.sessionID), - zap.Uint64("startTS", c.startTS), zap.Uint64("commitTS", c.commitTS), zap.Error(err)) - } - }() - return nil - } - return c.commitTxn(ctx, commitDetail) -} - -func (c *twoPhaseCommitter) commitTxn(ctx context.Context, commitDetail *util.CommitDetails) error { - c.txn.GetMemBuffer().DiscardValues() - start := time.Now() - - // Use the VeryLongMaxBackoff to commit the primary key. - commitBo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&VeryLongMaxBackoff)), c.txn.vars) - err := c.commitMutations(commitBo, c.mutations) - commitDetail.CommitTime = time.Since(start) - if commitBo.GetTotalSleep() > 0 { - commitDetail.Mu.Lock() - commitDetail.Mu.CommitBackoffTime += int64(commitBo.GetTotalSleep()) * int64(time.Millisecond) - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, commitBo.GetTypes()...) - commitDetail.Mu.Unlock() - } - if err != nil { - if undeterminedErr := c.getUndeterminedErr(); undeterminedErr != nil { - logutil.Logger(ctx).Error("2PC commit result undetermined", - zap.Error(err), - zap.NamedError("rpcErr", undeterminedErr), - zap.Uint64("txnStartTS", c.startTS)) - err = errors.Trace(terror.ErrResultUndetermined) - } - if !c.mu.committed { - logutil.Logger(ctx).Debug("2PC failed on commit", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - return errors.Trace(err) - } - logutil.Logger(ctx).Debug("got some exceptions, but 2PC was still successful", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - } - return nil -} - -func (c *twoPhaseCommitter) stripNoNeedCommitKeys() { - if !c.hasNoNeedCommitKeys { - return - } - m := c.mutations - var newIdx int - for oldIdx := range m.handles { - key := m.GetKey(oldIdx) - flags, err := c.txn.GetMemBuffer().GetFlags(key) - if err == nil && flags.HasPrewriteOnly() { - continue - } - m.handles[newIdx] = m.handles[oldIdx] - newIdx++ - } - c.mutations.handles = c.mutations.handles[:newIdx] -} - -// SchemaVer is the infoSchema which will return the schema version. -type SchemaVer interface { - // SchemaMetaVersion returns the meta schema version. - SchemaMetaVersion() int64 -} - -// SchemaLeaseChecker is used to validate schema version is not changed during transaction execution. -type SchemaLeaseChecker interface { - // CheckBySchemaVer checks if the schema has changed for the transaction related tables between the startSchemaVer - // and the schema version at txnTS, all the related schema changes will be returned. - CheckBySchemaVer(txnTS uint64, startSchemaVer SchemaVer) (*RelatedSchemaChange, error) -} - -// RelatedSchemaChange contains information about schema diff between two schema versions. -type RelatedSchemaChange struct { - PhyTblIDS []int64 - ActionTypes []uint64 - LatestInfoSchema SchemaVer - Amendable bool -} - -func (c *twoPhaseCommitter) amendPessimisticLock(ctx context.Context, addMutations CommitterMutations) error { - keysNeedToLock := NewPlainMutations(addMutations.Len()) - for i := 0; i < addMutations.Len(); i++ { - if addMutations.IsPessimisticLock(i) { - keysNeedToLock.Push(addMutations.GetOp(i), addMutations.GetKey(i), addMutations.GetValue(i), addMutations.IsPessimisticLock(i)) - } - } - // For unique index amend, we need to pessimistic lock the generated new index keys first. - // Set doingAmend to true to force the pessimistic lock do the exist check for these keys. - c.doingAmend = true - defer func() { c.doingAmend = false }() - if keysNeedToLock.Len() > 0 { - lCtx := &kv.LockCtx{ - Killed: c.lockCtx.Killed, - ForUpdateTS: c.forUpdateTS, - LockWaitTime: c.lockCtx.LockWaitTime, - WaitStartTime: time.Now(), - } - tryTimes := uint(0) - retryLimit := config.GetGlobalConfig().PessimisticTxn.MaxRetryCount - var err error - for tryTimes < retryLimit { - pessimisticLockBo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, c.txn.vars) - err = c.pessimisticLockMutations(pessimisticLockBo, lCtx, &keysNeedToLock) - if err != nil { - // KeysNeedToLock won't change, so don't async rollback pessimistic locks here for write conflict. - if _, ok := errors.Cause(err).(*tikverr.ErrWriteConflict); ok { - newForUpdateTSVer, err := c.store.CurrentTimestamp(oracle.GlobalTxnScope) - if err != nil { - return errors.Trace(err) - } - lCtx.ForUpdateTS = newForUpdateTSVer - c.forUpdateTS = newForUpdateTSVer - logutil.Logger(ctx).Info("amend pessimistic lock pessimistic retry lock", - zap.Uint("tryTimes", tryTimes), zap.Uint64("startTS", c.startTS), - zap.Uint64("newForUpdateTS", c.forUpdateTS)) - tryTimes++ - continue - } - logutil.Logger(ctx).Warn("amend pessimistic lock has failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) - return err - } - logutil.Logger(ctx).Info("amend pessimistic lock finished", zap.Uint64("startTS", c.startTS), - zap.Uint64("forUpdateTS", c.forUpdateTS), zap.Int("keys", keysNeedToLock.Len())) - break - } - if err != nil { - logutil.Logger(ctx).Warn("amend pessimistic lock failed after retry", - zap.Uint("tryTimes", tryTimes), zap.Uint64("startTS", c.startTS)) - return err - } - } - return nil -} - -func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema SchemaVer, change *RelatedSchemaChange) (bool, error) { - addMutations, err := c.txn.schemaAmender.AmendTxn(ctx, startInfoSchema, change, c.mutations) - if err != nil { - return false, err - } - // Add new mutations to the mutation list or prewrite them if prewrite already starts. - if addMutations != nil && addMutations.Len() > 0 { - err = c.amendPessimisticLock(ctx, addMutations) - if err != nil { - logutil.Logger(ctx).Info("amendPessimisticLock has failed", zap.Error(err)) - return false, err - } - if c.prewriteStarted { - prewriteBo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) - err = c.prewriteMutations(prewriteBo, addMutations) - if err != nil { - logutil.Logger(ctx).Warn("amend prewrite has failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) - return false, err - } - logutil.Logger(ctx).Info("amend prewrite finished", zap.Uint64("txnStartTS", c.startTS)) - return true, nil - } - memBuf := c.txn.GetMemBuffer() - for i := 0; i < addMutations.Len(); i++ { - key := addMutations.GetKey(i) - op := addMutations.GetOp(i) - var err error - if op == kvrpcpb.Op_Del { - err = memBuf.Delete(key) - } else { - err = memBuf.Set(key, addMutations.GetValue(i)) - } - if err != nil { - logutil.Logger(ctx).Warn("amend mutations has failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) - return false, err - } - handle := c.txn.GetMemBuffer().IterWithFlags(key, nil).Handle() - c.mutations.Push(op, addMutations.IsPessimisticLock(i), handle) - } - } - return false, nil -} - -func (c *twoPhaseCommitter) getCommitTS(ctx context.Context, commitDetail *util.CommitDetails) (uint64, error) { - start := time.Now() - logutil.Event(ctx, "start get commit ts") - commitTS, err := c.store.getTimestampWithRetry(retry.NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) - if err != nil { - logutil.Logger(ctx).Warn("2PC get commitTS failed", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - return 0, errors.Trace(err) - } - commitDetail.GetCommitTsTime = time.Since(start) - logutil.Event(ctx, "finish get commit ts") - logutil.SetTag(ctx, "commitTS", commitTS) - - // Check commitTS. - if commitTS <= c.startTS { - err = errors.Errorf("session %d invalid transaction tso with txnStartTS=%v while txnCommitTS=%v", - c.sessionID, c.startTS, commitTS) - logutil.BgLogger().Error("invalid transaction", zap.Error(err)) - return 0, errors.Trace(err) - } - return commitTS, nil -} - -// checkSchemaValid checks if the schema has changed, if tryAmend is set to true, committer will try to amend -// this transaction using the related schema changes. -func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64, startInfoSchema SchemaVer, - tryAmend bool) (*RelatedSchemaChange, bool, error) { - if _, err := util.EvalFailpoint("failCheckSchemaValid"); err == nil { - logutil.Logger(ctx).Info("[failpoint] injected fail schema check", - zap.Uint64("txnStartTS", c.startTS)) - err := errors.Errorf("mock check schema valid failure") - return nil, false, err - } - if c.txn.schemaLeaseChecker == nil { - if c.sessionID > 0 { - logutil.Logger(ctx).Warn("schemaLeaseChecker is not set for this transaction", - zap.Uint64("sessionID", c.sessionID), - zap.Uint64("startTS", c.startTS), - zap.Uint64("commitTS", checkTS)) - } - return nil, false, nil - } - relatedChanges, err := c.txn.schemaLeaseChecker.CheckBySchemaVer(checkTS, startInfoSchema) - if err != nil { - if tryAmend && relatedChanges != nil && relatedChanges.Amendable && c.txn.schemaAmender != nil { - memAmended, amendErr := c.tryAmendTxn(ctx, startInfoSchema, relatedChanges) - if amendErr != nil { - logutil.BgLogger().Info("txn amend has failed", zap.Uint64("sessionID", c.sessionID), - zap.Uint64("startTS", c.startTS), zap.Error(amendErr)) - return nil, false, err - } - logutil.Logger(ctx).Info("amend txn successfully", - zap.Uint64("sessionID", c.sessionID), zap.Uint64("txn startTS", c.startTS), zap.Bool("memAmended", memAmended), - zap.Uint64("checkTS", checkTS), zap.Int64("startInfoSchemaVer", startInfoSchema.SchemaMetaVersion()), - zap.Int64s("table ids", relatedChanges.PhyTblIDS), zap.Uint64s("action types", relatedChanges.ActionTypes)) - return relatedChanges, memAmended, nil - } - return nil, false, errors.Trace(err) - } - return nil, false, nil -} - -func (c *twoPhaseCommitter) calculateMaxCommitTS(ctx context.Context) error { - // Amend txn with current time first, then we can make sure we have another SafeWindow time to commit - currentTS := oracle.ComposeTS(int64(time.Since(c.txn.startTime)/time.Millisecond), 0) + c.startTS - _, _, err := c.checkSchemaValid(ctx, currentTS, c.txn.schemaVer, true) - if err != nil { - logutil.Logger(ctx).Info("Schema changed for async commit txn", - zap.Error(err), - zap.Uint64("startTS", c.startTS)) - return errors.Trace(err) - } - - safeWindow := config.GetGlobalConfig().TiKVClient.AsyncCommit.SafeWindow - maxCommitTS := oracle.ComposeTS(int64(safeWindow/time.Millisecond), 0) + currentTS - logutil.BgLogger().Debug("calculate MaxCommitTS", - zap.Time("startTime", c.txn.startTime), - zap.Duration("safeWindow", safeWindow), - zap.Uint64("startTS", c.startTS), - zap.Uint64("maxCommitTS", maxCommitTS)) - - c.maxCommitTS = maxCommitTS - return nil -} - -func (c *twoPhaseCommitter) shouldWriteBinlog() bool { - return c.binlog != nil -} - -// TiKV recommends each RPC packet should be less than ~1MB. We keep each packet's -// Key+Value size below 16KB. -const txnCommitBatchSize = 16 * 1024 - -type batchMutations struct { - region locate.RegionVerID - mutations CommitterMutations - isPrimary bool -} - -func (b *batchMutations) relocate(bo *Backoffer, c *RegionCache) (bool, error) { - begin, end := b.mutations.GetKey(0), b.mutations.GetKey(b.mutations.Len()-1) - loc, err := c.LocateKey(bo, begin) - if err != nil { - return false, errors.Trace(err) - } - if !loc.Contains(end) { - return false, nil - } - b.region = loc.Region - return true, nil -} - -type batched struct { - batches []batchMutations - primaryIdx int - primaryKey []byte -} - -func newBatched(primaryKey []byte) *batched { - return &batched{ - primaryIdx: -1, - primaryKey: primaryKey, - } -} - -// appendBatchMutationsBySize appends mutations to b. It may split the keys to make -// sure each batch's size does not exceed the limit. -func (b *batched) appendBatchMutationsBySize(region locate.RegionVerID, mutations CommitterMutations, sizeFn func(k, v []byte) int, limit int) { - if _, err := util.EvalFailpoint("twoPCRequestBatchSizeLimit"); err == nil { - limit = 1 - } - - var start, end int - for start = 0; start < mutations.Len(); start = end { - var size int - for end = start; end < mutations.Len() && size < limit; end++ { - var k, v []byte - k = mutations.GetKey(end) - v = mutations.GetValue(end) - size += sizeFn(k, v) - if b.primaryIdx < 0 && bytes.Equal(k, b.primaryKey) { - b.primaryIdx = len(b.batches) - } - } - b.batches = append(b.batches, batchMutations{ - region: region, - mutations: mutations.Slice(start, end), - }) - } -} - -func (b *batched) setPrimary() bool { - // If the batches include the primary key, put it to the first - if b.primaryIdx >= 0 { - if len(b.batches) > 0 { - b.batches[b.primaryIdx].isPrimary = true - b.batches[0], b.batches[b.primaryIdx] = b.batches[b.primaryIdx], b.batches[0] - b.primaryIdx = 0 - } - return true - } - - return false -} - -func (b *batched) allBatches() []batchMutations { - return b.batches -} - -// primaryBatch returns the batch containing the primary key. -// Precondition: `b.setPrimary() == true` -func (b *batched) primaryBatch() []batchMutations { - return b.batches[:1] -} - -func (b *batched) forgetPrimary() { - if len(b.batches) == 0 { - return - } - b.batches = b.batches[1:] -} - -// batchExecutor is txn controller providing rate control like utils -type batchExecutor struct { - rateLim int // concurrent worker numbers - rateLimiter *util.RateLimit // rate limiter for concurrency control, maybe more strategies - committer *twoPhaseCommitter // here maybe more different type committer in the future - action twoPhaseCommitAction // the work action type - backoffer *Backoffer // Backoffer - tokenWaitDuration time.Duration // get token wait time -} - -// newBatchExecutor create processor to handle concurrent batch works(prewrite/commit etc) -func newBatchExecutor(rateLimit int, committer *twoPhaseCommitter, - action twoPhaseCommitAction, backoffer *Backoffer) *batchExecutor { - return &batchExecutor{rateLimit, nil, committer, - action, backoffer, 0} -} - -// initUtils do initialize batchExecutor related policies like rateLimit util -func (batchExe *batchExecutor) initUtils() error { - // init rateLimiter by injected rate limit number - batchExe.rateLimiter = util.NewRateLimit(batchExe.rateLim) - return nil -} - -// startWork concurrently do the work for each batch considering rate limit -func (batchExe *batchExecutor) startWorker(exitCh chan struct{}, ch chan error, batches []batchMutations) { - for idx, batch1 := range batches { - waitStart := time.Now() - if exit := batchExe.rateLimiter.GetToken(exitCh); !exit { - batchExe.tokenWaitDuration += time.Since(waitStart) - batch := batch1 - go func() { - defer batchExe.rateLimiter.PutToken() - var singleBatchBackoffer *Backoffer - if _, ok := batchExe.action.(actionCommit); ok { - // Because the secondary batches of the commit actions are implemented to be - // committed asynchronously in background goroutines, we should not - // fork a child context and call cancel() while the foreground goroutine exits. - // Otherwise the background goroutines will be canceled execeptionally. - // Here we makes a new clone of the original backoffer for this goroutine - // exclusively to avoid the data race when using the same backoffer - // in concurrent goroutines. - singleBatchBackoffer = batchExe.backoffer.Clone() - } else { - var singleBatchCancel context.CancelFunc - singleBatchBackoffer, singleBatchCancel = batchExe.backoffer.Fork() - defer singleBatchCancel() - } - ch <- batchExe.action.handleSingleBatch(batchExe.committer, singleBatchBackoffer, batch) - commitDetail := batchExe.committer.getDetail() - // For prewrite, we record the max backoff time - if _, ok := batchExe.action.(actionPrewrite); ok { - commitDetail.Mu.Lock() - boSleep := int64(singleBatchBackoffer.GetTotalSleep()) * int64(time.Millisecond) - if boSleep > commitDetail.Mu.CommitBackoffTime { - commitDetail.Mu.CommitBackoffTime = boSleep - commitDetail.Mu.BackoffTypes = singleBatchBackoffer.GetTypes() - } - commitDetail.Mu.Unlock() - } - // Backoff time in the 2nd phase of a non-async-commit txn is added - // in the commitTxn method, so we don't add it here. - }() - } else { - logutil.Logger(batchExe.backoffer.GetCtx()).Info("break startWorker", - zap.Stringer("action", batchExe.action), zap.Int("batch size", len(batches)), - zap.Int("index", idx)) - break - } - } -} - -// process will start worker routine and collect results -func (batchExe *batchExecutor) process(batches []batchMutations) error { - var err error - err = batchExe.initUtils() - if err != nil { - logutil.Logger(batchExe.backoffer.GetCtx()).Error("batchExecutor initUtils failed", zap.Error(err)) - return err - } - - // For prewrite, stop sending other requests after receiving first error. - var cancel context.CancelFunc - if _, ok := batchExe.action.(actionPrewrite); ok { - batchExe.backoffer, cancel = batchExe.backoffer.Fork() - defer cancel() - } - // concurrently do the work for each batch. - ch := make(chan error, len(batches)) - exitCh := make(chan struct{}) - go batchExe.startWorker(exitCh, ch, batches) - // check results - for i := 0; i < len(batches); i++ { - if e := <-ch; e != nil { - logutil.Logger(batchExe.backoffer.GetCtx()).Debug("2PC doActionOnBatch failed", - zap.Uint64("session", batchExe.committer.sessionID), - zap.Stringer("action type", batchExe.action), - zap.Error(e), - zap.Uint64("txnStartTS", batchExe.committer.startTS)) - // Cancel other requests and return the first error. - if cancel != nil { - logutil.Logger(batchExe.backoffer.GetCtx()).Debug("2PC doActionOnBatch to cancel other actions", - zap.Uint64("session", batchExe.committer.sessionID), - zap.Stringer("action type", batchExe.action), - zap.Uint64("txnStartTS", batchExe.committer.startTS)) - atomic.StoreUint32(&batchExe.committer.prewriteCancelled, 1) - cancel() - } - if err == nil { - err = e - } - } - } - close(exitCh) - if batchExe.tokenWaitDuration > 0 { - metrics.TiKVTokenWaitDuration.Observe(float64(batchExe.tokenWaitDuration.Nanoseconds())) - } - return err -} - -func (c *twoPhaseCommitter) setDetail(d *util.CommitDetails) { - atomic.StorePointer(&c.detail, unsafe.Pointer(d)) -} - -func (c *twoPhaseCommitter) getDetail() *util.CommitDetails { - return (*util.CommitDetails)(atomic.LoadPointer(&c.detail)) -} - -func (c *twoPhaseCommitter) setUndeterminedErr(err error) { - c.mu.Lock() - defer c.mu.Unlock() - c.mu.undeterminedErr = err -} - -func (c *twoPhaseCommitter) getUndeterminedErr() error { - c.mu.RLock() - defer c.mu.RUnlock() - return c.mu.undeterminedErr -} - -func (c *twoPhaseCommitter) mutationsOfKeys(keys [][]byte) CommitterMutations { - var res PlainMutations - for i := 0; i < c.mutations.Len(); i++ { - for _, key := range keys { - if bytes.Equal(c.mutations.GetKey(i), key) { - res.Push(c.mutations.GetOp(i), c.mutations.GetKey(i), c.mutations.GetValue(i), c.mutations.IsPessimisticLock(i)) - break - } - } - } - return &res -} diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go deleted file mode 100644 index 34285a8a8d204..0000000000000 --- a/store/tikv/backoff.go +++ /dev/null @@ -1,86 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "context" - - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/retry" -) - -// Backoffer is a utility for retrying queries. -type Backoffer = retry.Backoffer - -// BackoffConfig defines the backoff configuration. -type BackoffConfig = retry.Config - -// Maximum total sleep time(in ms) for kv/cop commands. -const ( - gcResolveLockMaxBackoff = 100000 - // CommitSecondaryMaxBackoff is max sleep time of the 'commit' command - CommitSecondaryMaxBackoff = 41000 -) - -var ( - // CommitMaxBackoff is max sleep time of the 'commit' command - CommitMaxBackoff = uint64(41000) - // PrewriteMaxBackoff is max sleep time of the `pre-write` command. - PrewriteMaxBackoff = 20000 -) - -// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. -func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { - return retry.NewBackofferWithVars(ctx, maxSleep, vars) -} - -// NewBackoffer creates a Backoffer with maximum sleep time(in ms). -func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { - return retry.NewBackoffer(ctx, maxSleep) -} - -// TxnStartKey is a key for transaction start_ts info in context.Context. -func TxnStartKey() interface{} { - return retry.TxnStartKey -} - -// BoRegionMiss returns the default backoff config for RegionMiss. -func BoRegionMiss() *BackoffConfig { - return retry.BoRegionMiss -} - -// BoTiFlashRPC returns the default backoff config for TiFlashRPC. -func BoTiFlashRPC() *BackoffConfig { - return retry.BoTiFlashRPC -} - -// BoTxnLock returns the default backoff config for TxnLock. -func BoTxnLock() *BackoffConfig { - return retry.BoTxnLock -} - -// BoPDRPC returns the default backoff config for PDRPC. -func BoPDRPC() *BackoffConfig { - return retry.BoPDRPC -} - -// BoTiKVRPC returns the default backoff config for TiKVRPC. -func BoTiKVRPC() *BackoffConfig { - return retry.BoTiKVRPC -} - -// NewGcResolveLockMaxBackoffer creates a Backoffer for Gc to resolve lock. -func NewGcResolveLockMaxBackoffer(ctx context.Context) *Backoffer { - return retry.NewBackofferWithVars(ctx, gcResolveLockMaxBackoff, nil) -} diff --git a/store/tikv/binlog.go b/store/tikv/binlog.go deleted file mode 100644 index 7f5616999e5d3..0000000000000 --- a/store/tikv/binlog.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "context" -) - -// BinlogExecutor defines the logic to replicate binlogs during transaction commit. -type BinlogExecutor interface { - Prewrite(ctx context.Context, primary []byte) <-chan BinlogWriteResult - Commit(ctx context.Context, commitTS int64) - Skip() -} - -// BinlogWriteResult defines the result of prewrite binlog. -type BinlogWriteResult interface { - Skipped() bool - GetError() error -} diff --git a/store/tikv/cleanup.go b/store/tikv/cleanup.go deleted file mode 100644 index 08f65e4434236..0000000000000 --- a/store/tikv/cleanup.go +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -type actionCleanup struct{} - -var _ twoPhaseCommitAction = actionCleanup{} - -func (actionCleanup) String() string { - return "cleanup" -} - -func (actionCleanup) tiKVTxnRegionsNumHistogram() prometheus.Observer { - return metrics.TxnRegionsNumHistogramCleanup -} - -func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { - req := tikvrpc.NewRequest(tikvrpc.CmdBatchRollback, &kvrpcpb.BatchRollbackRequest{ - Keys: batch.mutations.GetKeys(), - StartVersion: c.startTS, - }, kvrpcpb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) - resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort) - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - err = c.cleanupMutations(bo, batch.mutations) - return errors.Trace(err) - } - if keyErr := resp.Resp.(*kvrpcpb.BatchRollbackResponse).GetError(); keyErr != nil { - err = errors.Errorf("session %d 2PC cleanup failed: %s", c.sessionID, keyErr) - logutil.BgLogger().Debug("2PC failed cleanup key", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - return errors.Trace(err) - } - return nil -} - -func (c *twoPhaseCommitter) cleanupMutations(bo *Backoffer, mutations CommitterMutations) error { - return c.doActionOnMutations(bo, actionCleanup{}, mutations) -} diff --git a/store/tikv/client.go b/store/tikv/client.go deleted file mode 100644 index cb446d4a300e0..0000000000000 --- a/store/tikv/client.go +++ /dev/null @@ -1,39 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/config" -) - -// Client is a client that sends RPC. -// It should not be used after calling Close(). -type Client = client.Client - -// Timeout durations. -const ( - ReadTimeoutMedium = client.ReadTimeoutMedium - ReadTimeoutShort = client.ReadTimeoutShort -) - -// NewTestRPCClient is for some external tests. -func NewTestRPCClient(security config.Security) Client { - return client.NewTestRPCClient(security) -} - -// NewRPCClient creates a client that manages connections and rpc calls with tikv-servers. -func NewRPCClient(security config.Security, opts ...func(c *client.RPCClient)) *client.RPCClient { - return client.NewRPCClient(security, opts...) -} diff --git a/store/tikv/client/client.go b/store/tikv/client/client.go deleted file mode 100644 index b2652aeabb977..0000000000000 --- a/store/tikv/client/client.go +++ /dev/null @@ -1,515 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package client provides tcp connection to kvserver. -package client - -import ( - "context" - "fmt" - "io" - "math" - "runtime/trace" - "strconv" - "sync" - "sync/atomic" - "time" - - grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing" - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/debugpb" - "github.com/pingcap/kvproto/pkg/mpp" - "github.com/pingcap/kvproto/pkg/tikvpb" - "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" - "google.golang.org/grpc/connectivity" - "google.golang.org/grpc/credentials" - "google.golang.org/grpc/encoding/gzip" - "google.golang.org/grpc/keepalive" - "google.golang.org/grpc/metadata" -) - -// MaxRecvMsgSize set max gRPC receive message size received from server. If any message size is larger than -// current value, an error will be reported from gRPC. -var MaxRecvMsgSize = math.MaxInt64 - -// Timeout durations. -const ( - dialTimeout = 5 * time.Second - ReadTimeoutShort = 20 * time.Second // For requests that read/write several key-values. - ReadTimeoutMedium = 60 * time.Second // For requests that may need scan region. -) - -// Grpc window size -const ( - GrpcInitialWindowSize = 1 << 30 - GrpcInitialConnWindowSize = 1 << 30 -) - -// forwardMetadataKey is the key of gRPC metadata which represents a forwarded request. -const forwardMetadataKey = "tikv-forwarded-host" - -// Client is a client that sends RPC. -// It should not be used after calling Close(). -type Client interface { - // Close should release all data. - Close() error - // SendRequest sends Request. - SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) -} - -type connArray struct { - // The target host. - target string - - index uint32 - v []*grpc.ClientConn - // streamTimeout binds with a background goroutine to process coprocessor streaming timeout. - streamTimeout chan *tikvrpc.Lease - dialTimeout time.Duration - // batchConn is not null when batch is enabled. - *batchConn - done chan struct{} -} - -func newConnArray(maxSize uint, addr string, security config.Security, idleNotify *uint32, enableBatch bool, dialTimeout time.Duration) (*connArray, error) { - a := &connArray{ - index: 0, - v: make([]*grpc.ClientConn, maxSize), - streamTimeout: make(chan *tikvrpc.Lease, 1024), - done: make(chan struct{}), - dialTimeout: dialTimeout, - } - if err := a.Init(addr, security, idleNotify, enableBatch); err != nil { - return nil, err - } - return a, nil -} - -func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, enableBatch bool) error { - a.target = addr - - opt := grpc.WithInsecure() - if len(security.ClusterSSLCA) != 0 { - tlsConfig, err := security.ToTLSConfig() - if err != nil { - return errors.Trace(err) - } - opt = grpc.WithTransportCredentials(credentials.NewTLS(tlsConfig)) - } - - cfg := config.GetGlobalConfig() - var ( - unaryInterceptor grpc.UnaryClientInterceptor - streamInterceptor grpc.StreamClientInterceptor - ) - if cfg.OpenTracingEnable { - unaryInterceptor = grpc_opentracing.UnaryClientInterceptor() - streamInterceptor = grpc_opentracing.StreamClientInterceptor() - } - - allowBatch := (cfg.TiKVClient.MaxBatchSize > 0) && enableBatch - if allowBatch { - a.batchConn = newBatchConn(uint(len(a.v)), cfg.TiKVClient.MaxBatchSize, idleNotify) - a.pendingRequests = metrics.TiKVBatchPendingRequests.WithLabelValues(a.target) - a.batchSize = metrics.TiKVBatchRequests.WithLabelValues(a.target) - } - keepAlive := cfg.TiKVClient.GrpcKeepAliveTime - keepAliveTimeout := cfg.TiKVClient.GrpcKeepAliveTimeout - for i := range a.v { - ctx, cancel := context.WithTimeout(context.Background(), a.dialTimeout) - var callOptions []grpc.CallOption - callOptions = append(callOptions, grpc.MaxCallRecvMsgSize(MaxRecvMsgSize)) - if cfg.TiKVClient.GrpcCompressionType == gzip.Name { - callOptions = append(callOptions, grpc.UseCompressor(gzip.Name)) - } - conn, err := grpc.DialContext( - ctx, - addr, - opt, - grpc.WithInitialWindowSize(GrpcInitialWindowSize), - grpc.WithInitialConnWindowSize(GrpcInitialConnWindowSize), - grpc.WithUnaryInterceptor(unaryInterceptor), - grpc.WithStreamInterceptor(streamInterceptor), - grpc.WithDefaultCallOptions(callOptions...), - grpc.WithConnectParams(grpc.ConnectParams{ - Backoff: backoff.Config{ - BaseDelay: 100 * time.Millisecond, // Default was 1s. - Multiplier: 1.6, // Default - Jitter: 0.2, // Default - MaxDelay: 3 * time.Second, // Default was 120s. - }, - MinConnectTimeout: a.dialTimeout, - }), - grpc.WithKeepaliveParams(keepalive.ClientParameters{ - Time: time.Duration(keepAlive) * time.Second, - Timeout: time.Duration(keepAliveTimeout) * time.Second, - PermitWithoutStream: true, - }), - ) - cancel() - if err != nil { - // Cleanup if the initialization fails. - a.Close() - return errors.Trace(err) - } - a.v[i] = conn - - if allowBatch { - batchClient := &batchCommandsClient{ - target: a.target, - conn: conn, - forwardedClients: make(map[string]*batchCommandsStream), - batched: sync.Map{}, - epoch: 0, - closed: 0, - tikvClientCfg: cfg.TiKVClient, - tikvLoad: &a.tikvTransportLayerLoad, - dialTimeout: a.dialTimeout, - tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false}, - } - a.batchCommandsClients = append(a.batchCommandsClients, batchClient) - } - } - go tikvrpc.CheckStreamTimeoutLoop(a.streamTimeout, a.done) - if allowBatch { - go a.batchSendLoop(cfg.TiKVClient) - } - - return nil -} - -func (a *connArray) Get() *grpc.ClientConn { - next := atomic.AddUint32(&a.index, 1) % uint32(len(a.v)) - return a.v[next] -} - -func (a *connArray) Close() { - if a.batchConn != nil { - a.batchConn.Close() - } - - for i, c := range a.v { - if c != nil { - err := c.Close() - terror.Log(errors.Trace(err)) - a.v[i] = nil - } - } - - close(a.done) -} - -// RPCClient is RPC client struct. -// TODO: Add flow control between RPC clients in TiDB ond RPC servers in TiKV. -// Since we use shared client connection to communicate to the same TiKV, it's possible -// that there are too many concurrent requests which overload the service of TiKV. -type RPCClient struct { - sync.RWMutex - - conns map[string]*connArray - security config.Security - - idleNotify uint32 - // recycleMu protect the conns from being modified during a connArray is taken out and used. - // That means recycleIdleConnArray() will wait until nobody doing sendBatchRequest() - recycleMu sync.RWMutex - // Periodically check whether there is any connection that is idle and then close and remove these connections. - // Implement background cleanup. - isClosed bool - dialTimeout time.Duration -} - -// NewRPCClient creates a client that manages connections and rpc calls with tikv-servers. -func NewRPCClient(security config.Security, opts ...func(c *RPCClient)) *RPCClient { - cli := &RPCClient{ - conns: make(map[string]*connArray), - security: security, - dialTimeout: dialTimeout, - } - for _, opt := range opts { - opt(cli) - } - return cli -} - -// NewTestRPCClient is for some external tests. -func NewTestRPCClient(security config.Security) Client { - return NewRPCClient(security) -} - -func (c *RPCClient) getConnArray(addr string, enableBatch bool, opt ...func(cfg *config.TiKVClient)) (*connArray, error) { - c.RLock() - if c.isClosed { - c.RUnlock() - return nil, errors.Errorf("rpcClient is closed") - } - array, ok := c.conns[addr] - c.RUnlock() - if !ok { - var err error - array, err = c.createConnArray(addr, enableBatch, opt...) - if err != nil { - return nil, err - } - } - return array, nil -} - -func (c *RPCClient) createConnArray(addr string, enableBatch bool, opts ...func(cfg *config.TiKVClient)) (*connArray, error) { - c.Lock() - defer c.Unlock() - array, ok := c.conns[addr] - if !ok { - var err error - client := config.GetGlobalConfig().TiKVClient - for _, opt := range opts { - opt(&client) - } - array, err = newConnArray(client.GrpcConnectionCount, addr, c.security, &c.idleNotify, enableBatch, c.dialTimeout) - if err != nil { - return nil, err - } - c.conns[addr] = array - } - return array, nil -} - -func (c *RPCClient) closeConns() { - c.Lock() - if !c.isClosed { - c.isClosed = true - // close all connections - for _, array := range c.conns { - array.Close() - } - } - 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 { - span1 := span.Tracer().StartSpan(fmt.Sprintf("rpcClient.SendRequest, region ID: %d, type: %s", req.RegionId, req.Type), opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - - start := time.Now() - defer func() { - stmtExec := ctx.Value(util.ExecDetailsKey) - if stmtExec != nil { - detail := stmtExec.(*util.ExecDetails) - atomic.AddInt64(&detail.WaitKVRespDuration, int64(time.Since(start))) - } - c.updateTiKVSendReqHistogram(req, start) - }() - - if atomic.CompareAndSwapUint32(&c.idleNotify, 1, 0) { - c.recycleMu.Lock() - c.recycleIdleConnArray() - c.recycleMu.Unlock() - } - - // TiDB will not send batch commands to TiFlash, to resolve the conflict with Batch Cop Request. - enableBatch := req.StoreTp != tikvrpc.TiDB && req.StoreTp != tikvrpc.TiFlash - c.recycleMu.RLock() - defer c.recycleMu.RUnlock() - connArray, err := c.getConnArray(addr, enableBatch) - if err != nil { - return nil, errors.Trace(err) - } - - // TiDB RPC server supports batch RPC, but batch connection will send heart beat, It's not necessary since - // request to TiDB is not high frequency. - if config.GetGlobalConfig().TiKVClient.MaxBatchSize > 0 && enableBatch { - if batchReq := req.ToBatchCommandsRequest(); batchReq != nil { - defer trace.StartRegion(ctx, req.Type.String()).End() - return sendBatchRequest(ctx, addr, req.ForwardedHost, connArray.batchConn, batchReq, timeout) - } - } - - clientConn := connArray.Get() - if state := clientConn.GetState(); state == connectivity.TransientFailure { - storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10) - metrics.TiKVGRPCConnTransientFailureCounter.WithLabelValues(addr, storeID).Inc() - } - - if req.IsDebugReq() { - client := debugpb.NewDebugClient(clientConn) - ctx1, cancel := context.WithTimeout(ctx, timeout) - defer cancel() - return tikvrpc.CallDebugRPC(ctx1, client, req) - } - - client := tikvpb.NewTikvClient(clientConn) - - // Set metadata for request forwarding. Needn't forward DebugReq. - if req.ForwardedHost != "" { - ctx = metadata.AppendToOutgoingContext(ctx, forwardMetadataKey, req.ForwardedHost) - } - switch req.Type { - case tikvrpc.CmdBatchCop: - return c.getBatchCopStreamResponse(ctx, client, req, timeout, connArray) - case tikvrpc.CmdCopStream: - return c.getCopStreamResponse(ctx, client, req, timeout, connArray) - case tikvrpc.CmdMPPConn: - return c.getMPPStreamResponse(ctx, client, req, timeout, connArray) - } - // Or else it's a unary call. - ctx1, cancel := context.WithTimeout(ctx, timeout) - defer cancel() - return tikvrpc.CallRPC(ctx1, client, req) -} - -func (c *RPCClient) getCopStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { - // Coprocessor streaming request. - // Use context to support timeout for grpc streaming client. - ctx1, cancel := context.WithCancel(ctx) - // Should NOT call defer cancel() here because it will cancel further stream.Recv() - // We put it in copStream.Lease.Cancel call this cancel at copStream.Close - // TODO: add unit test for SendRequest. - resp, err := tikvrpc.CallRPC(ctx1, client, req) - if err != nil { - cancel() - return nil, errors.Trace(err) - } - - // Put the lease object to the timeout channel, so it would be checked periodically. - copStream := resp.Resp.(*tikvrpc.CopStreamResponse) - copStream.Timeout = timeout - copStream.Lease.Cancel = cancel - connArray.streamTimeout <- &copStream.Lease - - // Read the first streaming response to get CopStreamResponse. - // This can make error handling much easier, because SendReq() retry on - // region error automatically. - var first *coprocessor.Response - first, err = copStream.Recv() - if err != nil { - if errors.Cause(err) != io.EOF { - return nil, errors.Trace(err) - } - logutil.BgLogger().Debug("copstream returns nothing for the request.") - } - copStream.Response = first - return resp, nil - -} - -func (c *RPCClient) getBatchCopStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { - // Coprocessor streaming request. - // Use context to support timeout for grpc streaming client. - ctx1, cancel := context.WithCancel(ctx) - // Should NOT call defer cancel() here because it will cancel further stream.Recv() - // We put it in copStream.Lease.Cancel call this cancel at copStream.Close - // TODO: add unit test for SendRequest. - resp, err := tikvrpc.CallRPC(ctx1, client, req) - if err != nil { - cancel() - return nil, errors.Trace(err) - } - - // Put the lease object to the timeout channel, so it would be checked periodically. - copStream := resp.Resp.(*tikvrpc.BatchCopStreamResponse) - copStream.Timeout = timeout - copStream.Lease.Cancel = cancel - connArray.streamTimeout <- &copStream.Lease - - // Read the first streaming response to get CopStreamResponse. - // This can make error handling much easier, because SendReq() retry on - // region error automatically. - var first *coprocessor.BatchResponse - first, err = copStream.Recv() - if err != nil { - if errors.Cause(err) != io.EOF { - return nil, errors.Trace(err) - } - logutil.BgLogger().Debug("batch copstream returns nothing for the request.") - } - copStream.BatchResponse = first - return resp, nil -} - -func (c *RPCClient) getMPPStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { - // MPP streaming request. - // Use context to support timeout for grpc streaming client. - ctx1, cancel := context.WithCancel(ctx) - // Should NOT call defer cancel() here because it will cancel further stream.Recv() - // We put it in copStream.Lease.Cancel call this cancel at copStream.Close - // TODO: add unit test for SendRequest. - resp, err := tikvrpc.CallRPC(ctx1, client, req) - if err != nil { - cancel() - return nil, errors.Trace(err) - } - - // Put the lease object to the timeout channel, so it would be checked periodically. - copStream := resp.Resp.(*tikvrpc.MPPStreamResponse) - copStream.Timeout = timeout - copStream.Lease.Cancel = cancel - connArray.streamTimeout <- &copStream.Lease - - // Read the first streaming response to get CopStreamResponse. - // This can make error handling much easier, because SendReq() retry on - // region error automatically. - var first *mpp.MPPDataPacket - first, err = copStream.Recv() - if err != nil { - if errors.Cause(err) != io.EOF { - return nil, errors.Trace(err) - } - } - copStream.MPPDataPacket = first - return resp, nil -} - -// Close closes all connections. -func (c *RPCClient) Close() error { - // TODO: add a unit test for SendRequest After Closed - c.closeConns() - return nil -} diff --git a/store/tikv/client/client_batch.go b/store/tikv/client/client_batch.go deleted file mode 100644 index c869944085318..0000000000000 --- a/store/tikv/client/client_batch.go +++ /dev/null @@ -1,799 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package client provides tcp connection to kvserver. -package client - -import ( - "context" - "math" - "runtime/trace" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/tikvpb" - "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/connectivity" - "google.golang.org/grpc/metadata" -) - -type batchCommandsEntry struct { - ctx context.Context - req *tikvpb.BatchCommandsRequest_Request - res chan *tikvpb.BatchCommandsResponse_Response - // forwardedHost is the address of a store which will handle the request. - // It's different from the address the request sent to. - forwardedHost string - // canceled indicated the request is canceled or not. - canceled int32 - err error -} - -func (b *batchCommandsEntry) isCanceled() bool { - return atomic.LoadInt32(&b.canceled) == 1 -} - -func (b *batchCommandsEntry) error(err error) { - b.err = err - close(b.res) -} - -// batchCommandsBuilder collects a batch of `batchCommandsEntry`s to build -// `BatchCommandsRequest`s. -type batchCommandsBuilder struct { - // Each BatchCommandsRequest_Request sent to a store has a unique identity to - // distinguish its response. - idAlloc uint64 - entries []*batchCommandsEntry - requests []*tikvpb.BatchCommandsRequest_Request - requestIDs []uint64 - // In most cases, there isn't any forwardingReq. - forwardingReqs map[string]*tikvpb.BatchCommandsRequest -} - -func (b *batchCommandsBuilder) len() int { - return len(b.entries) -} - -func (b *batchCommandsBuilder) push(entry *batchCommandsEntry) { - b.entries = append(b.entries, entry) -} - -// build builds BatchCommandsRequests and calls collect() for each valid entry. -// The first return value is the request that doesn't need forwarding. -// The second is a map that maps forwarded hosts to requests. -func (b *batchCommandsBuilder) build( - collect func(id uint64, e *batchCommandsEntry), -) (*tikvpb.BatchCommandsRequest, map[string]*tikvpb.BatchCommandsRequest) { - for _, e := range b.entries { - if e.isCanceled() { - continue - } - if collect != nil { - collect(b.idAlloc, e) - } - if e.forwardedHost == "" { - b.requestIDs = append(b.requestIDs, b.idAlloc) - b.requests = append(b.requests, e.req) - } else { - batchReq, ok := b.forwardingReqs[e.forwardedHost] - if !ok { - batchReq = &tikvpb.BatchCommandsRequest{} - b.forwardingReqs[e.forwardedHost] = batchReq - } - batchReq.RequestIds = append(batchReq.RequestIds, b.idAlloc) - batchReq.Requests = append(batchReq.Requests, e.req) - } - b.idAlloc++ - } - var req *tikvpb.BatchCommandsRequest - if len(b.requests) > 0 { - req = &tikvpb.BatchCommandsRequest{ - Requests: b.requests, - RequestIds: b.requestIDs, - } - } - return req, b.forwardingReqs -} - -func (b *batchCommandsBuilder) cancel(e error) { - for _, entry := range b.entries { - entry.error(e) - } -} - -// reset resets the builder to the initial state. -// Should call it before collecting a new batch. -func (b *batchCommandsBuilder) reset() { - // NOTE: We can't simply set entries = entries[:0] here. - // The data in the cap part of the slice would reference the prewrite keys whose - // underlying memory is borrowed from memdb. The reference cause GC can't release - // the memdb, leading to serious memory leak problems in the large transaction case. - for i := 0; i < len(b.entries); i++ { - b.entries[i] = nil - } - b.entries = b.entries[:0] - for i := 0; i < len(b.requests); i++ { - b.requests[i] = nil - } - b.requests = b.requests[:0] - b.requestIDs = b.requestIDs[:0] - - for k := range b.forwardingReqs { - delete(b.forwardingReqs, k) - } -} - -func newBatchCommandsBuilder(maxBatchSize uint) *batchCommandsBuilder { - return &batchCommandsBuilder{ - idAlloc: 0, - entries: make([]*batchCommandsEntry, 0, maxBatchSize), - requests: make([]*tikvpb.BatchCommandsRequest_Request, 0, maxBatchSize), - requestIDs: make([]uint64, 0, maxBatchSize), - forwardingReqs: make(map[string]*tikvpb.BatchCommandsRequest), - } -} - -type batchConn struct { - // An atomic flag indicates whether the batch is idle or not. - // 0 for busy, others for idle. - idle uint32 - - // batchCommandsCh used for batch commands. - batchCommandsCh chan *batchCommandsEntry - batchCommandsClients []*batchCommandsClient - tikvTransportLayerLoad uint64 - closed chan struct{} - - reqBuilder *batchCommandsBuilder - - // Notify rpcClient to check the idle flag - idleNotify *uint32 - idleDetect *time.Timer - - pendingRequests prometheus.Observer - batchSize prometheus.Observer - - index uint32 -} - -func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn { - return &batchConn{ - batchCommandsCh: make(chan *batchCommandsEntry, maxBatchSize), - batchCommandsClients: make([]*batchCommandsClient, 0, connCount), - tikvTransportLayerLoad: 0, - closed: make(chan struct{}), - reqBuilder: newBatchCommandsBuilder(maxBatchSize), - idleNotify: idleNotify, - idleDetect: time.NewTimer(idleTimeout), - } -} - -func (a *batchConn) isIdle() bool { - return atomic.LoadUint32(&a.idle) != 0 -} - -// fetchAllPendingRequests fetches all pending requests from the channel. -func (a *batchConn) fetchAllPendingRequests( - maxBatchSize int, -) time.Time { - // Block on the first element. - var headEntry *batchCommandsEntry - select { - case headEntry = <-a.batchCommandsCh: - if !a.idleDetect.Stop() { - <-a.idleDetect.C - } - a.idleDetect.Reset(idleTimeout) - case <-a.idleDetect.C: - a.idleDetect.Reset(idleTimeout) - atomic.AddUint32(&a.idle, 1) - atomic.CompareAndSwapUint32(a.idleNotify, 0, 1) - // This batchConn to be recycled - return time.Now() - case <-a.closed: - return time.Now() - } - if headEntry == nil { - return time.Now() - } - ts := time.Now() - a.reqBuilder.push(headEntry) - - // This loop is for trying best to collect more requests. - for a.reqBuilder.len() < maxBatchSize { - select { - case entry := <-a.batchCommandsCh: - if entry == nil { - return ts - } - a.reqBuilder.push(entry) - default: - return ts - } - } - return ts -} - -// fetchMorePendingRequests fetches more pending requests from the channel. -func (a *batchConn) fetchMorePendingRequests( - maxBatchSize int, - batchWaitSize int, - maxWaitTime time.Duration, -) { - // Try to collect `batchWaitSize` requests, or wait `maxWaitTime`. - after := time.NewTimer(maxWaitTime) - for a.reqBuilder.len() < batchWaitSize { - select { - case entry := <-a.batchCommandsCh: - if entry == nil { - return - } - a.reqBuilder.push(entry) - case <-after.C: - return - } - } - after.Stop() - - // Do an additional non-block try. Here we test the lengh with `maxBatchSize` instead - // of `batchWaitSize` because trying best to fetch more requests is necessary so that - // we can adjust the `batchWaitSize` dynamically. - for a.reqBuilder.len() < maxBatchSize { - select { - case entry := <-a.batchCommandsCh: - if entry == nil { - return - } - a.reqBuilder.push(entry) - default: - return - } - } -} - -const idleTimeout = 3 * time.Minute - -func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { - defer func() { - if r := recover(); r != nil { - metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchSendLoop).Inc() - logutil.BgLogger().Error("batchSendLoop", - zap.Reflect("r", r), - zap.Stack("stack")) - logutil.BgLogger().Info("restart batchSendLoop") - go a.batchSendLoop(cfg) - } - }() - - bestBatchWaitSize := cfg.BatchWaitSize - for { - a.reqBuilder.reset() - - start := a.fetchAllPendingRequests(int(cfg.MaxBatchSize)) - a.pendingRequests.Observe(float64(len(a.batchCommandsCh))) - a.batchSize.Observe(float64(a.reqBuilder.len())) - - // curl -XPUT -d 'return(true)' http://0.0.0.0:10080/fail/github.com/pingcap/tidb/store/tikv/mockBlockOnBatchClient - if val, err := util.EvalFailpoint("mockBlockOnBatchClient"); err == nil { - if val.(bool) { - time.Sleep(1 * time.Hour) - } - } - - if a.reqBuilder.len() < int(cfg.MaxBatchSize) && cfg.MaxBatchWaitTime > 0 { - // If the target TiKV is overload, wait a while to collect more requests. - if atomic.LoadUint64(&a.tikvTransportLayerLoad) >= uint64(cfg.OverloadThreshold) { - metrics.TiKVBatchWaitOverLoad.Inc() - a.fetchMorePendingRequests(int(cfg.MaxBatchSize), int(bestBatchWaitSize), cfg.MaxBatchWaitTime) - } - } - length := a.reqBuilder.len() - if uint(length) == 0 { - // The batch command channel is closed. - return - } else if uint(length) < bestBatchWaitSize && bestBatchWaitSize > 1 { - // Waits too long to collect requests, reduce the target batch size. - bestBatchWaitSize-- - } else if uint(length) > bestBatchWaitSize+4 && bestBatchWaitSize < cfg.MaxBatchSize { - bestBatchWaitSize++ - } - - a.getClientAndSend() - metrics.TiKVBatchSendLatency.Observe(float64(time.Since(start))) - } -} - -func (a *batchConn) getClientAndSend() { - // Choose a connection by round-robbin. - var ( - cli *batchCommandsClient - target string - ) - for i := 0; i < len(a.batchCommandsClients); i++ { - a.index = (a.index + 1) % uint32(len(a.batchCommandsClients)) - target = a.batchCommandsClients[a.index].target - // The lock protects the batchCommandsClient from been closed while it's inuse. - if a.batchCommandsClients[a.index].tryLockForSend() { - cli = a.batchCommandsClients[a.index] - break - } - } - if cli == nil { - logutil.BgLogger().Warn("no available connections", zap.String("target", target)) - metrics.TiKVNoAvailableConnectionCounter.Inc() - - // Please ensure the error is handled in region cache correctly. - a.reqBuilder.cancel(errors.New("no available connections")) - return - } - defer cli.unlockForSend() - - req, forwardingReqs := a.reqBuilder.build(func(id uint64, e *batchCommandsEntry) { - cli.batched.Store(id, e) - if trace.IsEnabled() { - trace.Log(e.ctx, "rpc", "send") - } - }) - if req != nil { - cli.send("", req) - } - for forwardedHost, req := range forwardingReqs { - cli.send(forwardedHost, req) - } -} - -type tryLock struct { - *sync.Cond - reCreating bool -} - -func (l *tryLock) tryLockForSend() bool { - l.L.Lock() - if l.reCreating { - l.L.Unlock() - return false - } - return true -} - -func (l *tryLock) unlockForSend() { - l.L.Unlock() -} - -func (l *tryLock) lockForRecreate() { - l.L.Lock() - for l.reCreating { - l.Wait() - } - l.reCreating = true - l.L.Unlock() -} - -func (l *tryLock) unlockForRecreate() { - l.L.Lock() - l.reCreating = false - l.Broadcast() - l.L.Unlock() -} - -type batchCommandsStream struct { - tikvpb.Tikv_BatchCommandsClient - forwardedHost string -} - -func (s *batchCommandsStream) recv() (resp *tikvpb.BatchCommandsResponse, err error) { - defer func() { - if r := recover(); r != nil { - metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc() - logutil.BgLogger().Error("batchCommandsClient.recv panic", - zap.Reflect("r", r), - zap.Stack("stack")) - err = errors.SuspendStack(errors.New("batch conn recv paniced")) - } - }() - if _, err := util.EvalFailpoint("gotErrorInRecvLoop"); err == nil { - return nil, errors.New("injected error in batchRecvLoop") - } - // When `conn.Close()` is called, `client.Recv()` will return an error. - resp, err = s.Recv() - return -} - -// recreate creates a new BatchCommands stream. The conn should be ready for work. -func (s *batchCommandsStream) recreate(conn *grpc.ClientConn) error { - tikvClient := tikvpb.NewTikvClient(conn) - ctx := context.TODO() - // Set metadata for forwarding stream. - if s.forwardedHost != "" { - ctx = metadata.AppendToOutgoingContext(ctx, forwardMetadataKey, s.forwardedHost) - } - streamClient, err := tikvClient.BatchCommands(ctx) - if err != nil { - return errors.Trace(err) - } - s.Tikv_BatchCommandsClient = streamClient - return nil -} - -type batchCommandsClient struct { - // The target host. - target string - - conn *grpc.ClientConn - // client and forwardedClients are protected by tryLock. - // - // client is the stream that needn't forwarding. - client *batchCommandsStream - // TiDB uses [gRPC-metadata](https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md) to - // indicate a request needs forwarding. gRPC doesn't support setting a metadata for each request in a stream, - // so we need to create a stream for each forwarded host. - // - // forwardedClients are clients that need forwarding. It's a map that maps forwarded hosts to streams - forwardedClients map[string]*batchCommandsStream - batched sync.Map - - tikvClientCfg config.TiKVClient - tikvLoad *uint64 - dialTimeout time.Duration - - // Increased in each reconnection. - // It's used to prevent the connection from reconnecting multiple times - // due to one failure because there may be more than 1 `batchRecvLoop`s. - epoch uint64 - // closed indicates the batch client is closed explicitly or not. - closed int32 - // tryLock protects client when re-create the streaming. - tryLock -} - -func (c *batchCommandsClient) isStopped() bool { - return atomic.LoadInt32(&c.closed) != 0 -} - -func (c *batchCommandsClient) send(forwardedHost string, req *tikvpb.BatchCommandsRequest) { - err := c.initBatchClient(forwardedHost) - if err != nil { - logutil.BgLogger().Warn( - "init create streaming fail", - zap.String("target", c.target), - zap.String("forwardedHost", forwardedHost), - zap.Error(err), - ) - c.failPendingRequests(err) - return - } - - client := c.client - if forwardedHost != "" { - client = c.forwardedClients[forwardedHost] - } - if err := client.Send(req); err != nil { - logutil.BgLogger().Info( - "sending batch commands meets error", - zap.String("target", c.target), - zap.String("forwardedHost", forwardedHost), - zap.Uint64s("requestIDs", req.RequestIds), - zap.Error(err), - ) - c.failPendingRequests(err) - } -} - -// `failPendingRequests` must be called in locked contexts in order to avoid double closing channels. -func (c *batchCommandsClient) failPendingRequests(err error) { - util.EvalFailpoint("panicInFailPendingRequests") - c.batched.Range(func(key, value interface{}) bool { - id, _ := key.(uint64) - entry, _ := value.(*batchCommandsEntry) - c.batched.Delete(id) - entry.error(err) - return true - }) -} - -func (c *batchCommandsClient) waitConnReady() (err error) { - if c.conn.GetState() == connectivity.Ready { - return - } - start := time.Now() - defer func() { - metrics.TiKVBatchClientWaitEstablish.Observe(time.Since(start).Seconds()) - }() - dialCtx, cancel := context.WithTimeout(context.Background(), c.dialTimeout) - for { - s := c.conn.GetState() - if s == connectivity.Ready { - cancel() - break - } - if !c.conn.WaitForStateChange(dialCtx, s) { - cancel() - err = dialCtx.Err() - return - } - } - return -} - -func (c *batchCommandsClient) recreateStreamingClientOnce(streamClient *batchCommandsStream) error { - err := c.waitConnReady() - // Re-establish a application layer stream. TCP layer is handled by gRPC. - if err == nil { - err := streamClient.recreate(c.conn) - if err == nil { - logutil.BgLogger().Info( - "batchRecvLoop re-create streaming success", - zap.String("target", c.target), - zap.String("forwardedHost", streamClient.forwardedHost), - ) - return nil - } - } - logutil.BgLogger().Info( - "batchRecvLoop re-create streaming fail", - zap.String("target", c.target), - zap.String("forwardedHost", streamClient.forwardedHost), - zap.Error(err), - ) - return err -} - -func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransportLayerLoad *uint64, streamClient *batchCommandsStream) { - defer func() { - if r := recover(); r != nil { - metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc() - logutil.BgLogger().Error("batchRecvLoop", - zap.Reflect("r", r), - zap.Stack("stack")) - logutil.BgLogger().Info("restart batchRecvLoop") - go c.batchRecvLoop(cfg, tikvTransportLayerLoad, streamClient) - } - }() - - epoch := atomic.LoadUint64(&c.epoch) - for { - resp, err := streamClient.recv() - if err != nil { - if c.isStopped() { - return - } - logutil.BgLogger().Info( - "batchRecvLoop fails when receiving, needs to reconnect", - zap.String("target", c.target), - zap.String("forwardedHost", streamClient.forwardedHost), - zap.Error(err), - ) - - now := time.Now() - if stopped := c.recreateStreamingClient(err, streamClient, &epoch); stopped { - return - } - metrics.TiKVBatchClientUnavailable.Observe(time.Since(now).Seconds()) - continue - } - - responses := resp.GetResponses() - for i, requestID := range resp.GetRequestIds() { - value, ok := c.batched.Load(requestID) - if !ok { - // this maybe caused by batchCommandsClient#send meets ambiguous error that request has be sent to TiKV but still report a error. - // then TiKV will send response back though stream and reach here. - logutil.BgLogger().Warn("batchRecvLoop receives outdated response", zap.Uint64("requestID", requestID), zap.String("forwardedHost", streamClient.forwardedHost)) - continue - } - entry := value.(*batchCommandsEntry) - - if trace.IsEnabled() { - trace.Log(entry.ctx, "rpc", "received") - } - logutil.Eventf(entry.ctx, "receive %T response with other %d batched requests from %s", responses[i].GetCmd(), len(responses), c.target) - if atomic.LoadInt32(&entry.canceled) == 0 { - // Put the response only if the request is not canceled. - entry.res <- responses[i] - } - c.batched.Delete(requestID) - } - - transportLayerLoad := resp.GetTransportLayerLoad() - if transportLayerLoad > 0.0 && cfg.MaxBatchWaitTime > 0 { - // We need to consider TiKV load only if batch-wait strategy is enabled. - atomic.StoreUint64(tikvTransportLayerLoad, transportLayerLoad) - } - } -} - -func (c *batchCommandsClient) recreateStreamingClient(err error, streamClient *batchCommandsStream, epoch *uint64) (stopped bool) { - // Forbids the batchSendLoop using the old client and - // blocks other streams trying to recreate. - c.lockForRecreate() - defer c.unlockForRecreate() - - // Each batchCommandsStream has a batchRecvLoop. There is only one stream waiting for - // the connection ready in every epoch to prevent the connection from reconnecting - // multiple times due to one failure. - // - // Check it in the locked scope to prevent the stream which gets the token from - // reconnecting lately, i.e. - // goroutine 1 | goroutine 2 - // CAS success | - // | CAS failure - // | lockForRecreate - // | recreate error - // | unlockForRecreate - // lockForRecreate | - // waitConnReady | - // recreate | - // unlockForRecreate | - waitConnReady := atomic.CompareAndSwapUint64(&c.epoch, *epoch, *epoch+1) - if !waitConnReady { - *epoch = atomic.LoadUint64(&c.epoch) - if err := streamClient.recreate(c.conn); err != nil { - logutil.BgLogger().Info( - "batchRecvLoop re-create streaming fail", - zap.String("target", c.target), - zap.String("forwardedHost", streamClient.forwardedHost), - zap.Error(err), - ) - } - return c.isStopped() - } - *epoch++ - - c.failPendingRequests(err) // fail all pending requests. - b := retry.NewBackofferWithVars(context.Background(), math.MaxInt32, nil) - for { // try to re-create the streaming in the loop. - if c.isStopped() { - return true - } - err1 := c.recreateStreamingClientOnce(streamClient) - if err1 == nil { - break - } - - err2 := b.Backoff(retry.BoTiKVRPC, err1) - // As timeout is set to math.MaxUint32, err2 should always be nil. - // This line is added to make the 'make errcheck' pass. - terror.Log(err2) - } - return false -} - -func (c *batchCommandsClient) newBatchStream(forwardedHost string) (*batchCommandsStream, error) { - batchStream := &batchCommandsStream{forwardedHost: forwardedHost} - if err := batchStream.recreate(c.conn); err != nil { - return nil, errors.Trace(err) - } - return batchStream, nil -} - -func (c *batchCommandsClient) initBatchClient(forwardedHost string) error { - if forwardedHost == "" && c.client != nil { - return nil - } - if _, ok := c.forwardedClients[forwardedHost]; ok { - return nil - } - - if err := c.waitConnReady(); err != nil { - return err - } - - streamClient, err := c.newBatchStream(forwardedHost) - if err != nil { - return errors.Trace(err) - } - if forwardedHost == "" { - c.client = streamClient - } else { - c.forwardedClients[forwardedHost] = streamClient - } - go c.batchRecvLoop(c.tikvClientCfg, c.tikvLoad, streamClient) - return nil -} - -func (a *batchConn) Close() { - // Close all batchRecvLoop. - for _, c := range a.batchCommandsClients { - // After connections are closed, `batchRecvLoop`s will check the flag. - atomic.StoreInt32(&c.closed, 1) - } - // Don't close(batchCommandsCh) because when Close() is called, someone maybe - // calling SendRequest and writing batchCommandsCh, if we close it here the - // writing goroutine will panic. - close(a.closed) -} - -func sendBatchRequest( - ctx context.Context, - addr string, - forwardedHost string, - batchConn *batchConn, - req *tikvpb.BatchCommandsRequest_Request, - timeout time.Duration, -) (*tikvrpc.Response, error) { - entry := &batchCommandsEntry{ - ctx: ctx, - req: req, - res: make(chan *tikvpb.BatchCommandsResponse_Response, 1), - forwardedHost: forwardedHost, - canceled: 0, - err: nil, - } - timer := time.NewTimer(timeout) - defer timer.Stop() - - start := time.Now() - select { - case batchConn.batchCommandsCh <- entry: - case <-ctx.Done(): - logutil.BgLogger().Warn("send request is cancelled", - zap.String("to", addr), zap.String("cause", ctx.Err().Error())) - return nil, errors.Trace(ctx.Err()) - case <-timer.C: - return nil, errors.SuspendStack(errors.Annotate(context.DeadlineExceeded, "wait sendLoop")) - } - metrics.TiKVBatchWaitDuration.Observe(float64(time.Since(start))) - - select { - case res, ok := <-entry.res: - if !ok { - return nil, errors.Trace(entry.err) - } - return tikvrpc.FromBatchCommandsResponse(res) - case <-ctx.Done(): - atomic.StoreInt32(&entry.canceled, 1) - logutil.BgLogger().Warn("wait response is cancelled", - zap.String("to", addr), zap.String("cause", ctx.Err().Error())) - return nil, errors.Trace(ctx.Err()) - case <-timer.C: - atomic.StoreInt32(&entry.canceled, 1) - return nil, errors.SuspendStack(errors.Annotate(context.DeadlineExceeded, "wait recvLoop")) - } -} - -func (c *RPCClient) recycleIdleConnArray() { - var addrs []string - c.RLock() - for _, conn := range c.conns { - if conn.batchConn != nil && conn.isIdle() { - addrs = append(addrs, conn.target) - } - } - c.RUnlock() - - for _, addr := range addrs { - c.Lock() - conn, ok := c.conns[addr] - if ok { - delete(c.conns, addr) - logutil.BgLogger().Info("recycle idle connection", - zap.String("target", addr)) - } - c.Unlock() - if conn != nil { - conn.Close() - } - } -} diff --git a/store/tikv/client/client_collapse.go b/store/tikv/client/client_collapse.go deleted file mode 100644 index 159b3dd1ef50b..0000000000000 --- a/store/tikv/client/client_collapse.go +++ /dev/null @@ -1,100 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package client provides tcp connection to kvserver. -package client - -import ( - "context" - "strconv" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "golang.org/x/sync/singleflight" -) - -var _ Client = reqCollapse{} - -var resolveRegionSf singleflight.Group - -type reqCollapse struct { - Client -} - -// NewReqCollapse creates a reqCollapse. -func NewReqCollapse(client Client) Client { - return &reqCollapse{client} -} -func (r reqCollapse) Close() error { - if r.Client == nil { - panic("client should not be nil") - } - return r.Client.Close() -} - -func (r reqCollapse) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - if r.Client == nil { - panic("client should not be nil") - } - if canCollapse, resp, err := r.tryCollapseRequest(ctx, addr, req, timeout); canCollapse { - return resp, err - } - return r.Client.SendRequest(ctx, addr, req, timeout) -} - -func (r reqCollapse) tryCollapseRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (canCollapse bool, resp *tikvrpc.Response, err error) { - switch req.Type { - case tikvrpc.CmdResolveLock: - resolveLock := req.ResolveLock() - if len(resolveLock.Keys) > 0 { - // can not collapse resolve lock lite - return - } - if len(resolveLock.TxnInfos) > 0 { - // can not collapse batch resolve locks which is only used by GC worker. - return - } - canCollapse = true - key := strconv.FormatUint(resolveLock.Context.RegionId, 10) + "-" + strconv.FormatUint(resolveLock.StartVersion, 10) - resp, err = r.collapse(ctx, key, &resolveRegionSf, addr, req, timeout) - return - default: - // now we only support collapse resolve lock. - return - } -} - -func (r reqCollapse) collapse(ctx context.Context, key string, sf *singleflight.Group, - addr string, req *tikvrpc.Request, timeout time.Duration) (resp *tikvrpc.Response, err error) { - rsC := sf.DoChan(key, func() (interface{}, error) { - return r.Client.SendRequest(context.Background(), addr, req, ReadTimeoutShort) // use resolveLock timeout. - }) - timer := time.NewTimer(timeout) - defer timer.Stop() - select { - case <-ctx.Done(): - err = errors.Trace(ctx.Err()) - return - case <-timer.C: - err = errors.Trace(context.DeadlineExceeded) - return - case rs := <-rsC: - if rs.Err != nil { - err = errors.Trace(rs.Err) - return - } - resp = rs.Val.(*tikvrpc.Response) - return - } -} diff --git a/store/tikv/client/client_fail_test.go b/store/tikv/client/client_fail_test.go deleted file mode 100644 index f1a0a73b15acf..0000000000000 --- a/store/tikv/client/client_fail_test.go +++ /dev/null @@ -1,154 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package client - -import ( - "context" - "fmt" - "sync/atomic" - "time" - - . "github.com/pingcap/check" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/tikvpb" - "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/tikvrpc" -) - -type testClientFailSuite struct { - OneByOneSuite -} - -func (s *testClientFailSuite) SetUpSuite(_ *C) { - // This lock make testClientFailSuite runs exclusively. - s.LockGlobalTiKV() -} - -func (s testClientFailSuite) TearDownSuite(_ *C) { - s.UnLockGlobalTiKV() -} - -func (s *testClientFailSuite) TestPanicInRecvLoop(c *C) { - c.Assert(failpoint.Enable("tikvclient/panicInFailPendingRequests", `panic`), IsNil) - c.Assert(failpoint.Enable("tikvclient/gotErrorInRecvLoop", `return("0")`), IsNil) - - server, port := startMockTikvService() - c.Assert(port > 0, IsTrue) - defer server.Stop() - - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) - rpcClient := NewRPCClient(config.Security{}, func(c *RPCClient) { - c.dialTimeout = time.Second / 3 - }) - - // Start batchRecvLoop, and it should panic in `failPendingRequests`. - _, err := rpcClient.getConnArray(addr, true, func(cfg *config.TiKVClient) { cfg.GrpcConnectionCount = 1 }) - c.Assert(err, IsNil, Commentf("cannot establish local connection due to env problems(e.g. heavy load in test machine), please retry again")) - - req := tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{}) - _, err = rpcClient.SendRequest(context.Background(), addr, req, time.Second/2) - c.Assert(err, NotNil) - - c.Assert(failpoint.Disable("tikvclient/gotErrorInRecvLoop"), IsNil) - c.Assert(failpoint.Disable("tikvclient/panicInFailPendingRequests"), IsNil) - time.Sleep(time.Second * 2) - - req = tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{}) - _, err = rpcClient.SendRequest(context.Background(), addr, req, time.Second*4) - c.Assert(err, IsNil) -} - -func (s *testClientFailSuite) TestRecvErrorInMultipleRecvLoops(c *C) { - server, port := startMockTikvService() - c.Assert(port > 0, IsTrue) - defer server.Stop() - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) - - // Enable batch and limit the connection count to 1 so that - // there is only one BatchCommands stream for each host or forwarded host. - defer config.UpdateGlobal(func(conf *config.Config) { - conf.TiKVClient.MaxBatchSize = 128 - conf.TiKVClient.GrpcConnectionCount = 1 - })() - rpcClient := NewRPCClient(config.Security{}) - defer rpcClient.closeConns() - - // Create 4 BatchCommands streams. - prewriteReq := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, &kvrpcpb.PrewriteRequest{}) - forwardedHosts := []string{"", "127.0.0.1:6666", "127.0.0.1:7777", "127.0.0.1:8888"} - for _, forwardedHost := range forwardedHosts { - prewriteReq.ForwardedHost = forwardedHost - _, err := rpcClient.SendRequest(context.Background(), addr, prewriteReq, 10*time.Second) - c.Assert(err, IsNil) - } - connArray, err := rpcClient.getConnArray(addr, true) - c.Assert(connArray, NotNil) - c.Assert(err, IsNil) - batchConn := connArray.batchConn - c.Assert(batchConn, NotNil) - c.Assert(len(batchConn.batchCommandsClients), Equals, 1) - batchClient := batchConn.batchCommandsClients[0] - c.Assert(batchClient.client, NotNil) - c.Assert(batchClient.client.forwardedHost, Equals, "") - c.Assert(len(batchClient.forwardedClients), Equals, 3) - for _, forwardedHosts := range forwardedHosts[1:] { - c.Assert(batchClient.forwardedClients[forwardedHosts].forwardedHost, Equals, forwardedHosts) - } - - // Save all streams - clientSave := batchClient.client.Tikv_BatchCommandsClient - forwardedClientsSave := make(map[string]tikvpb.Tikv_BatchCommandsClient) - for host, client := range batchClient.forwardedClients { - forwardedClientsSave[host] = client.Tikv_BatchCommandsClient - } - epoch := atomic.LoadUint64(&batchClient.epoch) - - fp := "github.com/pingcap/tidb/store/tikv/client/gotErrorInRecvLoop" - // Send a request to each stream to trigger reconnection. - for _, forwardedHost := range forwardedHosts { - c.Assert(failpoint.Enable(fp, `1*return("0")`), IsNil) - prewriteReq.ForwardedHost = forwardedHost - _, err := rpcClient.SendRequest(context.Background(), addr, prewriteReq, 10*time.Second) - c.Assert(err, IsNil) - time.Sleep(100 * time.Millisecond) - c.Assert(failpoint.Disable(fp), IsNil) - } - - // Wait for finishing reconnection. - for { - batchClient.lockForRecreate() - if atomic.LoadUint64(&batchClient.epoch) != epoch { - batchClient.unlockForRecreate() - break - } - batchClient.unlockForRecreate() - time.Sleep(time.Millisecond * 100) - } - - // send request after reconnection. - for _, forwardedHost := range forwardedHosts { - prewriteReq.ForwardedHost = forwardedHost - _, err := rpcClient.SendRequest(context.Background(), addr, prewriteReq, 10*time.Second) - c.Assert(err, IsNil) - } - // Should only reconnect once. - c.Assert(atomic.LoadUint64(&batchClient.epoch), Equals, epoch+1) - // All streams are refreshed. - c.Assert(batchClient.client.Tikv_BatchCommandsClient, Not(Equals), clientSave) - c.Assert(len(batchClient.forwardedClients), Equals, len(forwardedClientsSave)) - for host, clientSave := range forwardedClientsSave { - c.Assert(batchClient.forwardedClients[host].Tikv_BatchCommandsClient, Not(Equals), clientSave) - } -} diff --git a/store/tikv/client/client_test.go b/store/tikv/client/client_test.go deleted file mode 100644 index b3645d990723a..0000000000000 --- a/store/tikv/client/client_test.go +++ /dev/null @@ -1,448 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package client - -import ( - "context" - "fmt" - "sync" - "sync/atomic" - "testing" - "time" - - . "github.com/pingcap/check" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/tikvpb" - "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/mockstore" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "google.golang.org/grpc/metadata" -) - -// OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. -type OneByOneSuite = mockstore.OneByOneSuite - -func TestT(t *testing.T) { - CustomVerboseFlag = true - TestingT(t) -} - -type testClientSuite struct { - OneByOneSuite -} - -type testClientSerialSuite struct { - OneByOneSuite -} - -var _ = Suite(&testClientSuite{}) -var _ = SerialSuites(&testClientFailSuite{}) -var _ = SerialSuites(&testClientSerialSuite{}) - -func (s *testClientSerialSuite) TestConn(c *C) { - defer config.UpdateGlobal(func(conf *config.Config) { - conf.TiKVClient.MaxBatchSize = 0 - })() - - client := NewRPCClient(config.Security{}) - - addr := "127.0.0.1:6379" - conn1, err := client.getConnArray(addr, true) - c.Assert(err, IsNil) - - conn2, err := client.getConnArray(addr, true) - c.Assert(err, IsNil) - c.Assert(conn2.Get(), Not(Equals), conn1.Get()) - - client.Close() - conn3, err := client.getConnArray(addr, true) - c.Assert(err, NotNil) - c.Assert(conn3, IsNil) -} - -func (s *testClientSuite) TestCancelTimeoutRetErr(c *C) { - req := new(tikvpb.BatchCommandsRequest_Request) - a := newBatchConn(1, 1, nil) - - ctx, cancel := context.WithCancel(context.TODO()) - cancel() - _, err := sendBatchRequest(ctx, "", "", a, req, 2*time.Second) - c.Assert(errors.Cause(err), Equals, context.Canceled) - - _, err = sendBatchRequest(context.Background(), "", "", a, req, 0) - c.Assert(errors.Cause(err), Equals, context.DeadlineExceeded) -} - -func (s *testClientSuite) TestSendWhenReconnect(c *C) { - server, port := startMockTikvService() - c.Assert(port > 0, IsTrue) - - rpcClient := NewRPCClient(config.Security{}) - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) - conn, err := rpcClient.getConnArray(addr, true) - c.Assert(err, IsNil) - - // Suppose all connections are re-establishing. - for _, client := range conn.batchConn.batchCommandsClients { - client.lockForRecreate() - } - - req := tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{}) - _, err = rpcClient.SendRequest(context.Background(), addr, req, 100*time.Second) - c.Assert(err.Error() == "no available connections", IsTrue) - conn.Close() - server.Stop() -} - -// chanClient sends received requests to the channel. -type chanClient struct { - wg *sync.WaitGroup - ch chan<- *tikvrpc.Request -} - -func (c *chanClient) Close() error { - return nil -} - -func (c *chanClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - c.wg.Wait() - c.ch <- req - return nil, nil -} - -func (s *testClientSuite) TestCollapseResolveLock(c *C) { - buildResolveLockReq := func(regionID uint64, startTS uint64, commitTS uint64, keys [][]byte) *tikvrpc.Request { - region := &metapb.Region{Id: regionID} - req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, &kvrpcpb.ResolveLockRequest{ - StartVersion: startTS, - CommitVersion: commitTS, - Keys: keys, - }) - tikvrpc.SetContext(req, region, nil) - return req - } - buildBatchResolveLockReq := func(regionID uint64, txnInfos []*kvrpcpb.TxnInfo) *tikvrpc.Request { - region := &metapb.Region{Id: regionID} - req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, &kvrpcpb.ResolveLockRequest{ - TxnInfos: txnInfos, - }) - tikvrpc.SetContext(req, region, nil) - return req - } - - var wg sync.WaitGroup - reqCh := make(chan *tikvrpc.Request) - client := reqCollapse{&chanClient{wg: &wg, ch: reqCh}} - ctx := context.Background() - - // Collapse ResolveLock. - resolveLockReq := buildResolveLockReq(1, 10, 20, nil) - wg.Add(1) - go client.SendRequest(ctx, "", resolveLockReq, time.Second) - go client.SendRequest(ctx, "", resolveLockReq, time.Second) - time.Sleep(300 * time.Millisecond) - wg.Done() - req := <-reqCh - c.Assert(*req, DeepEquals, *resolveLockReq) - select { - case <-reqCh: - c.Fatal("fail to collapse ResolveLock") - default: - } - - // Don't collapse ResolveLockLite. - resolveLockLiteReq := buildResolveLockReq(1, 10, 20, [][]byte{[]byte("foo")}) - wg.Add(1) - go client.SendRequest(ctx, "", resolveLockLiteReq, time.Second) - go client.SendRequest(ctx, "", resolveLockLiteReq, time.Second) - time.Sleep(300 * time.Millisecond) - wg.Done() - for i := 0; i < 2; i++ { - req := <-reqCh - c.Assert(*req, DeepEquals, *resolveLockLiteReq) - } - - // Don't collapse BatchResolveLock. - batchResolveLockReq := buildBatchResolveLockReq(1, []*kvrpcpb.TxnInfo{ - {Txn: 10, Status: 20}, - }) - wg.Add(1) - go client.SendRequest(ctx, "", batchResolveLockReq, time.Second) - go client.SendRequest(ctx, "", batchResolveLockReq, time.Second) - time.Sleep(300 * time.Millisecond) - wg.Done() - for i := 0; i < 2; i++ { - req := <-reqCh - c.Assert(*req, DeepEquals, *batchResolveLockReq) - } - - // Mixed - wg.Add(1) - go client.SendRequest(ctx, "", resolveLockReq, time.Second) - go client.SendRequest(ctx, "", resolveLockLiteReq, time.Second) - go client.SendRequest(ctx, "", batchResolveLockReq, time.Second) - time.Sleep(300 * time.Millisecond) - wg.Done() - for i := 0; i < 3; i++ { - <-reqCh - } - select { - case <-reqCh: - c.Fatal("unexpected request") - default: - } -} - -func (s *testClientSerialSuite) TestForwardMetadataByUnaryCall(c *C) { - server, port := startMockTikvService() - c.Assert(port > 0, IsTrue) - defer server.Stop() - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) - - // Disable batch. - defer config.UpdateGlobal(func(conf *config.Config) { - conf.TiKVClient.MaxBatchSize = 0 - conf.TiKVClient.GrpcConnectionCount = 1 - })() - rpcClient := NewRPCClient(config.Security{}) - defer rpcClient.closeConns() - - var checkCnt uint64 - // Check no corresponding metadata if ForwardedHost is empty. - server.setMetaChecker(func(ctx context.Context) error { - atomic.AddUint64(&checkCnt, 1) - // gRPC may set some metadata by default, e.g. "context-type". - md, ok := metadata.FromIncomingContext(ctx) - if ok { - vals := md.Get(forwardMetadataKey) - c.Assert(len(vals), Equals, 0) - } - return nil - }) - - // Prewrite represents unary-unary call. - prewriteReq := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, &kvrpcpb.PrewriteRequest{}) - for i := 0; i < 3; i++ { - _, err := rpcClient.SendRequest(context.Background(), addr, prewriteReq, 10*time.Second) - c.Assert(err, IsNil) - } - c.Assert(atomic.LoadUint64(&checkCnt), Equals, uint64(3)) - - // CopStream represents unary-stream call. - copStreamReq := tikvrpc.NewRequest(tikvrpc.CmdCopStream, &coprocessor.Request{}) - _, err := rpcClient.SendRequest(context.Background(), addr, copStreamReq, 10*time.Second) - c.Assert(err, IsNil) - c.Assert(atomic.LoadUint64(&checkCnt), Equals, uint64(4)) - - checkCnt = 0 - forwardedHost := "127.0.0.1:6666" - // Check the metadata exists. - server.setMetaChecker(func(ctx context.Context) error { - atomic.AddUint64(&checkCnt, 1) - // gRPC may set some metadata by default, e.g. "context-type". - md, ok := metadata.FromIncomingContext(ctx) - c.Assert(ok, IsTrue) - vals := md.Get(forwardMetadataKey) - c.Assert(vals, DeepEquals, []string{forwardedHost}) - return nil - }) - - prewriteReq.ForwardedHost = forwardedHost - for i := 0; i < 3; i++ { - _, err = rpcClient.SendRequest(context.Background(), addr, prewriteReq, 10*time.Second) - c.Assert(err, IsNil) - } - // checkCnt should be 3 because we don't use BatchCommands for redirection for now. - c.Assert(atomic.LoadUint64(&checkCnt), Equals, uint64(3)) - - copStreamReq.ForwardedHost = forwardedHost - _, err = rpcClient.SendRequest(context.Background(), addr, copStreamReq, 10*time.Second) - c.Assert(err, IsNil) - c.Assert(atomic.LoadUint64(&checkCnt), Equals, uint64(4)) -} - -func (s *testClientSerialSuite) TestForwardMetadataByBatchCommands(c *C) { - server, port := startMockTikvService() - c.Assert(port > 0, IsTrue) - defer server.Stop() - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) - - // Enable batch and limit the connection count to 1 so that - // there is only one BatchCommands stream for each host or forwarded host. - defer config.UpdateGlobal(func(conf *config.Config) { - conf.TiKVClient.MaxBatchSize = 128 - conf.TiKVClient.GrpcConnectionCount = 1 - })() - rpcClient := NewRPCClient(config.Security{}) - defer rpcClient.closeConns() - - var checkCnt uint64 - setCheckHandler := func(forwardedHost string) { - server.setMetaChecker(func(ctx context.Context) error { - atomic.AddUint64(&checkCnt, 1) - md, ok := metadata.FromIncomingContext(ctx) - if forwardedHost == "" { - if ok { - vals := md.Get(forwardMetadataKey) - c.Assert(len(vals), Equals, 0) - } - } else { - c.Assert(ok, IsTrue) - vals := md.Get(forwardMetadataKey) - c.Assert(vals, DeepEquals, []string{forwardedHost}) - - } - return nil - }) - } - - prewriteReq := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, &kvrpcpb.PrewriteRequest{}) - forwardedHosts := []string{"", "127.0.0.1:6666", "127.0.0.1:7777", "127.0.0.1:8888"} - for i, forwardedHost := range forwardedHosts { - setCheckHandler(forwardedHost) - prewriteReq.ForwardedHost = forwardedHost - for i := 0; i < 3; i++ { - _, err := rpcClient.SendRequest(context.Background(), addr, prewriteReq, 10*time.Second) - c.Assert(err, IsNil) - } - // checkCnt should be i because there is a stream for each forwardedHost. - c.Assert(atomic.LoadUint64(&checkCnt), Equals, 1+uint64(i)) - } - - checkCnt = 0 - // CopStream is a unary-stream call which doesn't support batch. - copStreamReq := tikvrpc.NewRequest(tikvrpc.CmdCopStream, &coprocessor.Request{}) - // Check no corresponding metadata if forwardedHost is empty. - setCheckHandler("") - _, err := rpcClient.SendRequest(context.Background(), addr, copStreamReq, 10*time.Second) - c.Assert(err, IsNil) - c.Assert(atomic.LoadUint64(&checkCnt), Equals, uint64(1)) - - copStreamReq.ForwardedHost = "127.0.0.1:6666" - // Check the metadata exists. - setCheckHandler(copStreamReq.ForwardedHost) - _, err = rpcClient.SendRequest(context.Background(), addr, copStreamReq, 10*time.Second) - c.Assert(err, IsNil) - c.Assert(atomic.LoadUint64(&checkCnt), Equals, uint64(2)) -} - -func (s *testClientSuite) TestBatchCommandsBuilder(c *C) { - builder := newBatchCommandsBuilder(128) - - // Test no forwarding requests. - builder.reset() - req := new(tikvpb.BatchCommandsRequest_Request) - for i := 0; i < 10; i++ { - builder.push(&batchCommandsEntry{req: req}) - c.Assert(builder.len(), Equals, i+1) - } - entryMap := make(map[uint64]*batchCommandsEntry) - batchedReq, forwardingReqs := builder.build(func(id uint64, e *batchCommandsEntry) { - entryMap[id] = e - }) - c.Assert(len(batchedReq.GetRequests()), Equals, 10) - c.Assert(len(batchedReq.GetRequestIds()), Equals, 10) - c.Assert(len(entryMap), Equals, 10) - for i, id := range batchedReq.GetRequestIds() { - c.Assert(id, Equals, uint64(i)) - c.Assert(entryMap[id].req, Equals, batchedReq.GetRequests()[i]) - } - c.Assert(len(forwardingReqs), Equals, 0) - c.Assert(builder.idAlloc, Equals, uint64(10)) - - // Test collecting forwarding requests. - builder.reset() - forwardedHosts := []string{"", "127.0.0.1:6666", "127.0.0.1:7777", "127.0.0.1:8888"} - for i := range forwardedHosts { - for j, host := range forwardedHosts { - // Each forwarded host has incremental count of requests - // and interleaves with each other. - if i <= j { - builder.push(&batchCommandsEntry{req: req, forwardedHost: host}) - } - } - } - entryMap = make(map[uint64]*batchCommandsEntry) - batchedReq, forwardingReqs = builder.build(func(id uint64, e *batchCommandsEntry) { - entryMap[id] = e - }) - c.Assert(len(batchedReq.GetRequests()), Equals, 1) - c.Assert(len(batchedReq.GetRequestIds()), Equals, 1) - c.Assert(len(forwardingReqs), Equals, 3) - for i, host := range forwardedHosts[1:] { - c.Assert(len(forwardingReqs[host].GetRequests()), Equals, i+2) - c.Assert(len(forwardingReqs[host].GetRequestIds()), Equals, i+2) - } - c.Assert(builder.idAlloc, Equals, uint64(10+builder.len())) - c.Assert(len(entryMap), Equals, builder.len()) - for host, forwardingReq := range forwardingReqs { - for i, id := range forwardingReq.GetRequestIds() { - c.Assert(entryMap[id].req, Equals, forwardingReq.GetRequests()[i]) - c.Assert(entryMap[id].forwardedHost, Equals, host) - } - } - - // Test not collecting canceled requests - builder.reset() - entries := []*batchCommandsEntry{ - {canceled: 1, req: req}, - {canceled: 0, req: req}, - {canceled: 1, req: req}, - {canceled: 1, req: req}, - {canceled: 0, req: req}, - } - for _, entry := range entries { - builder.push(entry) - } - entryMap = make(map[uint64]*batchCommandsEntry) - batchedReq, forwardingReqs = builder.build(func(id uint64, e *batchCommandsEntry) { - entryMap[id] = e - }) - c.Assert(len(batchedReq.GetRequests()), Equals, 2) - c.Assert(len(batchedReq.GetRequestIds()), Equals, 2) - c.Assert(len(forwardingReqs), Equals, 0) - c.Assert(len(entryMap), Equals, 2) - for i, id := range batchedReq.GetRequestIds() { - c.Assert(entryMap[id].req, Equals, batchedReq.GetRequests()[i]) - c.Assert(entryMap[id].isCanceled(), IsFalse) - } - - // Test canceling all requests - builder.reset() - entries = entries[:0] - for i := 0; i < 3; i++ { - entry := &batchCommandsEntry{req: req, res: make(chan *tikvpb.BatchCommandsResponse_Response, 1)} - entries = append(entries, entry) - builder.push(entry) - } - err := errors.New("error") - builder.cancel(err) - for _, entry := range entries { - _, ok := <-entry.res - c.Assert(ok, IsFalse) - c.Assert(entry.err, Equals, err) - } - - // Test reset - builder.reset() - c.Assert(builder.len(), Equals, 0) - c.Assert(len(builder.entries), Equals, 0) - c.Assert(len(builder.requests), Equals, 0) - c.Assert(len(builder.requestIDs), Equals, 0) - c.Assert(len(builder.forwardingReqs), Equals, 0) - c.Assert(builder.idAlloc, Not(Equals), 0) -} diff --git a/store/tikv/client/mock_tikv_service_test.go b/store/tikv/client/mock_tikv_service_test.go deleted file mode 100644 index b2b5682457818..0000000000000 --- a/store/tikv/client/mock_tikv_service_test.go +++ /dev/null @@ -1,117 +0,0 @@ -package client - -import ( - "context" - "fmt" - "net" - "sync" - "time" - - "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/tikvpb" - "github.com/pingcap/tidb/store/tikv/logutil" - "go.uber.org/zap" - "google.golang.org/grpc" -) - -type server struct { - tikvpb.TikvServer - grpcServer *grpc.Server - // metaChecker check the metadata of each request. Now only requests - // which need redirection set it. - metaChecker struct { - sync.Mutex - check func(context.Context) error - } -} - -func (s *server) KvPrewrite(ctx context.Context, req *kvrpcpb.PrewriteRequest) (*kvrpcpb.PrewriteResponse, error) { - if err := s.checkMetadata(ctx); err != nil { - return nil, err - } - return &kvrpcpb.PrewriteResponse{}, nil -} - -func (s *server) CoprocessorStream(req *coprocessor.Request, ss tikvpb.Tikv_CoprocessorStreamServer) error { - if err := s.checkMetadata(ss.Context()); err != nil { - return err - } - return ss.Send(&coprocessor.Response{}) -} - -func (s *server) BatchCommands(ss tikvpb.Tikv_BatchCommandsServer) error { - if err := s.checkMetadata(ss.Context()); err != nil { - return err - } - for { - req, err := ss.Recv() - if err != nil { - logutil.BgLogger().Error("batch commands receive fail", zap.Error(err)) - return err - } - - responses := make([]*tikvpb.BatchCommandsResponse_Response, 0, len(req.GetRequestIds())) - for i := 0; i < len(req.GetRequestIds()); i++ { - responses = append(responses, &tikvpb.BatchCommandsResponse_Response{ - Cmd: &tikvpb.BatchCommandsResponse_Response_Empty{ - Empty: &tikvpb.BatchCommandsEmptyResponse{}, - }, - }) - } - - err = ss.Send(&tikvpb.BatchCommandsResponse{ - Responses: responses, - RequestIds: req.GetRequestIds(), - }) - if err != nil { - logutil.BgLogger().Error("batch commands send fail", zap.Error(err)) - return err - } - } -} - -func (s *server) setMetaChecker(check func(context.Context) error) { - s.metaChecker.Lock() - s.metaChecker.check = check - s.metaChecker.Unlock() -} - -func (s *server) checkMetadata(ctx context.Context) error { - s.metaChecker.Lock() - defer s.metaChecker.Unlock() - if s.metaChecker.check != nil { - return s.metaChecker.check(ctx) - } - return nil -} - -func (s *server) Stop() { - s.grpcServer.Stop() -} - -// Try to start a gRPC server and retrun the server instance and binded port. -func startMockTikvService() (*server, int) { - port := -1 - lis, err := net.Listen("tcp", fmt.Sprintf("%s:%d", "127.0.0.1", 0)) - if err != nil { - logutil.BgLogger().Error("can't listen", zap.Error(err)) - logutil.BgLogger().Error("can't start mock tikv service because no available ports") - return nil, port - } - port = lis.Addr().(*net.TCPAddr).Port - - server := &server{} - s := grpc.NewServer(grpc.ConnectionTimeout(time.Minute)) - tikvpb.RegisterTikvServer(s, server) - server.grpcServer = s - go func() { - if err = s.Serve(lis); err != nil { - logutil.BgLogger().Error( - "can't serve gRPC requests", - zap.Error(err), - ) - } - }() - return server, port -} diff --git a/store/tikv/client_helper.go b/store/tikv/client_helper.go deleted file mode 100644 index 5e1517363332a..0000000000000 --- a/store/tikv/client_helper.go +++ /dev/null @@ -1,92 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "time" - - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" -) - -// ClientHelper wraps LockResolver and RegionRequestSender. -// It's introduced to support the new lock resolving pattern in the large transaction. -// In the large transaction protocol, sending requests and resolving locks are -// context-dependent. For example, when a send request meets a secondary lock, we'll -// call ResolveLock, and if the lock belongs to a large transaction, we may retry -// the request. If there is no context information about the resolved locks, we'll -// meet the secondary lock again and run into a deadloop. -type ClientHelper struct { - lockResolver *LockResolver - regionCache *locate.RegionCache - resolvedLocks *util.TSSet - client Client - resolveLite bool - locate.RegionRequestRuntimeStats -} - -// NewClientHelper creates a helper instance. -func NewClientHelper(store *KVStore, resolvedLocks *util.TSSet) *ClientHelper { - return &ClientHelper{ - lockResolver: store.GetLockResolver(), - regionCache: store.GetRegionCache(), - resolvedLocks: resolvedLocks, - client: store.GetTiKVClient(), - } -} - -// ResolveLocks wraps the ResolveLocks function and store the resolved result. -func (ch *ClientHelper) ResolveLocks(bo *Backoffer, callerStartTS uint64, locks []*Lock) (int64, error) { - var err error - var resolvedLocks []uint64 - var msBeforeTxnExpired int64 - if ch.Stats != nil { - defer func(start time.Time) { - locate.RecordRegionRequestRuntimeStats(ch.Stats, tikvrpc.CmdResolveLock, time.Since(start)) - }(time.Now()) - } - if ch.resolveLite { - msBeforeTxnExpired, resolvedLocks, err = ch.lockResolver.ResolveLocksLite(bo, callerStartTS, locks) - } else { - msBeforeTxnExpired, resolvedLocks, err = ch.lockResolver.ResolveLocks(bo, callerStartTS, locks) - } - if err != nil { - return msBeforeTxnExpired, err - } - if len(resolvedLocks) > 0 { - ch.resolvedLocks.Put(resolvedLocks...) - return 0, nil - } - return msBeforeTxnExpired, nil -} - -// SendReqCtx wraps the SendReqCtx function and use the resolved lock result in the kvrpcpb.Context. -func (ch *ClientHelper) SendReqCtx(bo *Backoffer, req *tikvrpc.Request, regionID locate.RegionVerID, timeout time.Duration, et tikvrpc.EndpointType, directStoreAddr string, opts ...locate.StoreSelectorOption) (*tikvrpc.Response, *locate.RPCContext, string, error) { - sender := locate.NewRegionRequestSender(ch.regionCache, ch.client) - if len(directStoreAddr) > 0 { - sender.SetStoreAddr(directStoreAddr) - } - sender.Stats = ch.Stats - req.Context.ResolvedLocks = ch.resolvedLocks.GetAll() - if val, err := util.EvalFailpoint("assertStaleReadFlag"); err == nil { - if val.(bool) { - if len(opts) > 0 && !req.StaleRead { - panic("req.StaleRead shouldn't be false when opts is not empty") - } - } - } - resp, ctx, err := sender.SendReqCtx(bo, req, regionID, timeout, et, opts...) - return resp, ctx, sender.GetStoreAddr(), err -} diff --git a/store/tikv/commit.go b/store/tikv/commit.go deleted file mode 100644 index 9b503c3aa0eb5..0000000000000 --- a/store/tikv/commit.go +++ /dev/null @@ -1,190 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "encoding/hex" - "time" - - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/client" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -type actionCommit struct{ retry bool } - -var _ twoPhaseCommitAction = actionCommit{} - -func (actionCommit) String() string { - return "commit" -} - -func (actionCommit) tiKVTxnRegionsNumHistogram() prometheus.Observer { - return metrics.TxnRegionsNumHistogramCommit -} - -func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { - keys := batch.mutations.GetKeys() - req := tikvrpc.NewRequest(tikvrpc.CmdCommit, &kvrpcpb.CommitRequest{ - StartVersion: c.startTS, - Keys: keys, - CommitVersion: c.commitTS, - }, kvrpcpb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) - - tBegin := time.Now() - attempts := 0 - - sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) - for { - attempts++ - if time.Since(tBegin) > slowRequestThreshold { - logutil.BgLogger().Warn("slow commit request", zap.Uint64("startTS", c.startTS), zap.Stringer("region", &batch.region), zap.Int("attempts", attempts)) - tBegin = time.Now() - } - - resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) - // If we fail to receive response for the request that commits primary key, it will be undetermined whether this - // transaction has been successfully committed. - // Under this circumstance, we can not declare the commit is complete (may lead to data lost), nor can we throw - // an error (may lead to the duplicated key error when upper level restarts the transaction). Currently the best - // solution is to populate this error and let upper layer drop the connection to the corresponding mysql client. - if batch.isPrimary && sender.GetRPCError() != nil && !c.isAsyncCommit() { - c.setUndeterminedErr(errors.Trace(sender.GetRPCError())) - } - - // Unexpected error occurs, return it. - if err != nil { - return errors.Trace(err) - } - - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - // For other region error and the fake region error, backoff because - // there's something wrong. - // For the real EpochNotMatch error, don't backoff. - if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - } - same, err := batch.relocate(bo, c.store.regionCache) - if err != nil { - return errors.Trace(err) - } - if same { - continue - } - err = c.doActionOnMutations(bo, actionCommit{true}, batch.mutations) - return errors.Trace(err) - } - - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - commitResp := resp.Resp.(*kvrpcpb.CommitResponse) - // Here we can make sure tikv has processed the commit primary key request. So - // we can clean undetermined error. - if batch.isPrimary && !c.isAsyncCommit() { - c.setUndeterminedErr(nil) - } - if keyErr := commitResp.GetError(); keyErr != nil { - if rejected := keyErr.GetCommitTsExpired(); rejected != nil { - logutil.Logger(bo.GetCtx()).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS", - zap.Uint64("txnStartTS", c.startTS), - zap.Stringer("info", logutil.Hex(rejected))) - - // Do not retry for a txn which has a too large MinCommitTs - // 3600000 << 18 = 943718400000 - if rejected.MinCommitTs-rejected.AttemptedCommitTs > 943718400000 { - err := errors.Errorf("2PC MinCommitTS is too large, we got MinCommitTS: %d, and AttemptedCommitTS: %d", - rejected.MinCommitTs, rejected.AttemptedCommitTs) - return errors.Trace(err) - } - - // Update commit ts and retry. - commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) - if err != nil { - logutil.Logger(bo.GetCtx()).Warn("2PC get commitTS failed", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - return errors.Trace(err) - } - - c.mu.Lock() - c.commitTS = commitTS - c.mu.Unlock() - // Update the commitTS of the request and retry. - req.Commit().CommitVersion = commitTS - continue - } - - c.mu.RLock() - defer c.mu.RUnlock() - err = extractKeyErr(keyErr) - if c.mu.committed { - // No secondary key could be rolled back after it's primary key is committed. - // There must be a serious bug somewhere. - hexBatchKeys := func(keys [][]byte) []string { - var res []string - for _, k := range keys { - res = append(res, hex.EncodeToString(k)) - } - return res - } - logutil.Logger(bo.GetCtx()).Error("2PC failed commit key after primary key committed", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS), - zap.Uint64("commitTS", c.commitTS), - zap.Strings("keys", hexBatchKeys(keys))) - return errors.Trace(err) - } - // The transaction maybe rolled back by concurrent transactions. - logutil.Logger(bo.GetCtx()).Debug("2PC failed commit primary key", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - return err - } - break - } - - c.mu.Lock() - defer c.mu.Unlock() - // Group that contains primary key is always the first. - // We mark transaction's status committed when we receive the first success response. - c.mu.committed = true - return nil -} - -func (c *twoPhaseCommitter) commitMutations(bo *Backoffer, mutations CommitterMutations) error { - if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("twoPhaseCommitter.commitMutations", opentracing.ChildOf(span.Context())) - defer span1.Finish() - bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) - } - - return c.doActionOnMutations(bo, actionCommit{}, mutations) -} diff --git a/store/tikv/config/client.go b/store/tikv/config/client.go deleted file mode 100644 index 8e9169301c2f8..0000000000000 --- a/store/tikv/config/client.go +++ /dev/null @@ -1,143 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package config - -import ( - "fmt" - "time" - - "google.golang.org/grpc/encoding/gzip" -) - -const ( - // DefStoreLivenessTimeout is the default value for store liveness timeout. - DefStoreLivenessTimeout = "1s" -) - -// TiKVClient is the config for tikv client. -type TiKVClient struct { - // GrpcConnectionCount is the max gRPC connections that will be established - // with each tikv-server. - GrpcConnectionCount uint `toml:"grpc-connection-count" json:"grpc-connection-count"` - // After a duration of this time in seconds if the client doesn't see any activity it pings - // the server to see if the transport is still alive. - GrpcKeepAliveTime uint `toml:"grpc-keepalive-time" json:"grpc-keepalive-time"` - // After having pinged for keepalive check, the client waits for a duration of Timeout in seconds - // and if no activity is seen even after that the connection is closed. - GrpcKeepAliveTimeout uint `toml:"grpc-keepalive-timeout" json:"grpc-keepalive-timeout"` - // GrpcCompressionType is the compression type for gRPC channel: none or gzip. - GrpcCompressionType string `toml:"grpc-compression-type" json:"grpc-compression-type"` - // CommitTimeout is the max time which command 'commit' will wait. - CommitTimeout string `toml:"commit-timeout" json:"commit-timeout"` - AsyncCommit AsyncCommit `toml:"async-commit" json:"async-commit"` - // MaxBatchSize is the max batch size when calling batch commands API. - MaxBatchSize uint `toml:"max-batch-size" json:"max-batch-size"` - // If TiKV load is greater than this, TiDB will wait for a while to avoid little batch. - OverloadThreshold uint `toml:"overload-threshold" json:"overload-threshold"` - // MaxBatchWaitTime in nanosecond is the max wait time for batch. - MaxBatchWaitTime time.Duration `toml:"max-batch-wait-time" json:"max-batch-wait-time"` - // BatchWaitSize is the max wait size for batch. - BatchWaitSize uint `toml:"batch-wait-size" json:"batch-wait-size"` - // EnableChunkRPC indicate the data encode in chunk format for coprocessor requests. - EnableChunkRPC bool `toml:"enable-chunk-rpc" json:"enable-chunk-rpc"` - // If a Region has not been accessed for more than the given duration (in seconds), it - // will be reloaded from the PD. - RegionCacheTTL uint `toml:"region-cache-ttl" json:"region-cache-ttl"` - // If a store has been up to the limit, it will return error for successive request to - // prevent the store occupying too much token in dispatching level. - StoreLimit int64 `toml:"store-limit" json:"store-limit"` - // StoreLivenessTimeout is the timeout for store liveness check request. - StoreLivenessTimeout string `toml:"store-liveness-timeout" json:"store-liveness-timeout"` - CoprCache CoprocessorCache `toml:"copr-cache" json:"copr-cache"` - // TTLRefreshedTxnSize controls whether a transaction should update its TTL or not. - TTLRefreshedTxnSize int64 `toml:"ttl-refreshed-txn-size" json:"ttl-refreshed-txn-size"` -} - -// AsyncCommit is the config for the async commit feature. The switch to enable it is a system variable. -type AsyncCommit struct { - // Use async commit only if the number of keys does not exceed KeysLimit. - KeysLimit uint `toml:"keys-limit" json:"keys-limit"` - // Use async commit only if the total size of keys does not exceed TotalKeySizeLimit. - TotalKeySizeLimit uint64 `toml:"total-key-size-limit" json:"total-key-size-limit"` - // The duration within which is safe for async commit or 1PC to commit with an old schema. - // The following two fields should NOT be modified in most cases. If both async commit - // and 1PC are disabled in the whole cluster, they can be set to zero to avoid waiting in DDLs. - SafeWindow time.Duration `toml:"safe-window" json:"safe-window"` - // The duration in addition to SafeWindow to make DDL safe. - AllowedClockDrift time.Duration `toml:"allowed-clock-drift" json:"allowed-clock-drift"` -} - -// CoprocessorCache is the config for coprocessor cache. -type CoprocessorCache struct { - // The capacity in MB of the cache. Zero means disable coprocessor cache. - CapacityMB float64 `toml:"capacity-mb" json:"capacity-mb"` - - // No json fields for below config. Intend to hide them. - - // Only cache requests that containing small number of ranges. May to be changed in future. - AdmissionMaxRanges uint64 `toml:"admission-max-ranges" json:"-"` - // Only cache requests whose result set is small. - AdmissionMaxResultMB float64 `toml:"admission-max-result-mb" json:"-"` - // Only cache requests takes notable time to process. - AdmissionMinProcessMs uint64 `toml:"admission-min-process-ms" json:"-"` -} - -// DefaultTiKVClient returns default config for TiKVClient. -func DefaultTiKVClient() TiKVClient { - return TiKVClient{ - GrpcConnectionCount: 4, - GrpcKeepAliveTime: 10, - GrpcKeepAliveTimeout: 3, - GrpcCompressionType: "none", - CommitTimeout: "41s", - AsyncCommit: AsyncCommit{ - // FIXME: Find an appropriate default limit. - KeysLimit: 256, - TotalKeySizeLimit: 4 * 1024, // 4 KiB - SafeWindow: 2 * time.Second, - AllowedClockDrift: 500 * time.Millisecond, - }, - - MaxBatchSize: 128, - OverloadThreshold: 200, - MaxBatchWaitTime: 0, - BatchWaitSize: 8, - - EnableChunkRPC: true, - - RegionCacheTTL: 600, - StoreLimit: 0, - StoreLivenessTimeout: DefStoreLivenessTimeout, - - TTLRefreshedTxnSize: 32 * 1024 * 1024, - - CoprCache: CoprocessorCache{ - CapacityMB: 1000, - AdmissionMaxRanges: 500, - AdmissionMaxResultMB: 10, - AdmissionMinProcessMs: 5, - }, - } -} - -// Valid checks if this config is valid. -func (config *TiKVClient) Valid() error { - if config.GrpcConnectionCount == 0 { - return fmt.Errorf("grpc-connection-count should be greater than 0") - } - if config.GrpcCompressionType != "none" && config.GrpcCompressionType != gzip.Name { - return fmt.Errorf("grpc-compression-type should be none or %s, but got %s", gzip.Name, config.GrpcCompressionType) - } - return nil -} diff --git a/store/tikv/config/config.go b/store/tikv/config/config.go deleted file mode 100644 index 8d178f05eab67..0000000000000 --- a/store/tikv/config/config.go +++ /dev/null @@ -1,221 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package config - -import ( - "fmt" - "net/http" - "net/url" - "strings" - "sync" - "sync/atomic" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/util" - "go.uber.org/zap" -) - -var ( - globalConf atomic.Value -) - -const ( - // DefStoresRefreshInterval is the default value of StoresRefreshInterval - DefStoresRefreshInterval = 60 -) - -func init() { - conf := DefaultConfig() - StoreGlobalConfig(&conf) -} - -// Config contains configuration options. -type Config struct { - CommitterConcurrency int - MaxTxnTTL uint64 - TiKVClient TiKVClient - Security Security - PDClient PDClient - PessimisticTxn PessimisticTxn - TxnLocalLatches TxnLocalLatches - // StoresRefreshInterval indicates the interval of refreshing stores info, the unit is second. - StoresRefreshInterval uint64 - OpenTracingEnable bool - Path string - EnableForwarding bool - TxnScope string -} - -// DefaultConfig returns the default configuration. -func DefaultConfig() Config { - return Config{ - CommitterConcurrency: 128, - MaxTxnTTL: 60 * 60 * 1000, // 1hour - TiKVClient: DefaultTiKVClient(), - PDClient: DefaultPDClient(), - TxnLocalLatches: DefaultTxnLocalLatches(), - StoresRefreshInterval: DefStoresRefreshInterval, - OpenTracingEnable: false, - Path: "", - EnableForwarding: false, - TxnScope: "", - } -} - -// PDClient is the config for PD client. -type PDClient struct { - // PDServerTimeout is the max time which PD client will wait for the PD server in seconds. - PDServerTimeout uint `toml:"pd-server-timeout" json:"pd-server-timeout"` -} - -// DefaultPDClient returns the default configuration for PDClient -func DefaultPDClient() PDClient { - return PDClient{ - PDServerTimeout: 3, - } -} - -// TxnLocalLatches is the TxnLocalLatches section of the config. -type TxnLocalLatches struct { - Enabled bool `toml:"-" json:"-"` - Capacity uint `toml:"-" json:"-"` -} - -// DefaultTxnLocalLatches returns the default configuration for TxnLocalLatches -func DefaultTxnLocalLatches() TxnLocalLatches { - return TxnLocalLatches{ - Enabled: false, - Capacity: 0, - } -} - -// Valid returns true if the configuration is valid. -func (c *TxnLocalLatches) Valid() error { - if c.Enabled && c.Capacity == 0 { - return fmt.Errorf("txn-local-latches.capacity can not be 0") - } - return nil -} - -// PessimisticTxn is the config for pessimistic transaction. -type PessimisticTxn struct { - // The max count of retry for a single statement in a pessimistic transaction. - MaxRetryCount uint `toml:"max-retry-count" json:"max-retry-count"` -} - -// GetGlobalConfig returns the global configuration for this server. -// It should store configuration from command line and configuration file. -// Other parts of the system can read the global configuration use this function. -func GetGlobalConfig() *Config { - return globalConf.Load().(*Config) -} - -// StoreGlobalConfig stores a new config to the globalConf. It mostly uses in the test to avoid some data races. -func StoreGlobalConfig(config *Config) { - globalConf.Store(config) -} - -// UpdateGlobal updates the global config, and provide a restore function that can be used to restore to the original. -func UpdateGlobal(f func(conf *Config)) func() { - g := GetGlobalConfig() - restore := func() { - StoreGlobalConfig(g) - } - newConf := *g - f(&newConf) - StoreGlobalConfig(&newConf) - return restore -} - -const ( - globalTxnScope = "global" -) - -// GetTxnScopeFromConfig extracts @@txn_scope value from config -func GetTxnScopeFromConfig() (bool, string) { - if val, err := util.EvalFailpoint("injectTxnScope"); err == nil { - v := val.(string) - if len(v) > 0 { - return false, v - } - return true, globalTxnScope - } - - if kvcfg := GetGlobalConfig(); kvcfg != nil && len(kvcfg.TxnScope) > 0 { - return false, kvcfg.TxnScope - } - return true, globalTxnScope -} - -// ParsePath parses this path. -// Path example: tikv://etcd-node1:port,etcd-node2:port?cluster=1&disableGC=false -func ParsePath(path string) (etcdAddrs []string, disableGC bool, err error) { - var u *url.URL - u, err = url.Parse(path) - if err != nil { - err = errors.Trace(err) - return - } - if strings.ToLower(u.Scheme) != "tikv" { - err = errors.Errorf("Uri scheme expected [tikv] but found [%s]", u.Scheme) - logutil.BgLogger().Error("parsePath error", zap.Error(err)) - return - } - switch strings.ToLower(u.Query().Get("disableGC")) { - case "true": - disableGC = true - case "false", "": - default: - err = errors.New("disableGC flag should be true/false") - return - } - etcdAddrs = strings.Split(u.Host, ",") - return -} - -var ( - internalClientInit sync.Once - internalHTTPClient *http.Client - internalHTTPSchema string -) - -// InternalHTTPClient is used by TiDB-Server to request other components. -func InternalHTTPClient() *http.Client { - internalClientInit.Do(initInternalClient) - return internalHTTPClient -} - -// InternalHTTPSchema specifies use http or https to request other components. -func InternalHTTPSchema() string { - internalClientInit.Do(initInternalClient) - return internalHTTPSchema -} - -func initInternalClient() { - clusterSecurity := GetGlobalConfig().Security - tlsCfg, err := clusterSecurity.ToTLSConfig() - if err != nil { - logutil.BgLogger().Fatal("could not load cluster ssl", zap.Error(err)) - } - if tlsCfg == nil { - internalHTTPSchema = "http" - internalHTTPClient = http.DefaultClient - return - } - internalHTTPSchema = "https" - internalHTTPClient = &http.Client{ - Transport: &http.Transport{TLSClientConfig: tlsCfg}, - } -} diff --git a/store/tikv/config/config_test.go b/store/tikv/config/config_test.go deleted file mode 100644 index 0dfdfdb38ac90..0000000000000 --- a/store/tikv/config/config_test.go +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package config - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/failpoint" -) - -var _ = SerialSuites(&testConfigSuite{}) - -func (s *testConfigSuite) TestParsePath(c *C) { - etcdAddrs, disableGC, err := ParsePath("tikv://node1:2379,node2:2379") - c.Assert(err, IsNil) - c.Assert(etcdAddrs, DeepEquals, []string{"node1:2379", "node2:2379"}) - c.Assert(disableGC, IsFalse) - - _, _, err = ParsePath("tikv://node1:2379") - c.Assert(err, IsNil) - _, disableGC, err = ParsePath("tikv://node1:2379?disableGC=true") - c.Assert(err, IsNil) - c.Assert(disableGC, IsTrue) -} - -func (s *testConfigSuite) TestTxnScopeValue(c *C) { - c.Assert(failpoint.Enable("tikvclient/injectTxnScope", `return("bj")`), IsNil) - isGlobal, v := GetTxnScopeFromConfig() - c.Assert(isGlobal, IsFalse) - c.Assert(v, Equals, "bj") - c.Assert(failpoint.Disable("tikvclient/injectTxnScope"), IsNil) - c.Assert(failpoint.Enable("tikvclient/injectTxnScope", `return("")`), IsNil) - isGlobal, v = GetTxnScopeFromConfig() - c.Assert(isGlobal, IsTrue) - c.Assert(v, Equals, "global") - c.Assert(failpoint.Disable("tikvclient/injectTxnScope"), IsNil) - c.Assert(failpoint.Enable("tikvclient/injectTxnScope", `return("global")`), IsNil) - isGlobal, v = GetTxnScopeFromConfig() - c.Assert(isGlobal, IsFalse) - c.Assert(v, Equals, "global") - c.Assert(failpoint.Disable("tikvclient/injectTxnScope"), IsNil) -} diff --git a/store/tikv/config/security.go b/store/tikv/config/security.go deleted file mode 100644 index 513dc40791509..0000000000000 --- a/store/tikv/config/security.go +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package config - -import ( - "crypto/tls" - "crypto/x509" - "os" - - "github.com/pingcap/errors" -) - -// Security is the security section of the config. -type Security struct { - ClusterSSLCA string `toml:"cluster-ssl-ca" json:"cluster-ssl-ca"` - ClusterSSLCert string `toml:"cluster-ssl-cert" json:"cluster-ssl-cert"` - ClusterSSLKey string `toml:"cluster-ssl-key" json:"cluster-ssl-key"` - ClusterVerifyCN []string `toml:"cluster-verify-cn" json:"cluster-verify-cn"` -} - -// NewSecurity creates a Security. -func NewSecurity(sslCA, sslCert, sslKey string, verityCN []string) Security { - return Security{ - ClusterSSLCA: sslCA, - ClusterSSLCert: sslCert, - ClusterSSLKey: sslKey, - ClusterVerifyCN: verityCN, - } -} - -// ToTLSConfig generates tls's config based on security section of the config. -func (s *Security) ToTLSConfig() (tlsConfig *tls.Config, err error) { - if len(s.ClusterSSLCA) != 0 { - certPool := x509.NewCertPool() - // Create a certificate pool from the certificate authority - var ca []byte - ca, err = os.ReadFile(s.ClusterSSLCA) - if err != nil { - err = errors.Errorf("could not read ca certificate: %s", err) - return - } - // Append the certificates from the CA - if !certPool.AppendCertsFromPEM(ca) { - err = errors.New("failed to append ca certs") - return - } - tlsConfig = &tls.Config{ - RootCAs: certPool, - ClientCAs: certPool, - } - - if len(s.ClusterSSLCert) != 0 && len(s.ClusterSSLKey) != 0 { - getCert := func() (*tls.Certificate, error) { - // Load the client certificates from disk - cert, err := tls.LoadX509KeyPair(s.ClusterSSLCert, s.ClusterSSLKey) - if err != nil { - return nil, errors.Errorf("could not load client key pair: %s", err) - } - return &cert, nil - } - // pre-test cert's loading. - if _, err = getCert(); err != nil { - return - } - tlsConfig.GetClientCertificate = func(info *tls.CertificateRequestInfo) (certificate *tls.Certificate, err error) { - return getCert() - } - tlsConfig.GetCertificate = func(info *tls.ClientHelloInfo) (certificate *tls.Certificate, err error) { - return getCert() - } - } - } - return -} diff --git a/store/tikv/config/security_test.go b/store/tikv/config/security_test.go deleted file mode 100644 index ad08c5b39452b..0000000000000 --- a/store/tikv/config/security_test.go +++ /dev/null @@ -1,113 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package config - -import ( - "os" - "path/filepath" - "runtime" - "testing" - - . "github.com/pingcap/check" -) - -var _ = SerialSuites(&testConfigSuite{}) - -type testConfigSuite struct{} - -func TestT(t *testing.T) { - CustomVerboseFlag = true - TestingT(t) -} - -func (s *testConfigSuite) TestConfig(c *C) { - // Test for TLS config. - certFile := "cert.pem" - _, localFile, _, _ := runtime.Caller(0) - certFile = filepath.Join(filepath.Dir(localFile), certFile) - f, err := os.Create(certFile) - c.Assert(err, IsNil) - _, err = f.WriteString(`-----BEGIN CERTIFICATE----- -MIIC+jCCAeKgAwIBAgIRALsvlisKJzXtiwKcv7toreswDQYJKoZIhvcNAQELBQAw -EjEQMA4GA1UEChMHQWNtZSBDbzAeFw0xOTAzMTMwNzExNDhaFw0yMDAzMTIwNzEx -NDhaMBIxEDAOBgNVBAoTB0FjbWUgQ28wggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAw -ggEKAoIBAQDECyY5cZ4SccQdk4XCgENwOLsE92uZvutBcYHk8ndIpxuxQnmS/2af -JxWlduKgauuLlwRYrzwvmUQumzB0LIJIwZN37KMeepTv+cf1Iv0U1Tw2PyXa7jD1 -VxccI7lHxqObYrnLdZ1AOG2SyWoJp/g6jZqbdGnYAbBxbZXYv9FyA6h0FksDysEP -62zu5YwtRcmhob7L5Wezq0/eV/2U1WdbGGWMCUs2LKQav4TP7Kaopk+MAl9UpSoc -arl+NGxs39TsvrxQvT7k/W6g7mo0rOc5PEc6Zho2+E8JjnEYCdGKmMW/Bea6V1yc -ShMe79lwN7ISCw3e7GZhZGM2XFTjvhH/AgMBAAGjSzBJMA4GA1UdDwEB/wQEAwIF -oDATBgNVHSUEDDAKBggrBgEFBQcDATAMBgNVHRMBAf8EAjAAMBQGA1UdEQQNMAuC -CWxvY2FsaG9zdDANBgkqhkiG9w0BAQsFAAOCAQEAK+pS76DxAbQBdbpyqt0Xi1QY -SnWxFEFepP3pHC28oy8fzHiys9fwMvJwgMgLcwyB9GUhMZ/xsO2ehutWbzYCCRmV -4einEx9Ipr26i2txzZPphqXNkV+ZhPeQK54fWrzAkRq4qKNyoYfvhldZ+uTuKNiS -If0KbvbS6qDfimA+m0m6n5yDzc5tPl+kgKyeivSyqeG7T9m40gvCLAMgI7iTFhIZ -BvUPi88z3wGa8rmhn9dOvkwauLFU5i5dqoz6m9HXmaEKzAAigGzgU8vPDt/Dxxgu -c933WW1E0hCtvuGxWFIFtoJMQoyH0Pl4ACmY/6CokCCZKDInrPdhhf3MGRjkkw== ------END CERTIFICATE----- -`) - c.Assert(err, IsNil) - c.Assert(f.Close(), IsNil) - - keyFile := "key.pem" - keyFile = filepath.Join(filepath.Dir(localFile), keyFile) - f, err = os.Create(keyFile) - c.Assert(err, IsNil) - _, err = f.WriteString(`-----BEGIN RSA PRIVATE KEY----- -MIIEowIBAAKCAQEAxAsmOXGeEnHEHZOFwoBDcDi7BPdrmb7rQXGB5PJ3SKcbsUJ5 -kv9mnycVpXbioGrri5cEWK88L5lELpswdCyCSMGTd+yjHnqU7/nH9SL9FNU8Nj8l -2u4w9VcXHCO5R8ajm2K5y3WdQDhtkslqCaf4Oo2am3Rp2AGwcW2V2L/RcgOodBZL -A8rBD+ts7uWMLUXJoaG+y+Vns6tP3lf9lNVnWxhljAlLNiykGr+Ez+ymqKZPjAJf -VKUqHGq5fjRsbN/U7L68UL0+5P1uoO5qNKznOTxHOmYaNvhPCY5xGAnRipjFvwXm -uldcnEoTHu/ZcDeyEgsN3uxmYWRjNlxU474R/wIDAQABAoIBAGyZAIOxvK7a9pir -r90e0DzKME9//8sbR5bpGduJtSo558051b7oXCCttgAC62eR0wlwjqfR6rUzYeGv -dhfk0AcdtGMqYvHvVbHZ3DqfNzLjLIegU4gDintd0x9zap+oGdlpxyI99O4uVASM -LoFK2ucUqiCTTE6sIOG0ot1+5LcS9xlygmmBfl8Q+6dG1D+vtPlU4J1kQ1MZV/JI -01Mbea4iiUKD9qrbxfsMiu52u/J3MMoWJHsvAA/LpOp2Ua6pUECluZECslxYSnJJ -IyjeGYxAIfXj81bqBk3RpemlX7YAxMbn6noZPQ6KUzS4IT2clrG10boCBdUNK1pN -WjVOEoECgYEA0/aP1wvrC3sZtHmURHv1El0vmaZocmH3sdwUfrW5cMqqhOosax6d -5iKAJQ1cAL6ZivIB4WJ3X8mlfMOaHPOQxqdudPui0sMHQehT2NBl/gwX9wXMwxXl -t+ebqK5DSSbVuJQS45sSdYPQvrMVDB/owHHjfdeOk1EwmqxHv1r338UCgYEA7MXk -IIF+LETxkw4QqbEPzwJ8kVRjkU3jmlEClOatTe+RQJxanErgMiGi9NZMM+Vm5GjC -5kzAuNgMDuD/NAWyzPzWd+mbeG/2IHYf44OiK1TmnFHkTc0JW7s4tUQgDMQccheR -EgA3UDGU9aevUoUDUhpeXxBdtnf66qw0e1cSovMCgYBLJdg7UsNjT6J+ZLhXS2dI -unb8z42qN+d8TF2LytvTDFdGRku3MqSiicrK2CCtNuXy5/gYszNFZ5VfVW3XI9dJ -RuUXXnuMo454JGlNrhzq49i/QHQnGiVWfSunsxix363YAc9smHcD6NbiNVWZ9dos -GHSiEgE/Y4KK49eQFS1aTQKBgQC+xzznTC+j7/FOcjjO4hJA1FoWp46Kl93ai4eu -/qeJcozxKIqCAHrhKeUprjo8Xo0nYZoZAqMOzVX57yTyf9zv+pG8kQhqZJxGz6cm -JPxYOdKPBhUU8y6lMReiRsAkSSg6be7AOFhZT3oc7f4AWZixYPnFU2SPD+GnkRXA -hApKLQKBgHUG+SjrxQjiFipE52YNGFLzbMR6Uga4baACW05uGPpao/+MkCGRAidL -d/8eU66iPNt/23iVAbqkF8mRpCxC0+O5HRqTEzgrlWKabXfmhYqIVjq+tkonJ0NU -xkNuJ2BlEGkwWLiRbKy1lNBBFUXKuhh3L/EIY10WTnr3TQzeL6H1 ------END RSA PRIVATE KEY----- -`) - c.Assert(err, IsNil) - c.Assert(f.Close(), IsNil) - security := Security{ - ClusterSSLCA: certFile, - ClusterSSLCert: certFile, - ClusterSSLKey: keyFile, - } - - tlsConfig, err := security.ToTLSConfig() - c.Assert(err, IsNil) - c.Assert(tlsConfig, NotNil) - - // Note that on windows, we can't Remove a file if the file is not closed. - // The behavior is different on linux, we can always Remove a file even - // if it's open. The OS maintains a reference count for open/close, the file - // is recycled when the reference count drops to 0. - c.Assert(os.Remove(certFile), IsNil) - c.Assert(os.Remove(keyFile), IsNil) -} diff --git a/store/tikv/delete_range.go b/store/tikv/delete_range.go deleted file mode 100644 index f42d5a94ccdae..0000000000000 --- a/store/tikv/delete_range.go +++ /dev/null @@ -1,151 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/client" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" -) - -// DeleteRangeTask is used to delete all keys in a range. After -// performing DeleteRange, it keeps how many ranges it affects and -// if the task was canceled or not. -type DeleteRangeTask struct { - completedRegions int - store Storage - startKey []byte - endKey []byte - notifyOnly bool - concurrency int -} - -// NewDeleteRangeTask creates a DeleteRangeTask. Deleting will be performed when `Execute` method is invoked. -// Be careful while using this API. This API doesn't keep recent MVCC versions, but will delete all versions of all keys -// in the range immediately. Also notice that frequent invocation to this API may cause performance problems to TiKV. -func NewDeleteRangeTask(store Storage, startKey []byte, endKey []byte, concurrency int) *DeleteRangeTask { - return &DeleteRangeTask{ - completedRegions: 0, - store: store, - startKey: startKey, - endKey: endKey, - notifyOnly: false, - concurrency: concurrency, - } -} - -// NewNotifyDeleteRangeTask creates a task that sends delete range requests to all regions in the range, but with the -// flag `notifyOnly` set. TiKV will not actually delete the range after receiving request, but it will be replicated via -// raft. This is used to notify the involved regions before sending UnsafeDestroyRange requests. -func NewNotifyDeleteRangeTask(store Storage, startKey []byte, endKey []byte, concurrency int) *DeleteRangeTask { - task := NewDeleteRangeTask(store, startKey, endKey, concurrency) - task.notifyOnly = true - return task -} - -// getRunnerName returns a name for RangeTaskRunner. -func (t *DeleteRangeTask) getRunnerName() string { - if t.notifyOnly { - return "delete-range-notify" - } - return "delete-range" -} - -// Execute performs the delete range operation. -func (t *DeleteRangeTask) Execute(ctx context.Context) error { - runnerName := t.getRunnerName() - - runner := NewRangeTaskRunner(runnerName, t.store, t.concurrency, t.sendReqOnRange) - err := runner.RunOnRange(ctx, t.startKey, t.endKey) - t.completedRegions = runner.CompletedRegions() - - return err -} - -const deleteRangeOneRegionMaxBackoff = 100000 - -// Execute performs the delete range operation. -func (t *DeleteRangeTask) sendReqOnRange(ctx context.Context, r kv.KeyRange) (RangeTaskStat, error) { - startKey, rangeEndKey := r.StartKey, r.EndKey - var stat RangeTaskStat - for { - select { - case <-ctx.Done(): - return stat, errors.Trace(ctx.Err()) - default: - } - - if bytes.Compare(startKey, rangeEndKey) >= 0 { - break - } - - bo := retry.NewBackofferWithVars(ctx, deleteRangeOneRegionMaxBackoff, nil) - loc, err := t.store.GetRegionCache().LocateKey(bo, startKey) - if err != nil { - return stat, errors.Trace(err) - } - - // Delete to the end of the region, except if it's the last region overlapping the range - endKey := loc.EndKey - // If it is the last region - if loc.Contains(rangeEndKey) { - endKey = rangeEndKey - } - - req := tikvrpc.NewRequest(tikvrpc.CmdDeleteRange, &kvrpcpb.DeleteRangeRequest{ - StartKey: startKey, - EndKey: endKey, - NotifyOnly: t.notifyOnly, - }) - - resp, err := t.store.SendReq(bo, req, loc.Region, client.ReadTimeoutMedium) - if err != nil { - return stat, errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return stat, errors.Trace(err) - } - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return stat, errors.Trace(err) - } - continue - } - if resp.Resp == nil { - return stat, errors.Trace(tikverr.ErrBodyMissing) - } - deleteRangeResp := resp.Resp.(*kvrpcpb.DeleteRangeResponse) - if err := deleteRangeResp.GetError(); err != "" { - return stat, errors.Errorf("unexpected delete range err: %v", err) - } - stat.CompletedRegions++ - startKey = endKey - } - - return stat, nil -} - -// CompletedRegions returns the number of regions that are affected by this delete range task -func (t *DeleteRangeTask) CompletedRegions() int { - return t.completedRegions -} diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go deleted file mode 100644 index 4f280732cc603..0000000000000 --- a/store/tikv/error/error.go +++ /dev/null @@ -1,203 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package error - -import ( - "fmt" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/pdpb" -) - -var ( - // ErrBodyMissing response body is missing error - ErrBodyMissing = errors.New("response body is missing") - // ErrTiDBShuttingDown is returned when TiDB is closing and send request to tikv fail, do not retry. - ErrTiDBShuttingDown = errors.New("tidb server shutting down") - // ErrNotExist means the related data not exist. - ErrNotExist = errors.New("not exist") - // ErrCannotSetNilValue is the error when sets an empty value. - ErrCannotSetNilValue = errors.New("can not set nil value") - // ErrInvalidTxn is the error when commits or rollbacks in an invalid transaction. - ErrInvalidTxn = errors.New("invalid transaction") - // ErrTiKVServerTimeout is the error when tikv server is timeout. - ErrTiKVServerTimeout = errors.New("tikv server timeout") - // ErrTiFlashServerTimeout is the error when tiflash server is timeout. - ErrTiFlashServerTimeout = errors.New("tiflash server timeout") - // ErrQueryInterrupted is the error when the query is interrupted. - ErrQueryInterrupted = errors.New("query interruppted") - // ErrTiKVStaleCommand is the error that the command is stale in tikv. - ErrTiKVStaleCommand = errors.New("tikv stale command") - // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. - ErrTiKVMaxTimestampNotSynced = errors.New("tikv max timestamp not synced") - // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. - ErrLockAcquireFailAndNoWaitSet = errors.New("lock acquired failed and no wait is setted") - // ErrResolveLockTimeout is the error that resolve lock timeout. - ErrResolveLockTimeout = errors.New("resolve lock timeout") - // ErrLockWaitTimeout is the error that wait for the lock is timeout. - ErrLockWaitTimeout = errors.New("lock wait timeout") - // ErrTiKVServerBusy is the error when tikv server is busy. - ErrTiKVServerBusy = errors.New("tikv server busy") - // ErrTiFlashServerBusy is the error that tiflash server is busy. - ErrTiFlashServerBusy = errors.New("tiflash server busy") - // ErrRegionUnavailable is the error when region is not available. - ErrRegionUnavailable = errors.New("region unavailable") - // ErrRegionDataNotReady is the error when region's data is not ready when querying it with safe_ts - ErrRegionDataNotReady = errors.New("region data not ready") - // ErrRegionNotInitialized is error when region is not initialized - ErrRegionNotInitialized = errors.New("region not Initialized") - // ErrUnknown is the unknow error. - ErrUnknown = errors.New("unknow") -) - -// MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. -const MismatchClusterID = "mismatch cluster id" - -// IsErrNotFound checks if err is a kind of NotFound error. -func IsErrNotFound(err error) bool { - return errors.ErrorEqual(err, ErrNotExist) -} - -// ErrDeadlock wraps *kvrpcpb.Deadlock to implement the error interface. -// It also marks if the deadlock is retryable. -type ErrDeadlock struct { - *kvrpcpb.Deadlock - IsRetryable bool -} - -func (d *ErrDeadlock) Error() string { - return d.Deadlock.String() -} - -// PDError wraps *pdpb.Error to implement the error interface. -type PDError struct { - Err *pdpb.Error -} - -func (d *PDError) Error() string { - return d.Err.String() -} - -// ErrKeyExist wraps *pdpb.AlreadyExist to implement the error interface. -type ErrKeyExist struct { - *kvrpcpb.AlreadyExist -} - -func (k *ErrKeyExist) Error() string { - return k.AlreadyExist.String() -} - -// IsErrKeyExist returns true if it is ErrKeyExist. -func IsErrKeyExist(err error) bool { - _, ok := errors.Cause(err).(*ErrKeyExist) - return ok -} - -// ErrWriteConflict wraps *kvrpcpb.ErrWriteConflict to implement the error interface. -type ErrWriteConflict struct { - *kvrpcpb.WriteConflict -} - -func (k *ErrWriteConflict) Error() string { - return k.WriteConflict.String() -} - -// IsErrWriteConflict returns true if it is ErrWriteConflict. -func IsErrWriteConflict(err error) bool { - _, ok := errors.Cause(err).(*ErrWriteConflict) - return ok -} - -//NewErrWriteConfictWithArgs generates an ErrWriteConflict with args. -func NewErrWriteConfictWithArgs(startTs, conflictTs, conflictCommitTs uint64, key []byte) *ErrWriteConflict { - conflict := kvrpcpb.WriteConflict{ - StartTs: startTs, - ConflictTs: conflictTs, - Key: key, - ConflictCommitTs: conflictCommitTs, - } - return &ErrWriteConflict{WriteConflict: &conflict} -} - -// ErrWriteConflictInLatch is the error when the commit meets an write conflict error when local latch is enabled. -type ErrWriteConflictInLatch struct { - StartTS uint64 -} - -func (e *ErrWriteConflictInLatch) Error() string { - return fmt.Sprintf("write conflict in latch,startTS: %v", e.StartTS) -} - -// ErrRetryable wraps *kvrpcpb.Retryable to implement the error interface. -type ErrRetryable struct { - Retryable string -} - -func (k *ErrRetryable) Error() string { - return k.Retryable -} - -// ErrTxnTooLarge is the error when transaction is too large, lock time reached the maximum value. -type ErrTxnTooLarge struct { - Size int -} - -func (e *ErrTxnTooLarge) Error() string { - return fmt.Sprintf("txn too large, size: %v.", e.Size) -} - -// ErrEntryTooLarge is the error when a key value entry is too large. -type ErrEntryTooLarge struct { - Limit uint64 - Size uint64 -} - -func (e *ErrEntryTooLarge) Error() string { - return fmt.Sprintf("entry size too large, size: %v,limit: %v.", e.Size, e.Limit) -} - -// ErrPDServerTimeout is the error when pd server is timeout. -type ErrPDServerTimeout struct { - msg string -} - -// NewErrPDServerTimeout creates an ErrPDServerTimeout. -func NewErrPDServerTimeout(msg string) error { - return &ErrPDServerTimeout{msg} -} - -func (e *ErrPDServerTimeout) Error() string { - return e.msg -} - -// ErrGCTooEarly is the error that GC life time is shorter than transaction duration -type ErrGCTooEarly struct { - TxnStartTS time.Time - GCSafePoint time.Time -} - -func (e *ErrGCTooEarly) Error() string { - return fmt.Sprintf("GC life time is shorter than transaction duration, transaction starts at %v, GC safe point is %v", e.TxnStartTS, e.GCSafePoint) -} - -// ErrTokenLimit is the error that token is up to the limit. -type ErrTokenLimit struct { - StoreID uint64 -} - -func (e *ErrTokenLimit) Error() string { - return fmt.Sprintf("Store token is up to the limit, store id = %d.", e.StoreID) -} diff --git a/store/tikv/interface.go b/store/tikv/interface.go deleted file mode 100644 index 21097e7e3eb58..0000000000000 --- a/store/tikv/interface.go +++ /dev/null @@ -1,63 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "time" - - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/tikvrpc" -) - -// Storage represent the kv.Storage runs on TiKV. -type Storage interface { - // GetRegionCache gets the RegionCache. - GetRegionCache() *locate.RegionCache - - // SendReq sends a request to TiKV. - SendReq(bo *Backoffer, req *tikvrpc.Request, regionID locate.RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) - - // GetLockResolver gets the LockResolver. - GetLockResolver() *LockResolver - - // GetSafePointKV gets the SafePointKV. - GetSafePointKV() SafePointKV - - // UpdateSPCache updates the cache of safe point. - UpdateSPCache(cachedSP uint64, cachedTime time.Time) - - // SetOracle sets the Oracle. - SetOracle(oracle oracle.Oracle) - - // SetTiKVClient sets the TiKV client. - SetTiKVClient(client Client) - - // GetTiKVClient gets the TiKV client. - GetTiKVClient() Client - - // Closed returns the closed channel. - Closed() <-chan struct{} - - // Close store - Close() error - // UUID return a unique ID which represents a Storage. - UUID() string - // CurrentTimestamp returns current timestamp with the given txnScope (local or global). - CurrentTimestamp(txnScope string) (uint64, error) - // GetOracle gets a timestamp oracle client. - GetOracle() oracle.Oracle - // SupportDeleteRange gets the storage support delete range or not. - SupportDeleteRange() (supported bool) -} diff --git a/store/tikv/kv.go b/store/tikv/kv.go deleted file mode 100644 index e5b9d501bb471..0000000000000 --- a/store/tikv/kv.go +++ /dev/null @@ -1,435 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "context" - "crypto/tls" - "math" - "math/rand" - "strconv" - "sync" - "sync/atomic" - "time" - - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/config" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/latch" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/oracle/oracles" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.uber.org/zap" -) - -// DCLabelKey indicates the key of label which represents the dc for Store. -const DCLabelKey = "zone" - -func createEtcdKV(addrs []string, tlsConfig *tls.Config) (*clientv3.Client, error) { - cfg := config.GetGlobalConfig() - cli, err := clientv3.New(clientv3.Config{ - Endpoints: addrs, - AutoSyncInterval: 30 * time.Second, - DialTimeout: 5 * time.Second, - TLS: tlsConfig, - DialKeepAliveTime: time.Second * time.Duration(cfg.TiKVClient.GrpcKeepAliveTime), - DialKeepAliveTimeout: time.Second * time.Duration(cfg.TiKVClient.GrpcKeepAliveTimeout), - }) - if err != nil { - return nil, errors.Trace(err) - } - return cli, nil -} - -// update oracle's lastTS every 2000ms. -var oracleUpdateInterval = 2000 - -// KVStore contains methods to interact with a TiKV cluster. -type KVStore struct { - clusterID uint64 - uuid string - oracle oracle.Oracle - clientMu struct { - sync.RWMutex - client Client - } - pdClient pd.Client - regionCache *locate.RegionCache - lockResolver *LockResolver - txnLatches *latch.LatchesScheduler - - mock bool - - kv SafePointKV - safePoint uint64 - spTime time.Time - spMutex sync.RWMutex // this is used to update safePoint and spTime - closed chan struct{} // this is used to notify when the store is closed - - // storeID -> safeTS, stored as map[uint64]uint64 - // safeTS here will be used during the Stale Read process, - // it indicates the safe timestamp point that can be used to read consistent but may not the latest data. - safeTSMap sync.Map - - replicaReadSeed uint32 // this is used to load balance followers / learners when replica read is enabled -} - -// UpdateSPCache updates cached safepoint. -func (s *KVStore) UpdateSPCache(cachedSP uint64, cachedTime time.Time) { - s.spMutex.Lock() - s.safePoint = cachedSP - s.spTime = cachedTime - s.spMutex.Unlock() -} - -// CheckVisibility checks if it is safe to read using given ts. -func (s *KVStore) CheckVisibility(startTime uint64) error { - s.spMutex.RLock() - cachedSafePoint := s.safePoint - cachedTime := s.spTime - s.spMutex.RUnlock() - diff := time.Since(cachedTime) - - if diff > (GcSafePointCacheInterval - gcCPUTimeInaccuracyBound) { - return tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point") - } - - if startTime < cachedSafePoint { - t1 := oracle.GetTimeFromTS(startTime) - t2 := oracle.GetTimeFromTS(cachedSafePoint) - return &tikverr.ErrGCTooEarly{ - TxnStartTS: t1, - GCSafePoint: t2, - } - } - - return nil -} - -// NewKVStore creates a new TiKV store instance. -func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Client) (*KVStore, error) { - o, err := oracles.NewPdOracle(pdClient, time.Duration(oracleUpdateInterval)*time.Millisecond) - if err != nil { - return nil, errors.Trace(err) - } - store := &KVStore{ - clusterID: pdClient.GetClusterID(context.TODO()), - uuid: uuid, - oracle: o, - pdClient: pdClient, - regionCache: locate.NewRegionCache(pdClient), - kv: spkv, - safePoint: 0, - spTime: time.Now(), - closed: make(chan struct{}), - replicaReadSeed: rand.Uint32(), - } - store.clientMu.client = client.NewReqCollapse(tikvclient) - store.lockResolver = newLockResolver(store) - - go store.runSafePointChecker() - go store.safeTSUpdater() - - return store, nil -} - -// EnableTxnLocalLatches enables txn latch. It should be called before using -// the store to serve any requests. -func (s *KVStore) EnableTxnLocalLatches(size uint) { - s.txnLatches = latch.NewScheduler(size) -} - -// IsLatchEnabled is used by mockstore.TestConfig. -func (s *KVStore) IsLatchEnabled() bool { - return s.txnLatches != nil -} - -func (s *KVStore) runSafePointChecker() { - d := gcSafePointUpdateInterval - for { - select { - case spCachedTime := <-time.After(d): - cachedSafePoint, err := loadSafePoint(s.GetSafePointKV()) - if err == nil { - metrics.TiKVLoadSafepointCounter.WithLabelValues("ok").Inc() - s.UpdateSPCache(cachedSafePoint, spCachedTime) - d = gcSafePointUpdateInterval - } else { - metrics.TiKVLoadSafepointCounter.WithLabelValues("fail").Inc() - logutil.BgLogger().Error("fail to load safepoint from pd", zap.Error(err)) - d = gcSafePointQuickRepeatInterval - } - case <-s.Closed(): - return - } - } -} - -// Begin a global transaction. -func (s *KVStore) Begin() (*KVTxn, error) { - return s.BeginWithOption(DefaultStartTSOption()) -} - -// BeginWithOption begins a transaction with the given StartTSOption -func (s *KVStore) BeginWithOption(options StartTSOption) (*KVTxn, error) { - return newTiKVTxnWithOptions(s, options) -} - -// GetSnapshot gets a snapshot that is able to read any data which data is <= ver. -// if ts is MaxVersion or > current max committed version, we will use current version for this snapshot. -func (s *KVStore) GetSnapshot(ts uint64) *KVSnapshot { - snapshot := newTiKVSnapshot(s, ts, s.nextReplicaReadSeed()) - return snapshot -} - -// Close store -func (s *KVStore) Close() error { - s.oracle.Close() - s.pdClient.Close() - - close(s.closed) - if err := s.GetTiKVClient().Close(); err != nil { - return errors.Trace(err) - } - - if s.txnLatches != nil { - s.txnLatches.Close() - } - s.regionCache.Close() - - if err := s.kv.Close(); err != nil { - return errors.Trace(err) - } - return nil -} - -// UUID return a unique ID which represents a Storage. -func (s *KVStore) UUID() string { - return s.uuid -} - -// CurrentTimestamp returns current timestamp with the given txnScope (local or global). -func (s *KVStore) CurrentTimestamp(txnScope string) (uint64, error) { - bo := retry.NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTS, err := s.getTimestampWithRetry(bo, txnScope) - if err != nil { - return 0, errors.Trace(err) - } - return startTS, nil -} - -func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, error) { - if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("TiKVStore.getTimestampWithRetry", opentracing.ChildOf(span.Context())) - defer span1.Finish() - bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) - } - - for { - startTS, err := s.oracle.GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: txnScope}) - // mockGetTSErrorInRetry should wait MockCommitErrorOnce first, then will run into retry() logic. - // Then mockGetTSErrorInRetry will return retryable error when first retry. - // Before PR #8743, we don't cleanup txn after meet error such as error like: PD server timeout - // This may cause duplicate data to be written. - if val, err := util.EvalFailpoint("mockGetTSErrorInRetry"); err == nil { - if val.(bool) && !IsMockCommitErrorEnable() { - err = tikverr.NewErrPDServerTimeout("mock PD timeout") - } - } - - if err == nil { - return startTS, nil - } - err = bo.Backoff(retry.BoPDRPC, errors.Errorf("get timestamp failed: %v", err)) - if err != nil { - return 0, errors.Trace(err) - } - } -} - -func (s *KVStore) nextReplicaReadSeed() uint32 { - return atomic.AddUint32(&s.replicaReadSeed, 1) -} - -// GetOracle gets a timestamp oracle client. -func (s *KVStore) GetOracle() oracle.Oracle { - return s.oracle -} - -// GetPDClient returns the PD client. -func (s *KVStore) GetPDClient() pd.Client { - return s.pdClient -} - -// SupportDeleteRange gets the storage support delete range or not. -func (s *KVStore) SupportDeleteRange() (supported bool) { - return !s.mock -} - -// SendReq sends a request to locate. -func (s *KVStore) SendReq(bo *Backoffer, req *tikvrpc.Request, regionID locate.RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) { - sender := locate.NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) - return sender.SendReq(bo, req, regionID, timeout) -} - -// GetRegionCache returns the region cache instance. -func (s *KVStore) GetRegionCache() *locate.RegionCache { - return s.regionCache -} - -// GetLockResolver returns the lock resolver instance. -func (s *KVStore) GetLockResolver() *LockResolver { - return s.lockResolver -} - -// Closed returns a channel that indicates if the store is closed. -func (s *KVStore) Closed() <-chan struct{} { - return s.closed -} - -// GetSafePointKV returns the kv store that used for safepoint. -func (s *KVStore) GetSafePointKV() SafePointKV { - return s.kv -} - -// SetOracle resets the oracle instance. -func (s *KVStore) SetOracle(oracle oracle.Oracle) { - s.oracle = oracle -} - -// SetTiKVClient resets the client instance. -func (s *KVStore) SetTiKVClient(client Client) { - s.clientMu.Lock() - defer s.clientMu.Unlock() - s.clientMu.client = client -} - -// GetTiKVClient gets the client instance. -func (s *KVStore) GetTiKVClient() (client Client) { - s.clientMu.RLock() - defer s.clientMu.RUnlock() - return s.clientMu.client -} - -// GetMinSafeTS return the minimal safeTS of the storage with given txnScope. -func (s *KVStore) GetMinSafeTS(txnScope string) uint64 { - stores := make([]*locate.Store, 0) - allStores := s.regionCache.GetStoresByType(tikvrpc.TiKV) - if txnScope != oracle.GlobalTxnScope { - for _, store := range allStores { - if store.IsLabelsMatch([]*metapb.StoreLabel{ - { - Key: DCLabelKey, - Value: txnScope, - }, - }) { - stores = append(stores, store) - } - } - } else { - stores = allStores - } - return s.getMinSafeTSByStores(stores) -} - -func (s *KVStore) getSafeTS(storeID uint64) uint64 { - safeTS, ok := s.safeTSMap.Load(storeID) - if !ok { - return 0 - } - return safeTS.(uint64) -} - -// setSafeTS sets safeTs for store storeID, export for testing -func (s *KVStore) setSafeTS(storeID, safeTS uint64) { - s.safeTSMap.Store(storeID, safeTS) -} - -func (s *KVStore) getMinSafeTSByStores(stores []*locate.Store) uint64 { - if val, err := util.EvalFailpoint("injectSafeTS"); err == nil { - injectTS := val.(int) - return uint64(injectTS) - } - minSafeTS := uint64(math.MaxUint64) - // when there is no store, return 0 in order to let minStartTS become startTS directly - if len(stores) < 1 { - return 0 - } - for _, store := range stores { - safeTS := s.getSafeTS(store.StoreID()) - if safeTS < minSafeTS { - minSafeTS = safeTS - } - } - return minSafeTS -} - -func (s *KVStore) safeTSUpdater() { - t := time.NewTicker(time.Second * 2) - defer t.Stop() - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - for { - select { - case <-s.Closed(): - return - case <-t.C: - s.updateSafeTS(ctx) - } - } -} - -func (s *KVStore) updateSafeTS(ctx context.Context) { - stores := s.regionCache.GetStoresByType(tikvrpc.TiKV) - tikvClient := s.GetTiKVClient() - wg := &sync.WaitGroup{} - wg.Add(len(stores)) - for _, store := range stores { - storeID := store.StoreID() - storeAddr := store.GetAddr() - go func(ctx context.Context, wg *sync.WaitGroup, storeID uint64, storeAddr string) { - defer wg.Done() - resp, err := tikvClient.SendRequest(ctx, storeAddr, tikvrpc.NewRequest(tikvrpc.CmdStoreSafeTS, &kvrpcpb.StoreSafeTSRequest{KeyRange: &kvrpcpb.KeyRange{ - StartKey: []byte(""), - EndKey: []byte(""), - }}), client.ReadTimeoutShort) - storeIDStr := strconv.Itoa(int(storeID)) - if err != nil { - metrics.TiKVSafeTSUpdateCounter.WithLabelValues("fail", storeIDStr).Inc() - logutil.BgLogger().Debug("update safeTS failed", zap.Error(err), zap.Uint64("store-id", storeID)) - return - } - safeTSResp := resp.Resp.(*kvrpcpb.StoreSafeTSResponse) - s.setSafeTS(storeID, safeTSResp.GetSafeTs()) - metrics.TiKVSafeTSUpdateCounter.WithLabelValues("success", storeIDStr).Inc() - metrics.TiKVSafeTSUpdateStats.WithLabelValues(storeIDStr).Set(float64(safeTSResp.GetSafeTs())) - }(ctx, wg, storeID, storeAddr) - } - wg.Wait() -} - -// Variables defines the variables used by TiKV storage. -type Variables = kv.Variables diff --git a/store/tikv/kv/key.go b/store/tikv/kv/key.go deleted file mode 100644 index 6bc116c24b141..0000000000000 --- a/store/tikv/kv/key.go +++ /dev/null @@ -1,44 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -import ( - "bytes" - "encoding/hex" -) - -// NextKey returns the next key in byte-order. -func NextKey(k []byte) []byte { - // add 0x0 to the end of key - buf := make([]byte, len(k)+1) - copy(buf, k) - return buf -} - -// CmpKey returns the comparison result of two key. -// The result will be 0 if a==b, -1 if a < b, and +1 if a > b. -func CmpKey(k, another []byte) int { - return bytes.Compare(k, another) -} - -// StrKey returns string for key. -func StrKey(k []byte) string { - return hex.EncodeToString(k) -} - -// KeyRange represents a range where StartKey <= key < EndKey. -type KeyRange struct { - StartKey []byte - EndKey []byte -} diff --git a/store/tikv/kv/keyflags.go b/store/tikv/kv/keyflags.go deleted file mode 100644 index a98330f080f71..0000000000000 --- a/store/tikv/kv/keyflags.go +++ /dev/null @@ -1,129 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -// KeyFlags are metadata associated with key -type KeyFlags uint8 - -const ( - flagPresumeKNE KeyFlags = 1 << iota - flagKeyLocked - flagNeedLocked - flagKeyLockedValExist - flagNeedCheckExists - flagPrewriteOnly - flagIgnoredIn2PC - - persistentFlags = flagKeyLocked | flagKeyLockedValExist -) - -// HasPresumeKeyNotExists returns whether the associated key use lazy check. -func (f KeyFlags) HasPresumeKeyNotExists() bool { - return f&flagPresumeKNE != 0 -} - -// HasLocked returns whether the associated key has acquired pessimistic lock. -func (f KeyFlags) HasLocked() bool { - return f&flagKeyLocked != 0 -} - -// HasNeedLocked return whether the key needed to be locked -func (f KeyFlags) HasNeedLocked() bool { - return f&flagNeedLocked != 0 -} - -// HasLockedValueExists returns whether the value exists when key locked. -func (f KeyFlags) HasLockedValueExists() bool { - return f&flagKeyLockedValExist != 0 -} - -// HasNeedCheckExists returns whether the key need to check existence when it has been locked. -func (f KeyFlags) HasNeedCheckExists() bool { - return f&flagNeedCheckExists != 0 -} - -// HasPrewriteOnly returns whether the key should be used in 2pc commit phase. -func (f KeyFlags) HasPrewriteOnly() bool { - return f&flagPrewriteOnly != 0 -} - -// HasIgnoredIn2PC returns whether the key will be ignored in 2pc. -func (f KeyFlags) HasIgnoredIn2PC() bool { - return f&flagIgnoredIn2PC != 0 -} - -// AndPersistent returns the value of current flags&persistentFlags -func (f KeyFlags) AndPersistent() KeyFlags { - return f & persistentFlags -} - -// ApplyFlagsOps applys flagspos to origin. -func ApplyFlagsOps(origin KeyFlags, ops ...FlagsOp) KeyFlags { - for _, op := range ops { - switch op { - case SetPresumeKeyNotExists: - origin |= flagPresumeKNE | flagNeedCheckExists - case DelPresumeKeyNotExists: - origin &= ^(flagPresumeKNE | flagNeedCheckExists) - case SetKeyLocked: - origin |= flagKeyLocked - case DelKeyLocked: - origin &= ^flagKeyLocked - case SetNeedLocked: - origin |= flagNeedLocked - case DelNeedLocked: - origin &= ^flagNeedLocked - case SetKeyLockedValueExists: - origin |= flagKeyLockedValExist - case DelNeedCheckExists: - origin &= ^flagNeedCheckExists - case SetKeyLockedValueNotExists: - origin &= ^flagKeyLockedValExist - case SetPrewriteOnly: - origin |= flagPrewriteOnly - case SetIgnoredIn2PC: - origin |= flagIgnoredIn2PC - } - } - return origin -} - -// FlagsOp describes KeyFlags modify operation. -type FlagsOp uint16 - -const ( - // SetPresumeKeyNotExists marks the existence of the associated key is checked lazily. - // Implies KeyFlags.HasNeedCheckExists() == true. - SetPresumeKeyNotExists FlagsOp = 1 << iota - // DelPresumeKeyNotExists reverts SetPresumeKeyNotExists. - DelPresumeKeyNotExists - // SetKeyLocked marks the associated key has acquired lock. - SetKeyLocked - // DelKeyLocked reverts SetKeyLocked. - DelKeyLocked - // SetNeedLocked marks the associated key need to be acquired lock. - SetNeedLocked - // DelNeedLocked reverts SetKeyNeedLocked. - DelNeedLocked - // SetKeyLockedValueExists marks the value exists when key has been locked in Transaction.LockKeys. - SetKeyLockedValueExists - // SetKeyLockedValueNotExists marks the value doesn't exists when key has been locked in Transaction.LockKeys. - SetKeyLockedValueNotExists - // DelNeedCheckExists marks the key no need to be checked in Transaction.LockKeys. - DelNeedCheckExists - // SetPrewriteOnly marks the key shouldn't be used in 2pc commit phase. - SetPrewriteOnly - // SetIgnoredIn2PC marks the key will be ignored in 2pc. - SetIgnoredIn2PC -) diff --git a/store/tikv/kv/kv.go b/store/tikv/kv/kv.go deleted file mode 100644 index 0d900d6facddb..0000000000000 --- a/store/tikv/kv/kv.go +++ /dev/null @@ -1,61 +0,0 @@ -package kv - -import ( - "sync" - "time" - - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/util" -) - -// ReturnedValue pairs the Value and AlreadyLocked flag for PessimisticLock return values result. -type ReturnedValue struct { - Value []byte - AlreadyLocked bool -} - -// LockCtx contains information for LockKeys method. -type LockCtx struct { - Killed *uint32 - ForUpdateTS uint64 - LockWaitTime int64 - WaitStartTime time.Time - PessimisticLockWaited *int32 - LockKeysDuration *int64 - LockKeysCount *int32 - ReturnValues bool - Values map[string]ReturnedValue - ValuesLock sync.Mutex - LockExpired *uint32 - Stats *util.LockKeysDetails - ResourceGroupTag []byte - OnDeadlock func(*tikverr.ErrDeadlock) -} - -// InitReturnValues creates the map to store returned value. -func (ctx *LockCtx) InitReturnValues(valueLen int) { - ctx.ReturnValues = true - ctx.Values = make(map[string]ReturnedValue, valueLen) -} - -// GetValueNotLocked returns a value if the key is not already locked. -// (nil, false) means already locked. -func (ctx *LockCtx) GetValueNotLocked(key []byte) ([]byte, bool) { - rv := ctx.Values[string(key)] - if !rv.AlreadyLocked { - return rv.Value, true - } - return nil, false -} - -// IterateValuesNotLocked applies f to all key-values that are not already -// locked. -func (ctx *LockCtx) IterateValuesNotLocked(f func([]byte, []byte)) { - ctx.ValuesLock.Lock() - defer ctx.ValuesLock.Unlock() - for key, val := range ctx.Values { - if !val.AlreadyLocked { - f([]byte(key), val.Value) - } - } -} diff --git a/store/tikv/kv/store_vars.go b/store/tikv/kv/store_vars.go deleted file mode 100644 index 02d87018213a9..0000000000000 --- a/store/tikv/kv/store_vars.go +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -import ( - "go.uber.org/atomic" -) - -// StoreLimit will update from config reload and global variable set. -var StoreLimit atomic.Int64 - -// ReplicaReadType is the type of replica to read data from -type ReplicaReadType byte - -const ( - // ReplicaReadLeader stands for 'read from leader'. - ReplicaReadLeader ReplicaReadType = iota - // ReplicaReadFollower stands for 'read from follower'. - ReplicaReadFollower - // ReplicaReadMixed stands for 'read from leader and follower and learner'. - ReplicaReadMixed -) - -// IsFollowerRead checks if follower is going to be used to read data. -func (r ReplicaReadType) IsFollowerRead() bool { - return r != ReplicaReadLeader -} diff --git a/store/tikv/kv/variables.go b/store/tikv/kv/variables.go deleted file mode 100644 index 5e7a4c83b669a..0000000000000 --- a/store/tikv/kv/variables.go +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -// Variables defines the variables used by KV storage. -type Variables struct { - // BackoffLockFast specifies the LockFast backoff base duration in milliseconds. - BackoffLockFast int - - // BackOffWeight specifies the weight of the max back off time duration. - BackOffWeight int - - // Pointer to SessionVars.Killed - // Killed is a flag to indicate that this query is killed. - Killed *uint32 -} - -// NewVariables create a new Variables instance with default values. -func NewVariables(killed *uint32) *Variables { - return &Variables{ - BackoffLockFast: DefBackoffLockFast, - BackOffWeight: DefBackOffWeight, - Killed: killed, - } -} - -var ignoreKill uint32 - -// DefaultVars is the default variables instance. -var DefaultVars = NewVariables(&ignoreKill) - -// Default values -const ( - DefBackoffLockFast = 100 - DefBackOffWeight = 2 -) diff --git a/store/tikv/latch/latch.go b/store/tikv/latch/latch.go deleted file mode 100644 index 812e15c96fbd3..0000000000000 --- a/store/tikv/latch/latch.go +++ /dev/null @@ -1,304 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package latch - -import ( - "bytes" - "math/bits" - "sort" - "sync" - "time" - - "github.com/cznic/mathutil" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/twmb/murmur3" - "go.uber.org/zap" -) - -type node struct { - slotID int - key []byte - maxCommitTS uint64 - value *Lock - - next *node -} - -// latch stores a key's waiting transactions information. -type latch struct { - queue *node - count int - waiting []*Lock - sync.Mutex -} - -// Lock is the locks' information required for a transaction. -type Lock struct { - keys [][]byte - // requiredSlots represents required slots. - // The slot IDs of the latches(keys) that a startTS must acquire before being able to processed. - requiredSlots []int - // acquiredCount represents the number of latches that the transaction has acquired. - // For status is stale, it include the latch whose front is current lock already. - acquiredCount int - // startTS represents current transaction's. - startTS uint64 - // commitTS represents current transaction's. - commitTS uint64 - - wg sync.WaitGroup - isStale bool -} - -// acquireResult is the result type for acquire() -type acquireResult int32 - -const ( - // acquireSuccess is a type constant for acquireResult. - // which means acquired success - acquireSuccess acquireResult = iota - // acquireLocked is a type constant for acquireResult - // which means still locked by other Lock. - acquireLocked - // acquireStale is a type constant for acquireResult - // which means current Lock's startTS is stale. - acquireStale -) - -// IsStale returns whether the status is stale. -func (l *Lock) IsStale() bool { - return l.isStale -} - -func (l *Lock) isLocked() bool { - return !l.isStale && l.acquiredCount != len(l.requiredSlots) -} - -// SetCommitTS sets the lock's commitTS. -func (l *Lock) SetCommitTS(commitTS uint64) { - l.commitTS = commitTS -} - -// Latches which are used for concurrency control. -// Each latch is indexed by a slot's ID, hence the term latch and slot are used in interchangeable, -// but conceptually a latch is a queue, and a slot is an index to the queue -type Latches struct { - slots []latch -} - -type bytesSlice [][]byte - -func (s bytesSlice) Len() int { - return len(s) -} - -func (s bytesSlice) Swap(i, j int) { - s[i], s[j] = s[j], s[i] -} - -func (s bytesSlice) Less(i, j int) bool { - return bytes.Compare(s[i], s[j]) < 0 -} - -// NewLatches create a Latches with fixed length, -// the size will be rounded up to the power of 2. -func NewLatches(size uint) *Latches { - powerOfTwoSize := 1 << uint32(bits.Len32(uint32(size-1))) - slots := make([]latch, powerOfTwoSize) - return &Latches{ - slots: slots, - } -} - -// genLock generates Lock for the transaction with startTS and keys. -func (latches *Latches) genLock(startTS uint64, keys [][]byte) *Lock { - sort.Sort(bytesSlice(keys)) - return &Lock{ - keys: keys, - requiredSlots: latches.genSlotIDs(keys), - acquiredCount: 0, - startTS: startTS, - } -} - -func (latches *Latches) genSlotIDs(keys [][]byte) []int { - slots := make([]int, 0, len(keys)) - for _, key := range keys { - slots = append(slots, latches.slotID(key)) - } - return slots -} - -// slotID return slotID for current key. -func (latches *Latches) slotID(key []byte) int { - return int(murmur3.Sum32(key)) & (len(latches.slots) - 1) -} - -// acquire tries to acquire the lock for a transaction. -func (latches *Latches) acquire(lock *Lock) acquireResult { - if lock.IsStale() { - return acquireStale - } - for lock.acquiredCount < len(lock.requiredSlots) { - status := latches.acquireSlot(lock) - if status != acquireSuccess { - return status - } - } - return acquireSuccess -} - -// release releases all latches owned by the `lock` and returns the wakeup list. -// Preconditions: the caller must ensure the transaction's status is not locked. -func (latches *Latches) release(lock *Lock, wakeupList []*Lock) []*Lock { - wakeupList = wakeupList[:0] - for lock.acquiredCount > 0 { - if nextLock := latches.releaseSlot(lock); nextLock != nil { - wakeupList = append(wakeupList, nextLock) - } - } - return wakeupList -} - -func (latches *Latches) releaseSlot(lock *Lock) (nextLock *Lock) { - key := lock.keys[lock.acquiredCount-1] - slotID := lock.requiredSlots[lock.acquiredCount-1] - latch := &latches.slots[slotID] - lock.acquiredCount-- - latch.Lock() - defer latch.Unlock() - - find := findNode(latch.queue, key) - if find.value != lock { - panic("releaseSlot wrong") - } - find.maxCommitTS = mathutil.MaxUint64(find.maxCommitTS, lock.commitTS) - find.value = nil - // Make a copy of the key, so latch does not reference the transaction's memory. - // If we do not do it, transaction memory can't be recycle by GC and there will - // be a leak. - copyKey := make([]byte, len(find.key)) - copy(copyKey, find.key) - find.key = copyKey - if len(latch.waiting) == 0 { - return nil - } - - var idx int - for idx = 0; idx < len(latch.waiting); idx++ { - waiting := latch.waiting[idx] - if bytes.Equal(waiting.keys[waiting.acquiredCount], key) { - break - } - } - // Wake up the first one in waiting queue. - if idx < len(latch.waiting) { - nextLock = latch.waiting[idx] - // Delete element latch.waiting[idx] from the array. - copy(latch.waiting[idx:], latch.waiting[idx+1:]) - latch.waiting[len(latch.waiting)-1] = nil - latch.waiting = latch.waiting[:len(latch.waiting)-1] - - if find.maxCommitTS > nextLock.startTS { - find.value = nextLock - nextLock.acquiredCount++ - nextLock.isStale = true - } - } - - return -} - -func (latches *Latches) acquireSlot(lock *Lock) acquireResult { - key := lock.keys[lock.acquiredCount] - slotID := lock.requiredSlots[lock.acquiredCount] - latch := &latches.slots[slotID] - latch.Lock() - defer latch.Unlock() - - // Try to recycle to limit the memory usage. - if latch.count >= latchListCount { - latch.recycle(lock.startTS) - } - - find := findNode(latch.queue, key) - if find == nil { - tmp := &node{ - slotID: slotID, - key: key, - value: lock, - } - tmp.next = latch.queue - latch.queue = tmp - latch.count++ - - lock.acquiredCount++ - return acquireSuccess - } - - if find.maxCommitTS > lock.startTS { - lock.isStale = true - return acquireStale - } - - if find.value == nil { - find.value = lock - lock.acquiredCount++ - return acquireSuccess - } - - // Push the current transaction into waitingQueue. - latch.waiting = append(latch.waiting, lock) - return acquireLocked -} - -// recycle is not thread safe, the latch should acquire its lock before executing this function. -func (l *latch) recycle(currentTS uint64) int { - total := 0 - fakeHead := node{next: l.queue} - prev := &fakeHead - for curr := prev.next; curr != nil; curr = curr.next { - if tsoSub(currentTS, curr.maxCommitTS) >= expireDuration && curr.value == nil { - l.count-- - prev.next = curr.next - total++ - } else { - prev = curr - } - } - l.queue = fakeHead.next - return total -} - -func (latches *Latches) recycle(currentTS uint64) { - total := 0 - for i := 0; i < len(latches.slots); i++ { - latch := &latches.slots[i] - latch.Lock() - total += latch.recycle(currentTS) - latch.Unlock() - } - logutil.BgLogger().Debug("recycle", - zap.Time("start at", time.Now()), - zap.Int("count", total)) -} - -func findNode(list *node, key []byte) *node { - for n := list; n != nil; n = n.next { - if bytes.Equal(n.key, key) { - return n - } - } - return nil -} diff --git a/store/tikv/latch/latch_test.go b/store/tikv/latch/latch_test.go deleted file mode 100644 index ce53794d44f12..0000000000000 --- a/store/tikv/latch/latch_test.go +++ /dev/null @@ -1,152 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package latch - -import ( - "sync/atomic" - "testing" - "time" - - . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/tikv/oracle" -) - -func TestT(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&testLatchSuite{}) - -var baseTso uint64 - -type testLatchSuite struct { - latches *Latches -} - -func (s *testLatchSuite) SetUpTest(c *C) { - s.latches = NewLatches(256) -} - -func (s *testLatchSuite) newLock(keys [][]byte) (startTS uint64, lock *Lock) { - startTS = getTso() - lock = s.latches.genLock(startTS, keys) - return -} - -func getTso() uint64 { - return atomic.AddUint64(&baseTso, uint64(1)) -} - -func (s *testLatchSuite) TestWakeUp(c *C) { - keysA := [][]byte{ - []byte("a"), []byte("b"), []byte("c")} - _, lockA := s.newLock(keysA) - - keysB := [][]byte{[]byte("d"), []byte("e"), []byte("a"), []byte("c")} - startTSB, lockB := s.newLock(keysB) - - // A acquire lock success. - result := s.latches.acquire(lockA) - c.Assert(result, Equals, acquireSuccess) - - // B acquire lock failed. - result = s.latches.acquire(lockB) - c.Assert(result, Equals, acquireLocked) - - // A release lock, and get wakeup list. - commitTSA := getTso() - wakeupList := make([]*Lock, 0) - lockA.SetCommitTS(commitTSA) - wakeupList = s.latches.release(lockA, wakeupList) - c.Assert(wakeupList[0].startTS, Equals, startTSB) - - // B acquire failed since startTSB has stale for some keys. - result = s.latches.acquire(lockB) - c.Assert(result, Equals, acquireStale) - - // B release lock since it received a stale. - wakeupList = s.latches.release(lockB, wakeupList) - c.Assert(wakeupList, HasLen, 0) - - // B restart:get a new startTS. - startTSB = getTso() - lockB = s.latches.genLock(startTSB, keysB) - result = s.latches.acquire(lockB) - c.Assert(result, Equals, acquireSuccess) -} - -func (s *testLatchSuite) TestFirstAcquireFailedWithStale(c *C) { - keys := [][]byte{ - []byte("a"), []byte("b"), []byte("c")} - _, lockA := s.newLock(keys) - startTSB, lockB := s.newLock(keys) - // acquire lockA success - result := s.latches.acquire(lockA) - c.Assert(result, Equals, acquireSuccess) - // release lockA - commitTSA := getTso() - wakeupList := make([]*Lock, 0) - lockA.SetCommitTS(commitTSA) - s.latches.release(lockA, wakeupList) - - c.Assert(commitTSA, Greater, startTSB) - // acquire lockB first time, should be failed with stale since commitTSA > startTSB - result = s.latches.acquire(lockB) - c.Assert(result, Equals, acquireStale) - s.latches.release(lockB, wakeupList) -} - -func (s *testLatchSuite) TestRecycle(c *C) { - latches := NewLatches(8) - now := time.Now() - startTS := oracle.GoTimeToTS(now) - lock := latches.genLock(startTS, [][]byte{ - []byte("a"), []byte("b"), - }) - lock1 := latches.genLock(startTS, [][]byte{ - []byte("b"), []byte("c"), - }) - c.Assert(latches.acquire(lock), Equals, acquireSuccess) - c.Assert(latches.acquire(lock1), Equals, acquireLocked) - lock.SetCommitTS(startTS + 1) - var wakeupList []*Lock - latches.release(lock, wakeupList) - // Release lock will grant latch to lock1 automatically, - // so release lock1 is called here. - latches.release(lock1, wakeupList) - - lock2 := latches.genLock(startTS+3, [][]byte{ - []byte("b"), []byte("c"), - }) - c.Assert(latches.acquire(lock2), Equals, acquireSuccess) - wakeupList = wakeupList[:0] - latches.release(lock2, wakeupList) - - allEmpty := true - for i := 0; i < len(latches.slots); i++ { - latch := &latches.slots[i] - if latch.queue != nil { - allEmpty = false - } - } - c.Assert(allEmpty, IsFalse) - - currentTS := oracle.GoTimeToTS(now.Add(expireDuration)) + 3 - latches.recycle(currentTS) - - for i := 0; i < len(latches.slots); i++ { - latch := &latches.slots[i] - c.Assert(latch.queue, IsNil) - } -} diff --git a/store/tikv/latch/scheduler.go b/store/tikv/latch/scheduler.go deleted file mode 100644 index cbf2cb8157dfa..0000000000000 --- a/store/tikv/latch/scheduler.go +++ /dev/null @@ -1,120 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package latch - -import ( - "sync" - "time" - - "github.com/pingcap/tidb/store/tikv/oracle" -) - -const lockChanSize = 100 - -// LatchesScheduler is used to schedule latches for transactions. -type LatchesScheduler struct { - latches *Latches - unlockCh chan *Lock - closed bool - lastRecycleTime uint64 - sync.RWMutex -} - -// NewScheduler create the LatchesScheduler. -func NewScheduler(size uint) *LatchesScheduler { - latches := NewLatches(size) - unlockCh := make(chan *Lock, lockChanSize) - scheduler := &LatchesScheduler{ - latches: latches, - unlockCh: unlockCh, - closed: false, - } - go scheduler.run() - return scheduler -} - -const expireDuration = 2 * time.Minute -const checkInterval = 1 * time.Minute -const checkCounter = 50000 -const latchListCount = 5 - -func (scheduler *LatchesScheduler) run() { - var counter int - wakeupList := make([]*Lock, 0) - for lock := range scheduler.unlockCh { - wakeupList = scheduler.latches.release(lock, wakeupList) - if len(wakeupList) > 0 { - scheduler.wakeup(wakeupList) - } - - if lock.commitTS > lock.startTS { - currentTS := lock.commitTS - elapsed := tsoSub(currentTS, scheduler.lastRecycleTime) - if elapsed > checkInterval || counter > checkCounter { - go scheduler.latches.recycle(lock.commitTS) - scheduler.lastRecycleTime = currentTS - counter = 0 - } - } - counter++ - } -} - -func (scheduler *LatchesScheduler) wakeup(wakeupList []*Lock) { - for _, lock := range wakeupList { - if scheduler.latches.acquire(lock) != acquireLocked { - lock.wg.Done() - } - } -} - -// Close closes LatchesScheduler. -func (scheduler *LatchesScheduler) Close() { - scheduler.RWMutex.Lock() - defer scheduler.RWMutex.Unlock() - if !scheduler.closed { - close(scheduler.unlockCh) - scheduler.closed = true - } -} - -// Lock acquire the lock for transaction with startTS and keys. The caller goroutine -// would be blocked if the lock can't be obtained now. When this function returns, -// the lock state would be either success or stale(call lock.IsStale) -func (scheduler *LatchesScheduler) Lock(startTS uint64, keys [][]byte) *Lock { - lock := scheduler.latches.genLock(startTS, keys) - lock.wg.Add(1) - if scheduler.latches.acquire(lock) == acquireLocked { - lock.wg.Wait() - } - if lock.isLocked() { - panic("should never run here") - } - return lock -} - -// UnLock unlocks a lock. -func (scheduler *LatchesScheduler) UnLock(lock *Lock) { - scheduler.RLock() - defer scheduler.RUnlock() - if !scheduler.closed { - scheduler.unlockCh <- lock - } -} - -func tsoSub(ts1, ts2 uint64) time.Duration { - t1 := oracle.GetTimeFromTS(ts1) - t2 := oracle.GetTimeFromTS(ts2) - return t1.Sub(t2) -} diff --git a/store/tikv/latch/scheduler_test.go b/store/tikv/latch/scheduler_test.go deleted file mode 100644 index 30f8a0d21b0bc..0000000000000 --- a/store/tikv/latch/scheduler_test.go +++ /dev/null @@ -1,94 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package latch - -import ( - "bytes" - "math/rand" - "sync" - "time" - - . "github.com/pingcap/check" -) - -var _ = Suite(&testSchedulerSuite{}) - -type testSchedulerSuite struct { -} - -func (s *testSchedulerSuite) SetUpTest(c *C) { -} - -func (s *testSchedulerSuite) TestWithConcurrency(c *C) { - sched := NewScheduler(7) - defer sched.Close() - rand.Seed(time.Now().Unix()) - - ch := make(chan [][]byte, 100) - const workerCount = 10 - var wg sync.WaitGroup - wg.Add(workerCount) - for i := 0; i < workerCount; i++ { - go func(ch <-chan [][]byte, wg *sync.WaitGroup) { - for txn := range ch { - lock := sched.Lock(getTso(), txn) - if lock.IsStale() { - // Should restart the transaction or return error - } else { - lock.SetCommitTS(getTso()) - // Do 2pc - } - sched.UnLock(lock) - } - wg.Done() - }(ch, &wg) - } - - for i := 0; i < 999; i++ { - ch <- generate() - } - close(ch) - - wg.Wait() -} - -// generate generates something like: -// {[]byte("a"), []byte("b"), []byte("c")} -// {[]byte("a"), []byte("d"), []byte("e"), []byte("f")} -// {[]byte("e"), []byte("f"), []byte("g"), []byte("h")} -// The data should not repeat in the sequence. -func generate() [][]byte { - table := []byte{'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h'} - ret := make([][]byte, 0, 5) - chance := []int{100, 60, 40, 20} - for i := 0; i < len(chance); i++ { - needMore := rand.Intn(100) < chance[i] - if needMore { - randBytes := []byte{table[rand.Intn(len(table))]} - if !contains(randBytes, ret) { - ret = append(ret, randBytes) - } - } - } - return ret -} - -func contains(x []byte, set [][]byte) bool { - for _, y := range set { - if bytes.Equal(x, y) { - return true - } - } - return false -} diff --git a/store/tikv/locate/accessmode.go b/store/tikv/locate/accessmode.go deleted file mode 100644 index 62fe3f3b8b2e4..0000000000000 --- a/store/tikv/locate/accessmode.go +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package locate - -import ( - "fmt" -) - -// accessMode uses to index stores for different region cache access requirements. -type accessMode int - -const ( - // tiKVOnly indicates stores list that use for TiKv access(include both leader request and follower read). - tiKVOnly accessMode = iota - // tiFlashOnly indicates stores list that use for TiFlash request. - tiFlashOnly - // numAccessMode reserved to keep max access mode value. - numAccessMode -) - -func (a accessMode) String() string { - switch a { - case tiKVOnly: - return "TiKvOnly" - case tiFlashOnly: - return "TiFlashOnly" - default: - return fmt.Sprintf("%d", a) - } -} diff --git a/store/tikv/locate/pd_codec.go b/store/tikv/locate/pd_codec.go deleted file mode 100644 index 5c0a5224712ef..0000000000000 --- a/store/tikv/locate/pd_codec.go +++ /dev/null @@ -1,130 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package locate - -import ( - "context" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/tikv/util/codec" - pd "github.com/tikv/pd/client" -) - -// CodecPDClient wraps a PD Client to decode the encoded keys in region meta. -type CodecPDClient struct { - pd.Client -} - -// NewCodeCPDClient creates a CodecPDClient. -func NewCodeCPDClient(client pd.Client) *CodecPDClient { - return &CodecPDClient{client} -} - -// GetRegion encodes the key before send requests to pd-server and decodes the -// returned StartKey && EndKey from pd-server. -func (c *CodecPDClient) GetRegion(ctx context.Context, key []byte) (*pd.Region, error) { - encodedKey := codec.EncodeBytes([]byte(nil), key) - region, err := c.Client.GetRegion(ctx, encodedKey) - return processRegionResult(region, err) -} - -// GetPrevRegion encodes the key before send requests to pd-server and decodes the -// returned StartKey && EndKey from pd-server. -func (c *CodecPDClient) GetPrevRegion(ctx context.Context, key []byte) (*pd.Region, error) { - encodedKey := codec.EncodeBytes([]byte(nil), key) - region, err := c.Client.GetPrevRegion(ctx, encodedKey) - return processRegionResult(region, err) -} - -// GetRegionByID encodes the key before send requests to pd-server and decodes the -// returned StartKey && EndKey from pd-server. -func (c *CodecPDClient) GetRegionByID(ctx context.Context, regionID uint64) (*pd.Region, error) { - region, err := c.Client.GetRegionByID(ctx, regionID) - return processRegionResult(region, err) -} - -// ScanRegions encodes the key before send requests to pd-server and decodes the -// returned StartKey && EndKey from pd-server. -func (c *CodecPDClient) ScanRegions(ctx context.Context, startKey []byte, endKey []byte, limit int) ([]*pd.Region, error) { - startKey = codec.EncodeBytes([]byte(nil), startKey) - if len(endKey) > 0 { - endKey = codec.EncodeBytes([]byte(nil), endKey) - } - - regions, err := c.Client.ScanRegions(ctx, startKey, endKey, limit) - if err != nil { - return nil, errors.Trace(err) - } - for _, region := range regions { - if region != nil { - err = decodeRegionMetaKeyInPlace(region.Meta) - if err != nil { - return nil, errors.Trace(err) - } - } - } - return regions, nil -} - -func processRegionResult(region *pd.Region, err error) (*pd.Region, error) { - if err != nil { - return nil, errors.Trace(err) - } - if region == nil || region.Meta == nil { - return nil, nil - } - err = decodeRegionMetaKeyInPlace(region.Meta) - if err != nil { - return nil, errors.Trace(err) - } - return region, nil -} - -func decodeRegionMetaKeyInPlace(r *metapb.Region) error { - if len(r.StartKey) != 0 { - _, decoded, err := codec.DecodeBytes(r.StartKey, nil) - if err != nil { - return errors.Trace(err) - } - r.StartKey = decoded - } - if len(r.EndKey) != 0 { - _, decoded, err := codec.DecodeBytes(r.EndKey, nil) - if err != nil { - return errors.Trace(err) - } - r.EndKey = decoded - } - return nil -} - -func decodeRegionMetaKeyWithShallowCopy(r *metapb.Region) (*metapb.Region, error) { - nr := *r - if len(r.StartKey) != 0 { - _, decoded, err := codec.DecodeBytes(r.StartKey, nil) - if err != nil { - return nil, errors.Trace(err) - } - nr.StartKey = decoded - } - if len(r.EndKey) != 0 { - _, decoded, err := codec.DecodeBytes(r.EndKey, nil) - if err != nil { - return nil, errors.Trace(err) - } - nr.EndKey = decoded - } - return &nr, nil -} diff --git a/store/tikv/locate/region_cache.go b/store/tikv/locate/region_cache.go deleted file mode 100644 index ee9172e1d6ba1..0000000000000 --- a/store/tikv/locate/region_cache.go +++ /dev/null @@ -1,2312 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package locate - -import ( - "bytes" - "context" - "fmt" - "math/rand" - "strings" - "sync" - "sync/atomic" - "time" - "unsafe" - - "github.com/gogo/protobuf/proto" - "github.com/google/btree" - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - pd "github.com/tikv/pd/client" - atomic2 "go.uber.org/atomic" - "go.uber.org/zap" - "golang.org/x/sync/singleflight" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" - "google.golang.org/grpc/credentials" - healthpb "google.golang.org/grpc/health/grpc_health_v1" - "google.golang.org/grpc/keepalive" -) - -const ( - btreeDegree = 32 - invalidatedLastAccessTime = -1 - defaultRegionsPerBatch = 128 -) - -// regionCacheTTLSec is the max idle time for regions in the region cache. -var regionCacheTTLSec int64 = 600 - -// SetRegionCacheTTLSec sets regionCacheTTLSec to t. -func SetRegionCacheTTLSec(t int64) { - regionCacheTTLSec = t -} - -const ( - updated int32 = iota // region is updated and no need to reload. - needSync // need sync new region info. -) - -// InvalidReason is the reason why a cached region is invalidated. -// The region cache may take different strategies to handle different reasons. -// For example, when a cached region is invalidated due to no leader, region cache -// will always access to a different peer. -type InvalidReason int32 - -const ( - // Ok indicates the cached region is valid - Ok InvalidReason = iota - // NoLeader indicates it's invalidated due to no leader - NoLeader - // RegionNotFound indicates it's invalidated due to region not found in the store - RegionNotFound - // EpochNotMatch indicates it's invalidated due to epoch not match - EpochNotMatch - // StoreNotFound indicates it's invalidated due to store not found in PD - StoreNotFound - // Other indicates it's invalidated due to other reasons, e.g., the store - // is removed from the cluster, fail to send requests to the store. - Other -) - -// Region presents kv region -type Region struct { - meta *metapb.Region // raw region meta from PD immutable after init - store unsafe.Pointer // point to region store info, see RegionStore - syncFlag int32 // region need be sync in next turn - lastAccess int64 // last region access time, see checkRegionCacheTTL - invalidReason InvalidReason // the reason why the region is invalidated -} - -// AccessIndex represent the index for accessIndex array -type AccessIndex int - -// regionStore represents region stores info -// it will be store as unsafe.Pointer and be load at once -type regionStore struct { - workTiKVIdx AccessIndex // point to current work peer in meta.Peers and work store in stores(same idx) for tikv peer - proxyTiKVIdx AccessIndex // point to the tikv peer that can forward requests to the leader. -1 means not using proxy - workTiFlashIdx int32 // point to current work peer in meta.Peers and work store in stores(same idx) for tiflash peer - stores []*Store // stores in this region - storeEpochs []uint32 // snapshots of store's epoch, need reload when `storeEpochs[curr] != stores[cur].fail` - accessIndex [numAccessMode][]int // AccessMode => idx in stores -} - -func (r *regionStore) accessStore(mode accessMode, idx AccessIndex) (int, *Store) { - sidx := r.accessIndex[mode][idx] - return sidx, r.stores[sidx] -} - -func (r *regionStore) getAccessIndex(mode accessMode, store *Store) AccessIndex { - for index, sidx := range r.accessIndex[mode] { - if r.stores[sidx].storeID == store.storeID { - return AccessIndex(index) - } - } - return -1 -} - -func (r *regionStore) accessStoreNum(mode accessMode) int { - return len(r.accessIndex[mode]) -} - -// clone clones region store struct. -func (r *regionStore) clone() *regionStore { - storeEpochs := make([]uint32, len(r.stores)) - rs := ®ionStore{ - workTiFlashIdx: r.workTiFlashIdx, - proxyTiKVIdx: r.proxyTiKVIdx, - workTiKVIdx: r.workTiKVIdx, - stores: r.stores, - storeEpochs: storeEpochs, - } - copy(storeEpochs, r.storeEpochs) - for i := 0; i < int(numAccessMode); i++ { - rs.accessIndex[i] = make([]int, len(r.accessIndex[i])) - copy(rs.accessIndex[i], r.accessIndex[i]) - } - return rs -} - -// return next follower store's index -func (r *regionStore) follower(seed uint32, op *storeSelectorOp) AccessIndex { - l := uint32(r.accessStoreNum(tiKVOnly)) - if l <= 1 { - return r.workTiKVIdx - } - - for retry := l - 1; retry > 0; retry-- { - followerIdx := AccessIndex(seed % (l - 1)) - if followerIdx >= r.workTiKVIdx { - followerIdx++ - } - storeIdx, s := r.accessStore(tiKVOnly, followerIdx) - if r.storeEpochs[storeIdx] == atomic.LoadUint32(&s.epoch) && r.filterStoreCandidate(followerIdx, op) { - return followerIdx - } - seed++ - } - return r.workTiKVIdx -} - -// return next leader or follower store's index -func (r *regionStore) kvPeer(seed uint32, op *storeSelectorOp) AccessIndex { - if op.leaderOnly { - return r.workTiKVIdx - } - candidates := make([]AccessIndex, 0, r.accessStoreNum(tiKVOnly)) - for i := 0; i < r.accessStoreNum(tiKVOnly); i++ { - accessIdx := AccessIndex(i) - storeIdx, s := r.accessStore(tiKVOnly, accessIdx) - if r.storeEpochs[storeIdx] != atomic.LoadUint32(&s.epoch) || !r.filterStoreCandidate(accessIdx, op) { - continue - } - candidates = append(candidates, accessIdx) - } - // If there is no candidates, send to current workTiKVIdx which generally is the leader. - if len(candidates) == 0 { - return r.workTiKVIdx - } - return candidates[seed%uint32(len(candidates))] -} - -func (r *regionStore) filterStoreCandidate(aidx AccessIndex, op *storeSelectorOp) bool { - _, s := r.accessStore(tiKVOnly, aidx) - // filter label unmatched store - return s.IsLabelsMatch(op.labels) -} - -// init initializes region after constructed. -func (r *Region) init(bo *retry.Backoffer, c *RegionCache) error { - // region store pull used store from global store map - // to avoid acquire storeMu in later access. - rs := ®ionStore{ - workTiKVIdx: 0, - proxyTiKVIdx: -1, - workTiFlashIdx: 0, - stores: make([]*Store, 0, len(r.meta.Peers)), - storeEpochs: make([]uint32, 0, len(r.meta.Peers)), - } - availablePeers := r.meta.GetPeers()[:0] - for _, p := range r.meta.Peers { - c.storeMu.RLock() - store, exists := c.storeMu.stores[p.StoreId] - c.storeMu.RUnlock() - if !exists { - store = c.getStoreByStoreID(p.StoreId) - } - addr, err := store.initResolve(bo, c) - if err != nil { - return err - } - // Filter the peer on a tombstone store. - if addr == "" { - continue - } - availablePeers = append(availablePeers, p) - switch store.storeType { - case tikvrpc.TiKV: - rs.accessIndex[tiKVOnly] = append(rs.accessIndex[tiKVOnly], len(rs.stores)) - case tikvrpc.TiFlash: - rs.accessIndex[tiFlashOnly] = append(rs.accessIndex[tiFlashOnly], len(rs.stores)) - } - rs.stores = append(rs.stores, store) - rs.storeEpochs = append(rs.storeEpochs, atomic.LoadUint32(&store.epoch)) - } - // TODO(youjiali1995): It's possible the region info in PD is stale for now but it can recover. - // Maybe we need backoff here. - if len(availablePeers) == 0 { - return errors.Errorf("no available peers, region: {%v}", r.meta) - } - r.meta.Peers = availablePeers - - atomic.StorePointer(&r.store, unsafe.Pointer(rs)) - - // mark region has been init accessed. - r.lastAccess = time.Now().Unix() - return nil -} - -func (r *Region) getStore() (store *regionStore) { - store = (*regionStore)(atomic.LoadPointer(&r.store)) - return -} - -func (r *Region) compareAndSwapStore(oldStore, newStore *regionStore) bool { - return atomic.CompareAndSwapPointer(&r.store, unsafe.Pointer(oldStore), unsafe.Pointer(newStore)) -} - -func (r *Region) checkRegionCacheTTL(ts int64) bool { - // Only consider use percentage on this failpoint, for example, "2%return" - if _, err := util.EvalFailpoint("invalidateRegionCache"); err == nil { - r.invalidate(Other) - } - for { - lastAccess := atomic.LoadInt64(&r.lastAccess) - if ts-lastAccess > regionCacheTTLSec { - return false - } - if atomic.CompareAndSwapInt64(&r.lastAccess, lastAccess, ts) { - return true - } - } -} - -// invalidate invalidates a region, next time it will got null result. -func (r *Region) invalidate(reason InvalidReason) { - metrics.RegionCacheCounterWithInvalidateRegionFromCacheOK.Inc() - atomic.StoreInt32((*int32)(&r.invalidReason), int32(reason)) - atomic.StoreInt64(&r.lastAccess, invalidatedLastAccessTime) -} - -// scheduleReload schedules reload region request in next LocateKey. -func (r *Region) scheduleReload() { - oldValue := atomic.LoadInt32(&r.syncFlag) - if oldValue != updated { - return - } - atomic.CompareAndSwapInt32(&r.syncFlag, oldValue, needSync) -} - -// checkNeedReloadAndMarkUpdated returns whether the region need reload and marks the region to be updated. -func (r *Region) checkNeedReloadAndMarkUpdated() bool { - oldValue := atomic.LoadInt32(&r.syncFlag) - if oldValue == updated { - return false - } - return atomic.CompareAndSwapInt32(&r.syncFlag, oldValue, updated) -} - -func (r *Region) checkNeedReload() bool { - v := atomic.LoadInt32(&r.syncFlag) - return v != updated -} - -func (r *Region) isValid() bool { - return r != nil && !r.checkNeedReload() && r.checkRegionCacheTTL(time.Now().Unix()) -} - -// RegionCache caches Regions loaded from PD. -type RegionCache struct { - pdClient pd.Client - enableForwarding bool - - mu struct { - sync.RWMutex // mutex protect cached region - regions map[RegionVerID]*Region // cached regions are organized as regionVerID to region ref mapping - latestVersions map[uint64]RegionVerID // cache the map from regionID to its latest RegionVerID - sorted *btree.BTree // cache regions are organized as sorted key to region ref mapping - } - storeMu struct { - sync.RWMutex - stores map[uint64]*Store - } - notifyCheckCh chan struct{} - closeCh chan struct{} - - testingKnobs struct { - // Replace the requestLiveness function for test purpose. Note that in unit tests, if this is not set, - // requestLiveness always returns unreachable. - mockRequestLiveness func(s *Store, bo *retry.Backoffer) livenessState - } -} - -// NewRegionCache creates a RegionCache. -func NewRegionCache(pdClient pd.Client) *RegionCache { - c := &RegionCache{ - pdClient: pdClient, - } - c.mu.regions = make(map[RegionVerID]*Region) - c.mu.latestVersions = make(map[uint64]RegionVerID) - c.mu.sorted = btree.New(btreeDegree) - c.storeMu.stores = make(map[uint64]*Store) - c.notifyCheckCh = make(chan struct{}, 1) - c.closeCh = make(chan struct{}) - interval := config.GetGlobalConfig().StoresRefreshInterval - go c.asyncCheckAndResolveLoop(time.Duration(interval) * time.Second) - c.enableForwarding = config.GetGlobalConfig().EnableForwarding - return c -} - -// clear clears all cached data in the RegionCache. It's only used in tests. -func (c *RegionCache) clear() { - c.mu.Lock() - c.mu.regions = make(map[RegionVerID]*Region) - c.mu.latestVersions = make(map[uint64]RegionVerID) - c.mu.sorted = btree.New(btreeDegree) - c.mu.Unlock() - c.storeMu.Lock() - c.storeMu.stores = make(map[uint64]*Store) - c.storeMu.Unlock() -} - -// Close releases region cache's resource. -func (c *RegionCache) Close() { - close(c.closeCh) -} - -// asyncCheckAndResolveLoop with -func (c *RegionCache) asyncCheckAndResolveLoop(interval time.Duration) { - ticker := time.NewTicker(interval) - defer ticker.Stop() - var needCheckStores []*Store - for { - needCheckStores = needCheckStores[:0] - select { - case <-c.closeCh: - return - case <-c.notifyCheckCh: - c.checkAndResolve(needCheckStores, func(s *Store) bool { - return s.getResolveState() == needCheck - }) - case <-ticker.C: - // refresh store to update labels. - c.checkAndResolve(needCheckStores, func(s *Store) bool { - state := s.getResolveState() - // Only valid stores should be reResolved. In fact, it's impossible - // there's a deleted store in the stores map which guaranteed by reReslve(). - return state != unresolved && state != tombstone && state != deleted - }) - } - } -} - -// checkAndResolve checks and resolve addr of failed stores. -// this method isn't thread-safe and only be used by one goroutine. -func (c *RegionCache) checkAndResolve(needCheckStores []*Store, needCheck func(*Store) bool) { - defer func() { - r := recover() - if r != nil { - logutil.BgLogger().Error("panic in the checkAndResolve goroutine", - zap.Reflect("r", r), - zap.Stack("stack trace")) - } - }() - - c.storeMu.RLock() - for _, store := range c.storeMu.stores { - if needCheck(store) { - needCheckStores = append(needCheckStores, store) - } - } - c.storeMu.RUnlock() - - for _, store := range needCheckStores { - _, err := store.reResolve(c) - terror.Log(err) - } -} - -// SetRegionCacheStore is used to set a store in region cache, for testing only -func (c *RegionCache) SetRegionCacheStore(id uint64, storeType tikvrpc.EndpointType, state uint64, labels []*metapb.StoreLabel) { - c.storeMu.Lock() - defer c.storeMu.Unlock() - c.storeMu.stores[id] = &Store{ - storeID: id, - storeType: storeType, - state: state, - labels: labels, - } -} - -// SetPDClient replaces pd client,for testing only -func (c *RegionCache) SetPDClient(client pd.Client) { - c.pdClient = client -} - -// RPCContext contains data that is needed to send RPC to a region. -type RPCContext struct { - Region RegionVerID - Meta *metapb.Region - Peer *metapb.Peer - AccessIdx AccessIndex - Store *Store - Addr string - AccessMode accessMode - ProxyStore *Store // nil means proxy is not used - ProxyAccessIdx AccessIndex // valid when ProxyStore is not nil - ProxyAddr string // valid when ProxyStore is not nil - TiKVNum int // Number of TiKV nodes among the region's peers. Assuming non-TiKV peers are all TiFlash peers. - - tryTimes int -} - -func (c *RPCContext) String() string { - var runStoreType string - if c.Store != nil { - runStoreType = c.Store.storeType.Name() - } - res := fmt.Sprintf("region ID: %d, meta: %s, peer: %s, addr: %s, idx: %d, reqStoreType: %s, runStoreType: %s", - c.Region.GetID(), c.Meta, c.Peer, c.Addr, c.AccessIdx, c.AccessMode, runStoreType) - if c.ProxyStore != nil { - res += fmt.Sprintf(", proxy store id: %d, proxy addr: %s, proxy idx: %d", c.ProxyStore.storeID, c.ProxyAddr, c.ProxyAccessIdx) - } - return res -} - -type storeSelectorOp struct { - leaderOnly bool - labels []*metapb.StoreLabel -} - -// StoreSelectorOption configures storeSelectorOp. -type StoreSelectorOption func(*storeSelectorOp) - -// WithMatchLabels indicates selecting stores with matched labels. -func WithMatchLabels(labels []*metapb.StoreLabel) StoreSelectorOption { - return func(op *storeSelectorOp) { - op.labels = append(op.labels, labels...) - } -} - -// WithLeaderOnly indicates selecting stores with leader only. -func WithLeaderOnly() StoreSelectorOption { - return func(op *storeSelectorOp) { - op.leaderOnly = true - } -} - -// GetTiKVRPCContext returns RPCContext for a region. If it returns nil, the region -// must be out of date and already dropped from cache. -func (c *RegionCache) GetTiKVRPCContext(bo *retry.Backoffer, id RegionVerID, replicaRead kv.ReplicaReadType, followerStoreSeed uint32, opts ...StoreSelectorOption) (*RPCContext, error) { - ts := time.Now().Unix() - - cachedRegion := c.GetCachedRegionWithRLock(id) - if cachedRegion == nil { - return nil, nil - } - - if cachedRegion.checkNeedReload() { - return nil, nil - } - - if !cachedRegion.checkRegionCacheTTL(ts) { - return nil, nil - } - - regionStore := cachedRegion.getStore() - var ( - store *Store - peer *metapb.Peer - storeIdx int - accessIdx AccessIndex - ) - options := &storeSelectorOp{} - for _, op := range opts { - op(options) - } - if val, err := util.EvalFailpoint("assertStoreLabels"); err == nil { - if len(opts) > 0 { - kv := strings.Split(val.(string), "_") - for _, label := range options.labels { - if label.Key == kv[0] && label.Value != kv[1] { - panic(fmt.Sprintf("StoreSelectorOption's label %v is not %v", kv[0], kv[1])) - } - } - } - } - isLeaderReq := false - switch replicaRead { - case kv.ReplicaReadFollower: - store, peer, accessIdx, storeIdx = cachedRegion.FollowerStorePeer(regionStore, followerStoreSeed, options) - case kv.ReplicaReadMixed: - store, peer, accessIdx, storeIdx = cachedRegion.AnyStorePeer(regionStore, followerStoreSeed, options) - default: - isLeaderReq = true - store, peer, accessIdx, storeIdx = cachedRegion.WorkStorePeer(regionStore) - } - addr, err := c.getStoreAddr(bo, cachedRegion, store) - if err != nil { - return nil, err - } - // enable by `curl -XPUT -d '1*return("[some-addr]")->return("")' http://host:port/tikvclient/injectWrongStoreAddr` - if val, err := util.EvalFailpoint("injectWrongStoreAddr"); err == nil { - if a, ok := val.(string); ok && len(a) > 0 { - addr = a - } - } - if store == nil || len(addr) == 0 { - // Store not found, region must be out of date. - cachedRegion.invalidate(StoreNotFound) - return nil, nil - } - - storeFailEpoch := atomic.LoadUint32(&store.epoch) - if storeFailEpoch != regionStore.storeEpochs[storeIdx] { - cachedRegion.invalidate(Other) - logutil.BgLogger().Info("invalidate current region, because others failed on same store", - zap.Uint64("region", id.GetID()), - zap.String("store", store.addr)) - return nil, nil - } - - var ( - proxyStore *Store - proxyAddr string - proxyAccessIdx AccessIndex - ) - if c.enableForwarding && isLeaderReq { - if atomic.LoadInt32(&store.needForwarding) == 0 { - regionStore.unsetProxyStoreIfNeeded(cachedRegion) - } else { - proxyStore, proxyAccessIdx, _ = c.getProxyStore(cachedRegion, store, regionStore, accessIdx) - if proxyStore != nil { - proxyAddr, err = c.getStoreAddr(bo, cachedRegion, proxyStore) - if err != nil { - return nil, err - } - } - } - } - - return &RPCContext{ - Region: id, - Meta: cachedRegion.meta, - Peer: peer, - AccessIdx: accessIdx, - Store: store, - Addr: addr, - AccessMode: tiKVOnly, - ProxyStore: proxyStore, - ProxyAccessIdx: proxyAccessIdx, - ProxyAddr: proxyAddr, - TiKVNum: regionStore.accessStoreNum(tiKVOnly), - }, nil -} - -// GetAllValidTiFlashStores returns the store ids of all valid TiFlash stores, the store id of currentStore is always the first one -func (c *RegionCache) GetAllValidTiFlashStores(id RegionVerID, currentStore *Store) []uint64 { - // set the cap to 2 because usually, TiFlash table will have 2 replicas - allStores := make([]uint64, 0, 2) - // make sure currentStore id is always the first in allStores - allStores = append(allStores, currentStore.storeID) - ts := time.Now().Unix() - cachedRegion := c.GetCachedRegionWithRLock(id) - if cachedRegion == nil { - return allStores - } - if !cachedRegion.checkRegionCacheTTL(ts) { - return allStores - } - regionStore := cachedRegion.getStore() - currentIndex := regionStore.getAccessIndex(tiFlashOnly, currentStore) - if currentIndex == -1 { - return allStores - } - for startOffset := 1; startOffset < regionStore.accessStoreNum(tiFlashOnly); startOffset++ { - accessIdx := AccessIndex((int(currentIndex) + startOffset) % regionStore.accessStoreNum(tiFlashOnly)) - storeIdx, store := regionStore.accessStore(tiFlashOnly, accessIdx) - if store.getResolveState() == needCheck { - continue - } - storeFailEpoch := atomic.LoadUint32(&store.epoch) - if storeFailEpoch != regionStore.storeEpochs[storeIdx] { - continue - } - allStores = append(allStores, store.storeID) - } - return allStores -} - -// GetTiFlashRPCContext returns RPCContext for a region must access flash store. If it returns nil, the region -// must be out of date and already dropped from cache or not flash store found. -// `loadBalance` is an option. For MPP and batch cop, it is pointless and might cause try the failed store repeatly. -func (c *RegionCache) GetTiFlashRPCContext(bo *retry.Backoffer, id RegionVerID, loadBalance bool) (*RPCContext, error) { - ts := time.Now().Unix() - - cachedRegion := c.GetCachedRegionWithRLock(id) - if cachedRegion == nil { - return nil, nil - } - if !cachedRegion.checkRegionCacheTTL(ts) { - return nil, nil - } - - regionStore := cachedRegion.getStore() - - // sIdx is for load balance of TiFlash store. - var sIdx int - if loadBalance { - sIdx = int(atomic.AddInt32(®ionStore.workTiFlashIdx, 1)) - } else { - sIdx = int(atomic.LoadInt32(®ionStore.workTiFlashIdx)) - } - for i := 0; i < regionStore.accessStoreNum(tiFlashOnly); i++ { - accessIdx := AccessIndex((sIdx + i) % regionStore.accessStoreNum(tiFlashOnly)) - storeIdx, store := regionStore.accessStore(tiFlashOnly, accessIdx) - addr, err := c.getStoreAddr(bo, cachedRegion, store) - if err != nil { - return nil, err - } - if len(addr) == 0 { - cachedRegion.invalidate(StoreNotFound) - return nil, nil - } - if store.getResolveState() == needCheck { - _, err := store.reResolve(c) - terror.Log(err) - } - atomic.StoreInt32(®ionStore.workTiFlashIdx, int32(accessIdx)) - peer := cachedRegion.meta.Peers[storeIdx] - storeFailEpoch := atomic.LoadUint32(&store.epoch) - if storeFailEpoch != regionStore.storeEpochs[storeIdx] { - cachedRegion.invalidate(Other) - logutil.BgLogger().Info("invalidate current region, because others failed on same store", - zap.Uint64("region", id.GetID()), - zap.String("store", store.addr)) - // TiFlash will always try to find out a valid peer, avoiding to retry too many times. - continue - } - return &RPCContext{ - Region: id, - Meta: cachedRegion.meta, - Peer: peer, - AccessIdx: accessIdx, - Store: store, - Addr: addr, - AccessMode: tiFlashOnly, - TiKVNum: regionStore.accessStoreNum(tiKVOnly), - }, nil - } - - cachedRegion.invalidate(Other) - return nil, nil -} - -// KeyLocation is the region and range that a key is located. -type KeyLocation struct { - Region RegionVerID - StartKey []byte - EndKey []byte -} - -// Contains checks if key is in [StartKey, EndKey). -func (l *KeyLocation) Contains(key []byte) bool { - return bytes.Compare(l.StartKey, key) <= 0 && - (bytes.Compare(key, l.EndKey) < 0 || len(l.EndKey) == 0) -} - -// String implements fmt.Stringer interface. -func (l *KeyLocation) String() string { - return fmt.Sprintf("region %s,startKey:%s,endKey:%s", l.Region.String(), kv.StrKey(l.StartKey), kv.StrKey(l.EndKey)) -} - -// LocateKey searches for the region and range that the key is located. -func (c *RegionCache) LocateKey(bo *retry.Backoffer, key []byte) (*KeyLocation, error) { - r, err := c.findRegionByKey(bo, key, false) - if err != nil { - return nil, err - } - return &KeyLocation{ - Region: r.VerID(), - StartKey: r.StartKey(), - EndKey: r.EndKey(), - }, nil -} - -// LocateEndKey searches for the region and range that the key is located. -// Unlike LocateKey, start key of a region is exclusive and end key is inclusive. -func (c *RegionCache) LocateEndKey(bo *retry.Backoffer, key []byte) (*KeyLocation, error) { - r, err := c.findRegionByKey(bo, key, true) - if err != nil { - return nil, err - } - return &KeyLocation{ - Region: r.VerID(), - StartKey: r.StartKey(), - EndKey: r.EndKey(), - }, nil -} - -func (c *RegionCache) findRegionByKey(bo *retry.Backoffer, key []byte, isEndKey bool) (r *Region, err error) { - r = c.searchCachedRegion(key, isEndKey) - if r == nil { - // load region when it is not exists or expired. - lr, err := c.loadRegion(bo, key, isEndKey) - if err != nil { - // no region data, return error if failure. - return nil, err - } - logutil.Eventf(bo.GetCtx(), "load region %d from pd, due to cache-miss", lr.GetID()) - r = lr - c.mu.Lock() - c.insertRegionToCache(r) - c.mu.Unlock() - } else if r.checkNeedReloadAndMarkUpdated() { - // load region when it be marked as need reload. - lr, err := c.loadRegion(bo, key, isEndKey) - if err != nil { - // ignore error and use old region info. - logutil.Logger(bo.GetCtx()).Error("load region failure", - zap.ByteString("key", key), zap.Error(err)) - } else { - logutil.Eventf(bo.GetCtx(), "load region %d from pd, due to need-reload", lr.GetID()) - r = lr - c.mu.Lock() - c.insertRegionToCache(r) - c.mu.Unlock() - } - } - return r, nil -} - -// OnSendFailForTiFlash handles send request fail logic for tiflash. -func (c *RegionCache) OnSendFailForTiFlash(bo *retry.Backoffer, store *Store, region RegionVerID, prev *metapb.Region, scheduleReload bool, err error) { - - r := c.GetCachedRegionWithRLock(region) - if r == nil { - return - } - - rs := r.getStore() - peersNum := len(r.GetMeta().Peers) - if len(prev.Peers) != peersNum { - logutil.Logger(bo.GetCtx()).Info("retry and refresh current region after send request fail and up/down stores length changed", - zap.Stringer("region", ®ion), - zap.Bool("needReload", scheduleReload), - zap.Reflect("oldPeers", prev.Peers), - zap.Reflect("newPeers", r.GetMeta().Peers), - zap.Error(err)) - return - } - - accessMode := tiFlashOnly - accessIdx := rs.getAccessIndex(accessMode, store) - if accessIdx == -1 { - logutil.Logger(bo.GetCtx()).Warn("can not get access index for region " + region.String()) - return - } - if err != nil { - storeIdx, s := rs.accessStore(accessMode, accessIdx) - c.markRegionNeedBeRefill(s, storeIdx, rs) - } - - // try next peer - rs.switchNextFlashPeer(r, accessIdx) - logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", - zap.Stringer("region", ®ion), - zap.Bool("needReload", scheduleReload), - zap.Error(err)) - - // force reload region when retry all known peers in region. - if scheduleReload { - r.scheduleReload() - } -} - -func (c *RegionCache) markRegionNeedBeRefill(s *Store, storeIdx int, rs *regionStore) int { - incEpochStoreIdx := -1 - // invalidate regions in store. - epoch := rs.storeEpochs[storeIdx] - if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { - logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) - incEpochStoreIdx = storeIdx - metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() - } - // schedule a store addr resolve. - s.markNeedCheck(c.notifyCheckCh) - return incEpochStoreIdx -} - -// OnSendFail handles send request fail logic. -func (c *RegionCache) OnSendFail(bo *retry.Backoffer, ctx *RPCContext, scheduleReload bool, err error) { - metrics.RegionCacheCounterWithSendFail.Inc() - r := c.GetCachedRegionWithRLock(ctx.Region) - if r == nil { - return - } - peersNum := len(r.meta.Peers) - if len(ctx.Meta.Peers) != peersNum { - logutil.Logger(bo.GetCtx()).Info("retry and refresh current ctx after send request fail and up/down stores length changed", - zap.Stringer("current", ctx), - zap.Bool("needReload", scheduleReload), - zap.Reflect("oldPeers", ctx.Meta.Peers), - zap.Reflect("newPeers", r.meta.Peers), - zap.Error(err)) - return - } - - rs := r.getStore() - startForwarding := false - incEpochStoreIdx := -1 - - if err != nil { - storeIdx, s := rs.accessStore(ctx.AccessMode, ctx.AccessIdx) - leaderReq := ctx.Store.storeType == tikvrpc.TiKV && rs.workTiKVIdx == ctx.AccessIdx - - // Mark the store as failure if it's not a redirection request because we - // can't know the status of the proxy store by it. - if ctx.ProxyStore == nil { - // send fail but store is reachable, keep retry current peer for replica leader request. - // but we still need switch peer for follower-read or learner-read(i.e. tiflash) - if leaderReq { - if s.requestLiveness(bo, c) == reachable { - return - } else if c.enableForwarding { - s.startHealthCheckLoopIfNeeded(c) - startForwarding = true - } - } - - // invalidate regions in store. - incEpochStoreIdx = c.markRegionNeedBeRefill(s, storeIdx, rs) - } - } - - // try next peer to found new leader. - if ctx.AccessMode == tiKVOnly { - if startForwarding || ctx.ProxyStore != nil { - var currentProxyIdx AccessIndex = -1 - if ctx.ProxyStore != nil { - currentProxyIdx = ctx.ProxyAccessIdx - } - // In case the epoch of the store is increased, try to avoid reloading the current region by also - // increasing the epoch stored in `rs`. - rs.switchNextProxyStore(r, currentProxyIdx, incEpochStoreIdx) - logutil.Logger(bo.GetCtx()).Info("switch region proxy peer to next due to send request fail", - zap.Stringer("current", ctx), - zap.Bool("needReload", scheduleReload), - zap.Error(err)) - } else { - rs.switchNextTiKVPeer(r, ctx.AccessIdx) - logutil.Logger(bo.GetCtx()).Info("switch region peer to next due to send request fail", - zap.Stringer("current", ctx), - zap.Bool("needReload", scheduleReload), - zap.Error(err)) - } - } else { - rs.switchNextFlashPeer(r, ctx.AccessIdx) - logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", - zap.Stringer("current", ctx), - zap.Bool("needReload", scheduleReload), - zap.Error(err)) - } - - // force reload region when retry all known peers in region. - if scheduleReload { - r.scheduleReload() - } - -} - -// LocateRegionByID searches for the region with ID. -func (c *RegionCache) LocateRegionByID(bo *retry.Backoffer, regionID uint64) (*KeyLocation, error) { - c.mu.RLock() - r := c.getRegionByIDFromCache(regionID) - c.mu.RUnlock() - if r != nil { - if r.checkNeedReloadAndMarkUpdated() { - lr, err := c.loadRegionByID(bo, regionID) - if err != nil { - // ignore error and use old region info. - logutil.Logger(bo.GetCtx()).Error("load region failure", - zap.Uint64("regionID", regionID), zap.Error(err)) - } else { - r = lr - c.mu.Lock() - c.insertRegionToCache(r) - c.mu.Unlock() - } - } - loc := &KeyLocation{ - Region: r.VerID(), - StartKey: r.StartKey(), - EndKey: r.EndKey(), - } - return loc, nil - } - - r, err := c.loadRegionByID(bo, regionID) - if err != nil { - return nil, errors.Trace(err) - } - - c.mu.Lock() - c.insertRegionToCache(r) - c.mu.Unlock() - return &KeyLocation{ - Region: r.VerID(), - StartKey: r.StartKey(), - EndKey: r.EndKey(), - }, nil -} - -// GroupKeysByRegion separates keys into groups by their belonging Regions. -// Specially it also returns the first key's region which may be used as the -// 'PrimaryLockKey' and should be committed ahead of others. -// filter is used to filter some unwanted keys. -func (c *RegionCache) GroupKeysByRegion(bo *retry.Backoffer, keys [][]byte, filter func(key, regionStartKey []byte) bool) (map[RegionVerID][][]byte, RegionVerID, error) { - groups := make(map[RegionVerID][][]byte) - var first RegionVerID - var lastLoc *KeyLocation - for i, k := range keys { - if lastLoc == nil || !lastLoc.Contains(k) { - var err error - lastLoc, err = c.LocateKey(bo, k) - if err != nil { - return nil, first, errors.Trace(err) - } - if filter != nil && filter(k, lastLoc.StartKey) { - continue - } - } - id := lastLoc.Region - if i == 0 { - first = id - } - groups[id] = append(groups[id], k) - } - return groups, first, nil -} - -// ListRegionIDsInKeyRange lists ids of regions in [start_key,end_key]. -func (c *RegionCache) ListRegionIDsInKeyRange(bo *retry.Backoffer, startKey, endKey []byte) (regionIDs []uint64, err error) { - for { - curRegion, err := c.LocateKey(bo, startKey) - if err != nil { - return nil, errors.Trace(err) - } - regionIDs = append(regionIDs, curRegion.Region.id) - if curRegion.Contains(endKey) { - break - } - startKey = curRegion.EndKey - } - return regionIDs, nil -} - -// LoadRegionsInKeyRange lists regions in [start_key,end_key]. -func (c *RegionCache) LoadRegionsInKeyRange(bo *retry.Backoffer, startKey, endKey []byte) (regions []*Region, err error) { - var batchRegions []*Region - for { - batchRegions, err = c.BatchLoadRegionsWithKeyRange(bo, startKey, endKey, defaultRegionsPerBatch) - if err != nil { - return nil, errors.Trace(err) - } - if len(batchRegions) == 0 { - // should never happen - break - } - regions = append(regions, batchRegions...) - endRegion := batchRegions[len(batchRegions)-1] - if endRegion.ContainsByEnd(endKey) { - break - } - startKey = endRegion.EndKey() - } - return -} - -// BatchLoadRegionsWithKeyRange loads at most given numbers of regions to the RegionCache, -// within the given key range from the startKey to endKey. Returns the loaded regions. -func (c *RegionCache) BatchLoadRegionsWithKeyRange(bo *retry.Backoffer, startKey []byte, endKey []byte, count int) (regions []*Region, err error) { - regions, err = c.scanRegions(bo, startKey, endKey, count) - if err != nil { - return - } - if len(regions) == 0 { - err = errors.New("PD returned no region") - return - } - - c.mu.Lock() - defer c.mu.Unlock() - - for _, region := range regions { - c.insertRegionToCache(region) - } - - return -} - -// BatchLoadRegionsFromKey loads at most given numbers of regions to the RegionCache, from the given startKey. Returns -// the endKey of the last loaded region. If some of the regions has no leader, their entries in RegionCache will not be -// updated. -func (c *RegionCache) BatchLoadRegionsFromKey(bo *retry.Backoffer, startKey []byte, count int) ([]byte, error) { - regions, err := c.BatchLoadRegionsWithKeyRange(bo, startKey, nil, count) - if err != nil { - return nil, errors.Trace(err) - } - return regions[len(regions)-1].EndKey(), nil -} - -// InvalidateCachedRegion removes a cached Region. -func (c *RegionCache) InvalidateCachedRegion(id RegionVerID) { - c.InvalidateCachedRegionWithReason(id, Other) -} - -// InvalidateCachedRegionWithReason removes a cached Region with the reason why it's invalidated. -func (c *RegionCache) InvalidateCachedRegionWithReason(id RegionVerID, reason InvalidReason) { - cachedRegion := c.GetCachedRegionWithRLock(id) - if cachedRegion == nil { - return - } - cachedRegion.invalidate(reason) -} - -// UpdateLeader update some region cache with newer leader info. -func (c *RegionCache) UpdateLeader(regionID RegionVerID, leader *metapb.Peer, currentPeerIdx AccessIndex) { - r := c.GetCachedRegionWithRLock(regionID) - if r == nil { - logutil.BgLogger().Debug("regionCache: cannot find region when updating leader", - zap.Uint64("regionID", regionID.GetID())) - return - } - - if leader == nil { - rs := r.getStore() - rs.switchNextTiKVPeer(r, currentPeerIdx) - logutil.BgLogger().Info("switch region peer to next due to NotLeader with NULL leader", - zap.Int("currIdx", int(currentPeerIdx)), - zap.Uint64("regionID", regionID.GetID())) - return - } - - if !c.switchWorkLeaderToPeer(r, leader) { - logutil.BgLogger().Info("invalidate region cache due to cannot find peer when updating leader", - zap.Uint64("regionID", regionID.GetID()), - zap.Int("currIdx", int(currentPeerIdx)), - zap.Uint64("leaderStoreID", leader.GetStoreId())) - r.invalidate(StoreNotFound) - } else { - logutil.BgLogger().Info("switch region leader to specific leader due to kv return NotLeader", - zap.Uint64("regionID", regionID.GetID()), - zap.Int("currIdx", int(currentPeerIdx)), - zap.Uint64("leaderStoreID", leader.GetStoreId())) - } -} - -// removeVersionFromCache removes a RegionVerID from cache, tries to cleanup -// both c.mu.regions and c.mu.versions. Note this function is not thread-safe. -func (c *RegionCache) removeVersionFromCache(oldVer RegionVerID, regionID uint64) { - delete(c.mu.regions, oldVer) - if ver, ok := c.mu.latestVersions[regionID]; ok && ver.Equals(oldVer) { - delete(c.mu.latestVersions, regionID) - } -} - -// insertRegionToCache tries to insert the Region to cache. -// It should be protected by c.mu.Lock(). -func (c *RegionCache) insertRegionToCache(cachedRegion *Region) { - old := c.mu.sorted.ReplaceOrInsert(newBtreeItem(cachedRegion)) - if old != nil { - store := cachedRegion.getStore() - oldRegion := old.(*btreeItem).cachedRegion - oldRegionStore := oldRegion.getStore() - // TODO(youjiali1995): remove this because the new retry logic can handle this issue. - // - // Joint consensus is enabled in v5.0, which is possible to make a leader step down as a learner during a conf change. - // And if hibernate region is enabled, after the leader step down, there can be a long time that there is no leader - // in the region and the leader info in PD is stale until requests are sent to followers or hibernate timeout. - // To solve it, one solution is always to try a different peer if the invalid reason of the old cached region is no-leader. - // There is a small probability that the current peer who reports no-leader becomes a leader and TiDB has to retry once in this case. - if InvalidReason(atomic.LoadInt32((*int32)(&oldRegion.invalidReason))) == NoLeader { - store.workTiKVIdx = (oldRegionStore.workTiKVIdx + 1) % AccessIndex(store.accessStoreNum(tiKVOnly)) - } - // Invalidate the old region in case it's not invalidated and some requests try with the stale region information. - oldRegion.invalidate(Other) - // Don't refresh TiFlash work idx for region. Otherwise, it will always goto a invalid store which - // is under transferring regions. - store.workTiFlashIdx = atomic.LoadInt32(&oldRegionStore.workTiFlashIdx) - c.removeVersionFromCache(oldRegion.VerID(), cachedRegion.VerID().id) - } - c.mu.regions[cachedRegion.VerID()] = cachedRegion - newVer := cachedRegion.VerID() - latest, ok := c.mu.latestVersions[cachedRegion.VerID().id] - if !ok || latest.GetVer() < newVer.GetVer() || latest.GetConfVer() < newVer.GetConfVer() { - c.mu.latestVersions[cachedRegion.VerID().id] = newVer - } -} - -// searchCachedRegion finds a region from cache by key. Like `getCachedRegion`, -// it should be called with c.mu.RLock(), and the returned Region should not be -// used after c.mu is RUnlock(). -// If the given key is the end key of the region that you want, you may set the second argument to true. This is useful -// when processing in reverse order. -func (c *RegionCache) searchCachedRegion(key []byte, isEndKey bool) *Region { - ts := time.Now().Unix() - var r *Region - c.mu.RLock() - c.mu.sorted.DescendLessOrEqual(newBtreeSearchItem(key), func(item btree.Item) bool { - r = item.(*btreeItem).cachedRegion - if isEndKey && bytes.Equal(r.StartKey(), key) { - r = nil // clear result - return true // iterate next item - } - if !r.checkRegionCacheTTL(ts) { - r = nil - return true - } - return false - }) - c.mu.RUnlock() - if r != nil && (!isEndKey && r.Contains(key) || isEndKey && r.ContainsByEnd(key)) { - return r - } - return nil -} - -// getRegionByIDFromCache tries to get region by regionID from cache. Like -// `getCachedRegion`, it should be called with c.mu.RLock(), and the returned -// Region should not be used after c.mu is RUnlock(). -func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region { - ts := time.Now().Unix() - ver, ok := c.mu.latestVersions[regionID] - if !ok { - return nil - } - latestRegion, ok := c.mu.regions[ver] - if !ok { - // should not happen - logutil.BgLogger().Warn("region version not found", - zap.Uint64("regionID", regionID), zap.Stringer("version", &ver)) - return nil - } - lastAccess := atomic.LoadInt64(&latestRegion.lastAccess) - if ts-lastAccess > regionCacheTTLSec { - return nil - } - if latestRegion != nil { - atomic.CompareAndSwapInt64(&latestRegion.lastAccess, atomic.LoadInt64(&latestRegion.lastAccess), ts) - } - return latestRegion -} - -// GetStoresByType gets stores by type `typ` -// TODO: revise it by get store by closure. -func (c *RegionCache) GetStoresByType(typ tikvrpc.EndpointType) []*Store { - c.storeMu.Lock() - defer c.storeMu.Unlock() - stores := make([]*Store, 0) - for _, store := range c.storeMu.stores { - if store.getResolveState() != resolved { - continue - } - if store.storeType == typ { - //TODO: revise it with store.clone() - storeLabel := make([]*metapb.StoreLabel, 0) - for _, label := range store.labels { - storeLabel = append(storeLabel, &metapb.StoreLabel{ - Key: label.Key, - Value: label.Value, - }) - } - stores = append(stores, &Store{ - addr: store.addr, - storeID: store.storeID, - labels: store.labels, - }) - } - } - return stores -} - -func filterUnavailablePeers(region *pd.Region) { - if len(region.DownPeers) == 0 { - return - } - new := region.Meta.Peers[:0] - for _, p := range region.Meta.Peers { - available := true - for _, downPeer := range region.DownPeers { - if p.Id == downPeer.Id && p.StoreId == downPeer.StoreId { - available = false - break - } - } - if available { - new = append(new, p) - } - } - region.Meta.Peers = new -} - -// loadRegion loads region from pd client, and picks the first peer as leader. -// If the given key is the end key of the region that you want, you may set the second argument to true. This is useful -// when processing in reverse order. -func (c *RegionCache) loadRegion(bo *retry.Backoffer, key []byte, isEndKey bool) (*Region, error) { - ctx := bo.GetCtx() - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("loadRegion", opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - - var backoffErr error - searchPrev := false - for { - if backoffErr != nil { - err := bo.Backoff(retry.BoPDRPC, backoffErr) - if err != nil { - return nil, errors.Trace(err) - } - } - var reg *pd.Region - var err error - if searchPrev { - reg, err = c.pdClient.GetPrevRegion(ctx, key) - } else { - reg, err = c.pdClient.GetRegion(ctx, key) - } - if err != nil { - metrics.RegionCacheCounterWithGetRegionError.Inc() - } else { - metrics.RegionCacheCounterWithGetRegionOK.Inc() - } - if err != nil { - backoffErr = errors.Errorf("loadRegion from PD failed, key: %q, err: %v", key, err) - continue - } - if reg == nil || reg.Meta == nil { - backoffErr = errors.Errorf("region not found for key %q", key) - continue - } - filterUnavailablePeers(reg) - if len(reg.Meta.Peers) == 0 { - return nil, errors.New("receive Region with no available peer") - } - if isEndKey && !searchPrev && bytes.Equal(reg.Meta.StartKey, key) && len(reg.Meta.StartKey) != 0 { - searchPrev = true - continue - } - region := &Region{meta: reg.Meta} - err = region.init(bo, c) - if err != nil { - return nil, err - } - if reg.Leader != nil { - c.switchWorkLeaderToPeer(region, reg.Leader) - } - return region, nil - } -} - -// loadRegionByID loads region from pd client, and picks the first peer as leader. -func (c *RegionCache) loadRegionByID(bo *retry.Backoffer, regionID uint64) (*Region, error) { - ctx := bo.GetCtx() - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("loadRegionByID", opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - var backoffErr error - for { - if backoffErr != nil { - err := bo.Backoff(retry.BoPDRPC, backoffErr) - if err != nil { - return nil, errors.Trace(err) - } - } - reg, err := c.pdClient.GetRegionByID(ctx, regionID) - if err != nil { - metrics.RegionCacheCounterWithGetRegionByIDError.Inc() - } else { - metrics.RegionCacheCounterWithGetRegionByIDOK.Inc() - } - if err != nil { - backoffErr = errors.Errorf("loadRegion from PD failed, regionID: %v, err: %v", regionID, err) - continue - } - if reg == nil || reg.Meta == nil { - return nil, errors.Errorf("region not found for regionID %d", regionID) - } - filterUnavailablePeers(reg) - if len(reg.Meta.Peers) == 0 { - return nil, errors.New("receive Region with no available peer") - } - region := &Region{meta: reg.Meta} - err = region.init(bo, c) - if err != nil { - return nil, err - } - if reg.Leader != nil { - c.switchWorkLeaderToPeer(region, reg.Leader) - } - return region, nil - } -} - -// scanRegions scans at most `limit` regions from PD, starts from the region containing `startKey` and in key order. -// Regions with no leader will not be returned. -func (c *RegionCache) scanRegions(bo *retry.Backoffer, startKey, endKey []byte, limit int) ([]*Region, error) { - if limit == 0 { - return nil, nil - } - ctx := bo.GetCtx() - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("scanRegions", opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - - var backoffErr error - for { - if backoffErr != nil { - err := bo.Backoff(retry.BoPDRPC, backoffErr) - if err != nil { - return nil, errors.Trace(err) - } - } - regionsInfo, err := c.pdClient.ScanRegions(ctx, startKey, endKey, limit) - if err != nil { - metrics.RegionCacheCounterWithScanRegionsError.Inc() - backoffErr = errors.Errorf( - "scanRegion from PD failed, startKey: %q, limit: %q, err: %v", - startKey, - limit, - err) - continue - } - - metrics.RegionCacheCounterWithScanRegionsOK.Inc() - - if len(regionsInfo) == 0 { - return nil, errors.New("PD returned no region") - } - regions := make([]*Region, 0, len(regionsInfo)) - for _, r := range regionsInfo { - region := &Region{meta: r.Meta} - err := region.init(bo, c) - if err != nil { - return nil, err - } - leader := r.Leader - // Leader id = 0 indicates no leader. - if leader != nil && leader.GetId() != 0 { - c.switchWorkLeaderToPeer(region, leader) - regions = append(regions, region) - } - } - if len(regions) == 0 { - return nil, errors.New("receive Regions with no peer") - } - if len(regions) < len(regionsInfo) { - logutil.Logger(context.Background()).Debug( - "regionCache: scanRegion finished but some regions has no leader.") - } - return regions, nil - } -} - -// GetCachedRegionWithRLock returns region with lock. -func (c *RegionCache) GetCachedRegionWithRLock(regionID RegionVerID) (r *Region) { - c.mu.RLock() - r = c.mu.regions[regionID] - c.mu.RUnlock() - return -} - -func (c *RegionCache) getStoreAddr(bo *retry.Backoffer, region *Region, store *Store) (addr string, err error) { - state := store.getResolveState() - switch state { - case resolved, needCheck: - addr = store.addr - return - case unresolved: - addr, err = store.initResolve(bo, c) - return - case deleted: - addr = c.changeToActiveStore(region, store) - return - case tombstone: - return "", nil - default: - panic("unsupported resolve state") - } -} - -func (c *RegionCache) getProxyStore(region *Region, store *Store, rs *regionStore, workStoreIdx AccessIndex) (proxyStore *Store, proxyAccessIdx AccessIndex, proxyStoreIdx int) { - if !c.enableForwarding || store.storeType != tikvrpc.TiKV || atomic.LoadInt32(&store.needForwarding) == 0 { - return - } - - if rs.proxyTiKVIdx >= 0 { - storeIdx, proxyStore := rs.accessStore(tiKVOnly, rs.proxyTiKVIdx) - return proxyStore, rs.proxyTiKVIdx, storeIdx - } - - tikvNum := rs.accessStoreNum(tiKVOnly) - if tikvNum <= 1 { - return - } - - // Randomly select an non-leader peer - first := rand.Intn(tikvNum - 1) - if first >= int(workStoreIdx) { - first = (first + 1) % tikvNum - } - - // If the current selected peer is not reachable, switch to the next one, until a reachable peer is found or all - // peers are checked. - for i := 0; i < tikvNum; i++ { - index := (i + first) % tikvNum - // Skip work store which is the actual store to be accessed - if index == int(workStoreIdx) { - continue - } - storeIdx, store := rs.accessStore(tiKVOnly, AccessIndex(index)) - // Skip unreachable stores. - if atomic.LoadInt32(&store.needForwarding) != 0 { - continue - } - - rs.setProxyStoreIdx(region, AccessIndex(index)) - return store, AccessIndex(index), storeIdx - } - - return nil, 0, 0 -} - -// changeToActiveStore replace the deleted store in the region by an up-to-date store in the stores map. -// The order is guaranteed by reResolve() which adds the new store before marking old store deleted. -func (c *RegionCache) changeToActiveStore(region *Region, store *Store) (addr string) { - c.storeMu.RLock() - store = c.storeMu.stores[store.storeID] - c.storeMu.RUnlock() - for { - oldRegionStore := region.getStore() - newRegionStore := oldRegionStore.clone() - newRegionStore.stores = make([]*Store, 0, len(oldRegionStore.stores)) - for _, s := range oldRegionStore.stores { - if s.storeID == store.storeID { - newRegionStore.stores = append(newRegionStore.stores, store) - } else { - newRegionStore.stores = append(newRegionStore.stores, s) - } - } - if region.compareAndSwapStore(oldRegionStore, newRegionStore) { - break - } - } - addr = store.addr - return -} - -func (c *RegionCache) getStoreByStoreID(storeID uint64) (store *Store) { - var ok bool - c.storeMu.Lock() - store, ok = c.storeMu.stores[storeID] - if ok { - c.storeMu.Unlock() - return - } - store = &Store{storeID: storeID} - c.storeMu.stores[storeID] = store - c.storeMu.Unlock() - return -} - -func (c *RegionCache) getStoresByLabels(labels []*metapb.StoreLabel) []*Store { - c.storeMu.RLock() - defer c.storeMu.RUnlock() - s := make([]*Store, 0) - for _, store := range c.storeMu.stores { - if store.IsLabelsMatch(labels) { - s = append(s, store) - } - } - return s -} - -// OnRegionEpochNotMatch removes the old region and inserts new regions into the cache. -// It returns whether retries the request because it's possible the region epoch is ahead of TiKV's due to slow appling. -func (c *RegionCache) OnRegionEpochNotMatch(bo *retry.Backoffer, ctx *RPCContext, currentRegions []*metapb.Region) (bool, error) { - if len(currentRegions) == 0 { - c.InvalidateCachedRegionWithReason(ctx.Region, EpochNotMatch) - return false, nil - } - - // Find whether the region epoch in `ctx` is ahead of TiKV's. If so, backoff. - for _, meta := range currentRegions { - if meta.GetId() == ctx.Region.id && - (meta.GetRegionEpoch().GetConfVer() < ctx.Region.confVer || - meta.GetRegionEpoch().GetVersion() < ctx.Region.ver) { - err := errors.Errorf("region epoch is ahead of tikv. rpc ctx: %+v, currentRegions: %+v", ctx, currentRegions) - logutil.BgLogger().Info("region epoch is ahead of tikv", zap.Error(err)) - return true, bo.Backoff(retry.BoRegionMiss, err) - } - } - - needInvalidateOld := true - newRegions := make([]*Region, 0, len(currentRegions)) - // If the region epoch is not ahead of TiKV's, replace region meta in region cache. - for _, meta := range currentRegions { - if _, ok := c.pdClient.(*CodecPDClient); ok { - var err error - if meta, err = decodeRegionMetaKeyWithShallowCopy(meta); err != nil { - return false, errors.Errorf("newRegion's range key is not encoded: %v, %v", meta, err) - } - } - region := &Region{meta: meta} - err := region.init(bo, c) - if err != nil { - return false, err - } - var initLeaderStoreID uint64 - if ctx.Store.storeType == tikvrpc.TiFlash { - initLeaderStoreID = region.findElectableStoreID() - } else { - initLeaderStoreID = ctx.Store.storeID - } - c.switchWorkLeaderToPeer(region, region.getPeerOnStore(initLeaderStoreID)) - newRegions = append(newRegions, region) - if ctx.Region == region.VerID() { - needInvalidateOld = false - } - } - c.mu.Lock() - for _, region := range newRegions { - c.insertRegionToCache(region) - } - if needInvalidateOld { - cachedRegion, ok := c.mu.regions[ctx.Region] - if ok { - cachedRegion.invalidate(EpochNotMatch) - } - } - c.mu.Unlock() - return false, nil -} - -// PDClient returns the pd.Client in RegionCache. -func (c *RegionCache) PDClient() pd.Client { - return c.pdClient -} - -// GetTiFlashStoreAddrs returns addresses of all tiflash nodes. -func (c *RegionCache) GetTiFlashStoreAddrs() []string { - c.storeMu.RLock() - defer c.storeMu.RUnlock() - var addrs []string - for _, s := range c.storeMu.stores { - if s.storeType == tikvrpc.TiFlash { - addrs = append(addrs, s.addr) - } - } - return addrs -} - -// btreeItem is BTree's Item that uses []byte to compare. -type btreeItem struct { - key []byte - cachedRegion *Region -} - -func newBtreeItem(cr *Region) *btreeItem { - return &btreeItem{ - key: cr.StartKey(), - cachedRegion: cr, - } -} - -func newBtreeSearchItem(key []byte) *btreeItem { - return &btreeItem{ - key: key, - } -} - -func (item *btreeItem) Less(other btree.Item) bool { - return bytes.Compare(item.key, other.(*btreeItem).key) < 0 -} - -// GetID returns id. -func (r *Region) GetID() uint64 { - return r.meta.GetId() -} - -// GetMeta returns region meta. -func (r *Region) GetMeta() *metapb.Region { - return proto.Clone(r.meta).(*metapb.Region) -} - -// GetLeaderPeerID returns leader peer ID. -func (r *Region) GetLeaderPeerID() uint64 { - store := r.getStore() - if int(store.workTiKVIdx) >= store.accessStoreNum(tiKVOnly) { - return 0 - } - storeIdx, _ := store.accessStore(tiKVOnly, store.workTiKVIdx) - return r.meta.Peers[storeIdx].Id -} - -// GetLeaderStoreID returns the store ID of the leader region. -func (r *Region) GetLeaderStoreID() uint64 { - store := r.getStore() - if int(store.workTiKVIdx) >= store.accessStoreNum(tiKVOnly) { - return 0 - } - storeIdx, _ := store.accessStore(tiKVOnly, store.workTiKVIdx) - return r.meta.Peers[storeIdx].StoreId -} - -func (r *Region) getKvStorePeer(rs *regionStore, aidx AccessIndex) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { - storeIdx, store = rs.accessStore(tiKVOnly, aidx) - peer = r.meta.Peers[storeIdx] - accessIdx = aidx - return -} - -// WorkStorePeer returns current work store with work peer. -func (r *Region) WorkStorePeer(rs *regionStore) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { - return r.getKvStorePeer(rs, rs.workTiKVIdx) -} - -// FollowerStorePeer returns a follower store with follower peer. -func (r *Region) FollowerStorePeer(rs *regionStore, followerStoreSeed uint32, op *storeSelectorOp) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { - return r.getKvStorePeer(rs, rs.follower(followerStoreSeed, op)) -} - -// AnyStorePeer returns a leader or follower store with the associated peer. -func (r *Region) AnyStorePeer(rs *regionStore, followerStoreSeed uint32, op *storeSelectorOp) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { - return r.getKvStorePeer(rs, rs.kvPeer(followerStoreSeed, op)) -} - -// RegionVerID is a unique ID that can identify a Region at a specific version. -type RegionVerID struct { - id uint64 - confVer uint64 - ver uint64 -} - -// NewRegionVerID creates a region ver id, which used for invalidating regions. -func NewRegionVerID(id, confVer, ver uint64) RegionVerID { - return RegionVerID{id, confVer, ver} -} - -// GetID returns the id of the region -func (r *RegionVerID) GetID() uint64 { - return r.id -} - -// GetVer returns the version of the region's epoch -func (r *RegionVerID) GetVer() uint64 { - return r.ver -} - -// GetConfVer returns the conf ver of the region's epoch -func (r *RegionVerID) GetConfVer() uint64 { - return r.confVer -} - -// String formats the RegionVerID to string -func (r *RegionVerID) String() string { - return fmt.Sprintf("{ region id: %v, ver: %v, confVer: %v }", r.id, r.ver, r.confVer) -} - -// Equals checks whether the RegionVerID equals to another one -func (r *RegionVerID) Equals(another RegionVerID) bool { - return r.id == another.id && r.confVer == another.confVer && r.ver == another.ver -} - -// VerID returns the Region's RegionVerID. -func (r *Region) VerID() RegionVerID { - return RegionVerID{ - id: r.meta.GetId(), - confVer: r.meta.GetRegionEpoch().GetConfVer(), - ver: r.meta.GetRegionEpoch().GetVersion(), - } -} - -// StartKey returns StartKey. -func (r *Region) StartKey() []byte { - return r.meta.StartKey -} - -// EndKey returns EndKey. -func (r *Region) EndKey() []byte { - return r.meta.EndKey -} - -// switchWorkLeaderToPeer switches current store to the one on specific store. It returns -// false if no peer matches the peer. -func (c *RegionCache) switchWorkLeaderToPeer(r *Region, peer *metapb.Peer) (found bool) { - globalStoreIdx, found := c.getPeerStoreIndex(r, peer) - if !found { - return - } -retry: - // switch to new leader. - oldRegionStore := r.getStore() - var leaderIdx AccessIndex - for i, gIdx := range oldRegionStore.accessIndex[tiKVOnly] { - if gIdx == globalStoreIdx { - leaderIdx = AccessIndex(i) - } - } - if oldRegionStore.workTiKVIdx == leaderIdx { - return - } - newRegionStore := oldRegionStore.clone() - newRegionStore.workTiKVIdx = leaderIdx - if !r.compareAndSwapStore(oldRegionStore, newRegionStore) { - goto retry - } - return -} - -func (r *regionStore) switchNextFlashPeer(rr *Region, currentPeerIdx AccessIndex) { - nextIdx := (currentPeerIdx + 1) % AccessIndex(r.accessStoreNum(tiFlashOnly)) - newRegionStore := r.clone() - newRegionStore.workTiFlashIdx = int32(nextIdx) - rr.compareAndSwapStore(r, newRegionStore) -} - -func (r *regionStore) switchNextTiKVPeer(rr *Region, currentPeerIdx AccessIndex) { - if r.workTiKVIdx != currentPeerIdx { - return - } - nextIdx := (currentPeerIdx + 1) % AccessIndex(r.accessStoreNum(tiKVOnly)) - newRegionStore := r.clone() - newRegionStore.workTiKVIdx = nextIdx - rr.compareAndSwapStore(r, newRegionStore) -} - -// switchNextProxyStore switches the index of the peer that will forward requests to the leader to the next peer. -// If proxy is currently not used on this region, the value of `currentProxyIdx` should be -1, and a random peer will -// be select in this case. -func (r *regionStore) switchNextProxyStore(rr *Region, currentProxyIdx AccessIndex, incEpochStoreIdx int) { - if r.proxyTiKVIdx != currentProxyIdx { - return - } - - tikvNum := r.accessStoreNum(tiKVOnly) - var nextIdx AccessIndex - - // If the region is not using proxy before, randomly select a non-leader peer for the first try. - if currentProxyIdx == -1 { - // Randomly select an non-leader peer - // TODO: Skip unreachable peers here. - nextIdx = AccessIndex(rand.Intn(tikvNum - 1)) - if nextIdx >= r.workTiKVIdx { - nextIdx++ - } - } else { - nextIdx = (currentProxyIdx + 1) % AccessIndex(tikvNum) - // skips the current workTiKVIdx - if nextIdx == r.workTiKVIdx { - nextIdx = (nextIdx + 1) % AccessIndex(tikvNum) - } - } - - newRegionStore := r.clone() - newRegionStore.proxyTiKVIdx = nextIdx - if incEpochStoreIdx >= 0 { - newRegionStore.storeEpochs[incEpochStoreIdx]++ - } - rr.compareAndSwapStore(r, newRegionStore) -} - -func (r *regionStore) setProxyStoreIdx(rr *Region, idx AccessIndex) { - if r.proxyTiKVIdx == idx { - return - } - - newRegionStore := r.clone() - newRegionStore.proxyTiKVIdx = idx - success := rr.compareAndSwapStore(r, newRegionStore) - logutil.BgLogger().Debug("try set proxy store index", - zap.Uint64("region", rr.GetID()), - zap.Int("index", int(idx)), - zap.Bool("success", success)) -} - -func (r *regionStore) unsetProxyStoreIfNeeded(rr *Region) { - r.setProxyStoreIdx(rr, -1) -} - -func (r *Region) findElectableStoreID() uint64 { - if len(r.meta.Peers) == 0 { - return 0 - } - for _, p := range r.meta.Peers { - if p.Role != metapb.PeerRole_Learner { - return p.StoreId - } - } - return 0 -} - -func (r *Region) getPeerOnStore(storeID uint64) *metapb.Peer { - for _, p := range r.meta.Peers { - if p.StoreId == storeID { - return p - } - } - return nil -} - -func (c *RegionCache) getPeerStoreIndex(r *Region, peer *metapb.Peer) (idx int, found bool) { - if len(r.meta.Peers) == 0 || peer == nil { - return - } - for i, p := range r.meta.Peers { - if isSamePeer(p, peer) { - idx = i - found = true - return - } - } - return -} - -// Contains checks whether the key is in the region, for the maximum region endKey is empty. -// startKey <= key < endKey. -func (r *Region) Contains(key []byte) bool { - return bytes.Compare(r.meta.GetStartKey(), key) <= 0 && - (bytes.Compare(key, r.meta.GetEndKey()) < 0 || len(r.meta.GetEndKey()) == 0) -} - -// ContainsByEnd check the region contains the greatest key that is less than key. -// for the maximum region endKey is empty. -// startKey < key <= endKey. -func (r *Region) ContainsByEnd(key []byte) bool { - return bytes.Compare(r.meta.GetStartKey(), key) < 0 && - (bytes.Compare(key, r.meta.GetEndKey()) <= 0 || len(r.meta.GetEndKey()) == 0) -} - -// Store contains a kv process's address. -type Store struct { - addr string // loaded store address - saddr string // loaded store status address - storeID uint64 // store's id - state uint64 // unsafe store storeState - labels []*metapb.StoreLabel // stored store labels - resolveMutex sync.Mutex // protect pd from concurrent init requests - epoch uint32 // store fail epoch, see RegionStore.storeEpochs - storeType tikvrpc.EndpointType // type of the store - tokenCount atomic2.Int64 // used store token count - - // whether the store is disconnected due to some reason, therefore requests to the store needs to be - // forwarded by other stores. this is also the flag that a checkUntilHealth goroutine is running for this store. - // this mechanism is currently only applicable for TiKV stores. - needForwarding int32 -} - -type resolveState uint64 - -const ( - // The store is just created and normally is being resolved. - // Store in this state will only be resolved by initResolve(). - unresolved resolveState = iota - // The store is resolved and its address is valid. - resolved - // Request failed on this store and it will be re-resolved by asyncCheckAndResolveLoop(). - needCheck - // The store's address or label is changed and marked deleted. - // There is a new store struct replaced it in the RegionCache and should - // call changeToActiveStore() to get the new struct. - deleted - // The store is a tombstone. Should invalidate the region if tries to access it. - tombstone -) - -// IsTiFlash returns true if the storeType is TiFlash -func (s *Store) IsTiFlash() bool { - return s.storeType == tikvrpc.TiFlash -} - -// StoreID returns storeID. -func (s *Store) StoreID() uint64 { - return s.storeID -} - -// initResolve resolves the address of the store that never resolved and returns an -// empty string if it's a tombstone. -func (s *Store) initResolve(bo *retry.Backoffer, c *RegionCache) (addr string, err error) { - s.resolveMutex.Lock() - state := s.getResolveState() - defer s.resolveMutex.Unlock() - if state != unresolved { - if state != tombstone { - addr = s.addr - } - return - } - var store *metapb.Store - for { - store, err = c.pdClient.GetStore(bo.GetCtx(), s.storeID) - if err != nil { - metrics.RegionCacheCounterWithGetStoreError.Inc() - } else { - metrics.RegionCacheCounterWithGetStoreOK.Inc() - } - if bo.GetCtx().Err() != nil && errors.Cause(bo.GetCtx().Err()) == context.Canceled { - return - } - if err != nil && !isStoreNotFoundError(err) { - // TODO: more refine PD error status handle. - err = errors.Errorf("loadStore from PD failed, id: %d, err: %v", s.storeID, err) - if err = bo.Backoff(retry.BoPDRPC, err); err != nil { - return - } - continue - } - // The store is a tombstone. - if store == nil { - s.setResolveState(tombstone) - return "", nil - } - addr = store.GetAddress() - if addr == "" { - return "", errors.Errorf("empty store(%d) address", s.storeID) - } - s.addr = addr - s.saddr = store.GetStatusAddress() - s.storeType = tikvrpc.GetStoreTypeByMeta(store) - s.labels = store.GetLabels() - // Shouldn't have other one changing its state concurrently, but we still use changeResolveStateTo for safety. - s.changeResolveStateTo(unresolved, resolved) - return s.addr, nil - } -} - -// A quick and dirty solution to find out whether an err is caused by StoreNotFound. -// todo: A better solution, maybe some err-code based error handling? -func isStoreNotFoundError(err error) bool { - return strings.Contains(err.Error(), "invalid store ID") && strings.Contains(err.Error(), "not found") -} - -// reResolve try to resolve addr for store that need check. Returns false if the region is in tombstone state or is -// deleted. -func (s *Store) reResolve(c *RegionCache) (bool, error) { - var addr string - store, err := c.pdClient.GetStore(context.Background(), s.storeID) - if err != nil { - metrics.RegionCacheCounterWithGetStoreError.Inc() - } else { - metrics.RegionCacheCounterWithGetStoreOK.Inc() - } - // `err` here can mean either "load Store from PD failed" or "store not found" - // If load Store from PD is successful but PD didn't find the store - // the err should be handled by next `if` instead of here - if err != nil && !isStoreNotFoundError(err) { - logutil.BgLogger().Error("loadStore from PD failed", zap.Uint64("id", s.storeID), zap.Error(err)) - // we cannot do backoff in reResolve loop but try check other store and wait tick. - return false, err - } - if store == nil { - // store has be removed in PD, we should invalidate all regions using those store. - logutil.BgLogger().Info("invalidate regions in removed store", - zap.Uint64("store", s.storeID), zap.String("add", s.addr)) - atomic.AddUint32(&s.epoch, 1) - s.setResolveState(tombstone) - metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() - return false, nil - } - - storeType := tikvrpc.GetStoreTypeByMeta(store) - addr = store.GetAddress() - if s.addr != addr || !s.IsSameLabels(store.GetLabels()) { - newStore := &Store{storeID: s.storeID, addr: addr, saddr: store.GetStatusAddress(), storeType: storeType, labels: store.GetLabels(), state: uint64(resolved)} - c.storeMu.Lock() - c.storeMu.stores[newStore.storeID] = newStore - c.storeMu.Unlock() - s.setResolveState(deleted) - return false, nil - } - s.changeResolveStateTo(needCheck, resolved) - return true, nil -} - -func (s *Store) getResolveState() resolveState { - var state resolveState - if s == nil { - return state - } - return resolveState(atomic.LoadUint64(&s.state)) -} - -func (s *Store) setResolveState(state resolveState) { - atomic.StoreUint64(&s.state, uint64(state)) -} - -// changeResolveStateTo changes the store resolveState from the old state to the new state. -// Returns true if it changes the state successfully, and false if the store's state -// is changed by another one. -func (s *Store) changeResolveStateTo(from, to resolveState) bool { - for { - state := s.getResolveState() - if state == to { - return true - } - if state != from { - return false - } - if atomic.CompareAndSwapUint64(&s.state, uint64(from), uint64(to)) { - return true - } - } -} - -// markNeedCheck marks resolved store to be async resolve to check store addr change. -func (s *Store) markNeedCheck(notifyCheckCh chan struct{}) { - if s.changeResolveStateTo(resolved, needCheck) { - select { - case notifyCheckCh <- struct{}{}: - default: - } - } -} - -// IsSameLabels returns whether the store have the same labels with target labels -func (s *Store) IsSameLabels(labels []*metapb.StoreLabel) bool { - if len(s.labels) != len(labels) { - return false - } - return s.IsLabelsMatch(labels) -} - -// IsLabelsMatch return whether the store's labels match the target labels -func (s *Store) IsLabelsMatch(labels []*metapb.StoreLabel) bool { - if len(labels) < 1 { - return true - } - for _, targetLabel := range labels { - match := false - for _, label := range s.labels { - if targetLabel.Key == label.Key && targetLabel.Value == label.Value { - match = true - break - } - } - if !match { - return false - } - } - return true -} - -type livenessState uint32 - -var ( - livenessSf singleflight.Group - // storeLivenessTimeout is the max duration of resolving liveness of a TiKV instance. - storeLivenessTimeout time.Duration -) - -// SetStoreLivenessTimeout sets storeLivenessTimeout to t. -func SetStoreLivenessTimeout(t time.Duration) { - storeLivenessTimeout = t -} - -// GetStoreLivenessTimeout returns storeLivenessTimeout. -func GetStoreLivenessTimeout() time.Duration { - return storeLivenessTimeout -} - -const ( - unknown livenessState = iota - reachable - unreachable -) - -func (s *Store) startHealthCheckLoopIfNeeded(c *RegionCache) { - // This mechanism doesn't support non-TiKV stores currently. - if s.storeType != tikvrpc.TiKV { - logutil.BgLogger().Info("[health check] skip running health check loop for non-tikv store", - zap.Uint64("storeID", s.storeID), zap.String("addr", s.addr)) - return - } - - // It may be already started by another thread. - if atomic.CompareAndSwapInt32(&s.needForwarding, 0, 1) { - go s.checkUntilHealth(c) - } -} - -func (s *Store) checkUntilHealth(c *RegionCache) { - defer atomic.CompareAndSwapInt32(&s.needForwarding, 1, 0) - - ticker := time.NewTicker(time.Second) - lastCheckPDTime := time.Now() - - // TODO(MyonKeminta): Set a more proper ctx here so that it can be interrupted immediately when the RegionCache is - // shutdown. - ctx := context.Background() - for { - select { - case <-c.closeCh: - return - case <-ticker.C: - if time.Since(lastCheckPDTime) > time.Second*30 { - lastCheckPDTime = time.Now() - - valid, err := s.reResolve(c) - if err != nil { - logutil.BgLogger().Warn("[health check] failed to re-resolve unhealthy store", zap.Error(err)) - } else if !valid { - logutil.BgLogger().Info("[health check] store meta deleted, stop checking", zap.Uint64("storeID", s.storeID), zap.String("addr", s.addr)) - return - } - } - - bo := retry.NewNoopBackoff(ctx) - l := s.requestLiveness(bo, c) - if l == reachable { - logutil.BgLogger().Info("[health check] store became reachable", zap.Uint64("storeID", s.storeID)) - - return - } - } - } -} - -func (s *Store) requestLiveness(bo *retry.Backoffer, c *RegionCache) (l livenessState) { - if c != nil && c.testingKnobs.mockRequestLiveness != nil { - return c.testingKnobs.mockRequestLiveness(s, bo) - } - - if storeLivenessTimeout == 0 { - return unreachable - } - - if s.getResolveState() != resolved { - l = unknown - return - } - addr := s.addr - rsCh := livenessSf.DoChan(addr, func() (interface{}, error) { - return invokeKVStatusAPI(addr, storeLivenessTimeout), nil - }) - var ctx context.Context - if bo != nil { - ctx = bo.GetCtx() - } else { - ctx = context.Background() - } - select { - case rs := <-rsCh: - l = rs.Val.(livenessState) - case <-ctx.Done(): - l = unknown - return - } - return -} - -// GetAddr returns the address of the store -func (s *Store) GetAddr() string { - return s.addr -} - -func invokeKVStatusAPI(addr string, timeout time.Duration) (l livenessState) { - start := time.Now() - defer func() { - if l == reachable { - metrics.StatusCountWithOK.Inc() - } else { - metrics.StatusCountWithError.Inc() - } - metrics.TiKVStatusDuration.WithLabelValues(addr).Observe(time.Since(start).Seconds()) - }() - ctx, cancel := context.WithTimeout(context.Background(), timeout) - defer cancel() - - conn, cli, err := createKVHealthClient(ctx, addr) - if err != nil { - logutil.BgLogger().Info("[health check] create grpc connection failed", zap.String("store", addr), zap.Error(err)) - l = unreachable - return - } - defer func() { - err := conn.Close() - if err != nil { - logutil.BgLogger().Info("[health check] failed to close the grpc connection for health check", zap.String("store", addr), zap.Error(err)) - } - }() - - req := &healthpb.HealthCheckRequest{} - resp, err := cli.Check(ctx, req) - if err != nil { - logutil.BgLogger().Info("[health check] check health error", zap.String("store", addr), zap.Error(err)) - l = unreachable - return - } - - status := resp.GetStatus() - if status == healthpb.HealthCheckResponse_UNKNOWN { - logutil.BgLogger().Info("[health check] check health returns unknown", zap.String("store", addr)) - l = unknown - return - } - - if status != healthpb.HealthCheckResponse_SERVING { - logutil.BgLogger().Info("[health check] service not serving", zap.Stringer("status", status)) - l = unreachable - return - } - - l = reachable - return -} - -func createKVHealthClient(ctx context.Context, addr string) (*grpc.ClientConn, healthpb.HealthClient, error) { - // Temporarily directly load the config from the global config, however it's not a good idea to let RegionCache to - // access it. - // TODO: Pass the config in a better way, or use the connArray inner the client directly rather than creating new - // connection. - - cfg := config.GetGlobalConfig() - - opt := grpc.WithInsecure() - if len(cfg.Security.ClusterSSLCA) != 0 { - tlsConfig, err := cfg.Security.ToTLSConfig() - if err != nil { - return nil, nil, errors.Trace(err) - } - opt = grpc.WithTransportCredentials(credentials.NewTLS(tlsConfig)) - } - keepAlive := cfg.TiKVClient.GrpcKeepAliveTime - keepAliveTimeout := cfg.TiKVClient.GrpcKeepAliveTimeout - conn, err := grpc.DialContext( - ctx, - addr, - opt, - grpc.WithInitialWindowSize(client.GrpcInitialWindowSize), - grpc.WithInitialConnWindowSize(client.GrpcInitialConnWindowSize), - grpc.WithConnectParams(grpc.ConnectParams{ - Backoff: backoff.Config{ - BaseDelay: 100 * time.Millisecond, // Default was 1s. - Multiplier: 1.6, // Default - Jitter: 0.2, // Default - MaxDelay: 3 * time.Second, // Default was 120s. - }, - MinConnectTimeout: 5 * time.Second, - }), - grpc.WithKeepaliveParams(keepalive.ClientParameters{ - Time: time.Duration(keepAlive) * time.Second, - Timeout: time.Duration(keepAliveTimeout) * time.Second, - PermitWithoutStream: true, - }), - ) - if err != nil { - return nil, nil, errors.Trace(err) - } - cli := healthpb.NewHealthClient(conn) - return conn, cli, nil -} - -func isSamePeer(lhs *metapb.Peer, rhs *metapb.Peer) bool { - return lhs == rhs || (lhs.GetId() == rhs.GetId() && lhs.GetStoreId() == rhs.GetStoreId()) -} diff --git a/store/tikv/locate/region_cache_test.go b/store/tikv/locate/region_cache_test.go deleted file mode 100644 index 155bacf5c9c4f..0000000000000 --- a/store/tikv/locate/region_cache_test.go +++ /dev/null @@ -1,1508 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package locate - -import ( - "context" - "errors" - "fmt" - "math/rand" - "sync/atomic" - "testing" - "time" - - "github.com/google/btree" - . "github.com/pingcap/check" - "github.com/pingcap/kvproto/pkg/errorpb" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/mockstore" - "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - pd "github.com/tikv/pd/client" -) - -type OneByOneSuite = mockstore.OneByOneSuite -type testRegionCacheSuite struct { - OneByOneSuite - cluster *mocktikv.Cluster - store1 uint64 // store1 is leader - store2 uint64 // store2 is follower - peer1 uint64 // peer1 is leader - peer2 uint64 // peer2 is follower - region1 uint64 - cache *RegionCache - bo *retry.Backoffer -} - -var _ = Suite(&testRegionCacheSuite{}) - -func (s *testRegionCacheSuite) SetUpTest(c *C) { - s.cluster = mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) - storeIDs, peerIDs, regionID, _ := mocktikv.BootstrapWithMultiStores(s.cluster, 2) - s.region1 = regionID - s.store1 = storeIDs[0] - s.store2 = storeIDs[1] - s.peer1 = peerIDs[0] - s.peer2 = peerIDs[1] - pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster)} - s.cache = NewRegionCache(pdCli) - s.bo = retry.NewBackofferWithVars(context.Background(), 5000, nil) -} - -func (s *testRegionCacheSuite) TearDownTest(c *C) { - s.cache.Close() -} - -func (s *testRegionCacheSuite) storeAddr(id uint64) string { - return fmt.Sprintf("store%d", id) -} - -func (s *testRegionCacheSuite) checkCache(c *C, len int) { - ts := time.Now().Unix() - c.Assert(validRegions(s.cache.mu.regions, ts), Equals, len) - c.Assert(validRegionsSearchedByVersions(s.cache.mu.latestVersions, s.cache.mu.regions, ts), Equals, len) - c.Assert(validRegionsInBtree(s.cache.mu.sorted, ts), Equals, len) -} - -func validRegionsSearchedByVersions( - versions map[uint64]RegionVerID, - regions map[RegionVerID]*Region, - ts int64, -) (count int) { - for _, ver := range versions { - region, ok := regions[ver] - if !ok || !region.checkRegionCacheTTL(ts) { - continue - } - count++ - } - return -} - -func validRegions(regions map[RegionVerID]*Region, ts int64) (len int) { - for _, region := range regions { - if !region.checkRegionCacheTTL(ts) { - continue - } - len++ - } - return -} - -func validRegionsInBtree(t *btree.BTree, ts int64) (len int) { - t.Descend(func(item btree.Item) bool { - r := item.(*btreeItem).cachedRegion - if !r.checkRegionCacheTTL(ts) { - return true - } - len++ - return true - }) - return -} - -func (s *testRegionCacheSuite) getRegion(c *C, key []byte) *Region { - _, err := s.cache.LocateKey(s.bo, key) - c.Assert(err, IsNil) - r := s.cache.searchCachedRegion(key, false) - c.Assert(r, NotNil) - return r -} - -func (s *testRegionCacheSuite) getRegionWithEndKey(c *C, key []byte) *Region { - _, err := s.cache.LocateEndKey(s.bo, key) - c.Assert(err, IsNil) - r := s.cache.searchCachedRegion(key, true) - c.Assert(r, NotNil) - return r -} - -func (s *testRegionCacheSuite) getAddr(c *C, key []byte, replicaRead kv.ReplicaReadType, seed uint32) string { - loc, err := s.cache.LocateKey(s.bo, key) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, replicaRead, seed) - c.Assert(err, IsNil) - if ctx == nil { - return "" - } - return ctx.Addr -} - -func (s *testRegionCacheSuite) TestStoreLabels(c *C) { - testcases := []struct { - storeID uint64 - }{ - { - storeID: s.store1, - }, - { - storeID: s.store2, - }, - } - for _, testcase := range testcases { - c.Log(testcase.storeID) - store := s.cache.getStoreByStoreID(testcase.storeID) - _, err := store.initResolve(s.bo, s.cache) - c.Assert(err, IsNil) - labels := []*metapb.StoreLabel{ - { - Key: "id", - Value: fmt.Sprintf("%v", testcase.storeID), - }, - } - stores := s.cache.getStoresByLabels(labels) - c.Assert(len(stores), Equals, 1) - c.Assert(stores[0].labels, DeepEquals, labels) - } -} - -func (s *testRegionCacheSuite) TestSimple(c *C) { - seed := rand.Uint32() - r := s.getRegion(c, []byte("a")) - c.Assert(r, NotNil) - c.Assert(r.GetID(), Equals, s.region1) - c.Assert(s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0), Equals, s.storeAddr(s.store1)) - c.Assert(s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed), Equals, s.storeAddr(s.store2)) - s.checkCache(c, 1) - c.Assert(r.GetMeta(), DeepEquals, r.meta) - c.Assert(r.GetLeaderPeerID(), Equals, r.meta.Peers[r.getStore().workTiKVIdx].Id) - s.cache.mu.regions[r.VerID()].lastAccess = 0 - r = s.cache.searchCachedRegion([]byte("a"), true) - c.Assert(r, IsNil) -} - -// TestResolveStateTransition verifies store's resolve state transition. For example, -// a newly added store is in unresolved state and will be resolved soon if it's an up store, -// or in tombstone state if it's a tombstone. -func (s *testRegionCacheSuite) TestResolveStateTransition(c *C) { - cache := s.cache - bo := retry.NewNoopBackoff(context.Background()) - - // Check resolving normal stores. The resolve state should be resolved. - for _, storeMeta := range s.cluster.GetAllStores() { - store := cache.getStoreByStoreID(storeMeta.GetId()) - c.Assert(store.getResolveState(), Equals, unresolved) - addr, err := store.initResolve(bo, cache) - c.Assert(err, IsNil) - c.Assert(addr, Equals, storeMeta.GetAddress()) - c.Assert(store.getResolveState(), Equals, resolved) - } - - waitResolve := func(s *Store) { - for i := 0; i < 10; i++ { - if s.getResolveState() != needCheck { - break - } - time.Sleep(50 * time.Millisecond) - } - } - - // Mark the store needCheck. The resolve state should be resolved soon. - store := cache.getStoreByStoreID(s.store1) - store.markNeedCheck(cache.notifyCheckCh) - waitResolve(store) - c.Assert(store.getResolveState(), Equals, resolved) - - // Mark the store needCheck and it becomes a tombstone. The resolve state should be tombstone. - s.cluster.MarkTombstone(s.store1) - store.markNeedCheck(cache.notifyCheckCh) - waitResolve(store) - c.Assert(store.getResolveState(), Equals, tombstone) - s.cluster.StartStore(s.store1) - - // Mark the store needCheck and it's deleted from PD. The resolve state should be tombstone. - cache.clear() - store = cache.getStoreByStoreID(s.store1) - store.initResolve(bo, cache) - c.Assert(store.getResolveState(), Equals, resolved) - storeMeta := s.cluster.GetStore(s.store1) - s.cluster.RemoveStore(s.store1) - store.markNeedCheck(cache.notifyCheckCh) - waitResolve(store) - c.Assert(store.getResolveState(), Equals, tombstone) - s.cluster.AddStore(storeMeta.GetId(), storeMeta.GetAddress(), storeMeta.GetLabels()...) - - // Mark the store needCheck and its address and labels are changed. - // The resolve state should be deleted and a new store is added to the cache. - cache.clear() - store = cache.getStoreByStoreID(s.store1) - store.initResolve(bo, cache) - c.Assert(store.getResolveState(), Equals, resolved) - s.cluster.UpdateStoreAddr(s.store1, store.addr+"0", &metapb.StoreLabel{Key: "k", Value: "v"}) - store.markNeedCheck(cache.notifyCheckCh) - waitResolve(store) - c.Assert(store.getResolveState(), Equals, deleted) - newStore := cache.getStoreByStoreID(s.store1) - c.Assert(newStore.getResolveState(), Equals, resolved) - c.Assert(newStore.addr, Equals, store.addr+"0") - c.Assert(newStore.labels, DeepEquals, []*metapb.StoreLabel{{Key: "k", Value: "v"}}) - - // Check initResolve()ing a tombstone store. The resolve state should be tombstone. - cache.clear() - s.cluster.MarkTombstone(s.store1) - store = cache.getStoreByStoreID(s.store1) - for i := 0; i < 2; i++ { - addr, err := store.initResolve(bo, cache) - c.Assert(err, IsNil) - c.Assert(addr, Equals, "") - c.Assert(store.getResolveState(), Equals, tombstone) - } - s.cluster.StartStore(s.store1) - cache.clear() - - // Check initResolve()ing a dropped store. The resolve state should be tombstone. - cache.clear() - storeMeta = s.cluster.GetStore(s.store1) - s.cluster.RemoveStore(s.store1) - store = cache.getStoreByStoreID(s.store1) - for i := 0; i < 2; i++ { - addr, err := store.initResolve(bo, cache) - c.Assert(err, IsNil) - c.Assert(addr, Equals, "") - c.Assert(store.getResolveState(), Equals, tombstone) - } - s.cluster.AddStore(storeMeta.GetId(), storeMeta.GetAddress(), storeMeta.GetLabels()...) -} - -// TestFilterDownPeersOrPeersOnTombstoneOrDroppedStore verifies the RegionCache filter -// region's down peers and peers on tombstone or dropped stores. RegionCache shouldn't -// report errors in such cases if there are available peers. -func (s *testRegionCacheSuite) TestFilterDownPeersOrPeersOnTombstoneOrDroppedStores(c *C) { - key := []byte("a") - bo := retry.NewBackofferWithVars(context.Background(), 100, nil) - - verifyGetRPCCtx := func(meta *metapb.Region) { - loc, err := s.cache.LocateKey(bo, key) - c.Assert(loc, NotNil) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx, NotNil) - c.Assert(ctx.Meta, DeepEquals, meta) - ctx, err = s.cache.GetTiKVRPCContext(bo, loc.Region, kv.ReplicaReadFollower, rand.Uint32()) - c.Assert(err, IsNil) - c.Assert(ctx, NotNil) - c.Assert(ctx.Meta, DeepEquals, meta) - } - - // When all peers are normal, the cached region should contain all peers. - reg, err := s.cache.findRegionByKey(bo, key, false) - c.Assert(reg, NotNil) - c.Assert(err, IsNil) - regInPD, _ := s.cluster.GetRegion(reg.GetID()) - c.Assert(reg.meta, DeepEquals, regInPD) - c.Assert(len(reg.meta.GetPeers()), Equals, len(reg.getStore().stores)) - verifyGetRPCCtx(reg.meta) - s.checkCache(c, 1) - s.cache.clear() - - // Shouldn't contain the peer on the tombstone store. - s.cluster.MarkTombstone(s.store1) - reg, err = s.cache.findRegionByKey(bo, key, false) - c.Assert(reg, NotNil) - c.Assert(err, IsNil) - c.Assert(len(reg.meta.GetPeers()), Equals, len(regInPD.GetPeers())-1) - c.Assert(len(reg.meta.GetPeers()), Equals, len(reg.getStore().stores)) - for _, peer := range reg.meta.GetPeers() { - c.Assert(peer.GetStoreId(), Not(Equals), s.store1) - } - for _, store := range reg.getStore().stores { - c.Assert(store.storeID, Not(Equals), s.store1) - } - verifyGetRPCCtx(reg.meta) - s.checkCache(c, 1) - s.cache.clear() - s.cluster.StartStore(s.store1) - - // Shouldn't contain the peer on the dropped store. - store := s.cluster.GetStore(s.store1) - s.cluster.RemoveStore(s.store1) - reg, err = s.cache.findRegionByKey(bo, key, false) - c.Assert(reg, NotNil) - c.Assert(err, IsNil) - c.Assert(len(reg.meta.GetPeers()), Equals, len(regInPD.GetPeers())-1) - c.Assert(len(reg.meta.GetPeers()), Equals, len(reg.getStore().stores)) - for _, peer := range reg.meta.GetPeers() { - c.Assert(peer.GetStoreId(), Not(Equals), s.store1) - } - for _, store := range reg.getStore().stores { - c.Assert(store.storeID, Not(Equals), s.store1) - } - verifyGetRPCCtx(reg.meta) - s.checkCache(c, 1) - s.cache.clear() - s.cluster.AddStore(store.GetId(), store.GetAddress(), store.GetLabels()...) - - // Report an error when there's no available peers. - s.cluster.MarkTombstone(s.store1) - s.cluster.MarkTombstone(s.store2) - _, err = s.cache.findRegionByKey(bo, key, false) - c.Assert(err, NotNil) - c.Assert(err.Error(), Matches, ".*no available peers.*") - s.cluster.StartStore(s.store1) - s.cluster.StartStore(s.store2) -} - -func (s *testRegionCacheSuite) TestUpdateLeader(c *C) { - seed := rand.Uint32() - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - // tikv-server reports `NotLeader` - s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, 0) - - r := s.getRegion(c, []byte("a")) - c.Assert(r, NotNil) - c.Assert(r.GetID(), Equals, s.region1) - c.Assert(s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0), Equals, s.storeAddr(s.store2)) - c.Assert(s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed), Equals, s.storeAddr(s.store1)) - - r = s.getRegionWithEndKey(c, []byte("z")) - c.Assert(r, NotNil) - c.Assert(r.GetID(), Equals, s.region1) - c.Assert(s.getAddr(c, []byte("z"), kv.ReplicaReadLeader, 0), Equals, s.storeAddr(s.store2)) - c.Assert(s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed), Equals, s.storeAddr(s.store1)) -} - -func (s *testRegionCacheSuite) TestUpdateLeader2(c *C) { - seed := rand.Uint32() - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - // new store3 becomes leader - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - // tikv-server reports `NotLeader` - s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: peer3, StoreId: store3}, 0) - - // Store3 does not exist in cache, causes a reload from PD. - r := s.getRegion(c, []byte("a")) - c.Assert(r, NotNil) - c.Assert(r.GetID(), Equals, s.region1) - c.Assert(s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0), Equals, s.storeAddr(s.store1)) - follower := s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed) - if seed%2 == 0 { - c.Assert(follower, Equals, s.storeAddr(s.store2)) - } else { - c.Assert(follower, Equals, s.storeAddr(store3)) - } - follower2 := s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed+1) - if (seed+1)%2 == 0 { - c.Assert(follower2, Equals, s.storeAddr(s.store2)) - } else { - c.Assert(follower2, Equals, s.storeAddr(store3)) - } - c.Assert(follower, Not(Equals), follower2) - - // tikv-server notifies new leader to pd-server. - s.cluster.ChangeLeader(s.region1, peer3) - // tikv-server reports `NotLeader` again. - s.cache.UpdateLeader(r.VerID(), &metapb.Peer{Id: peer3, StoreId: store3}, 0) - r = s.getRegion(c, []byte("a")) - c.Assert(r, NotNil) - c.Assert(r.GetID(), Equals, s.region1) - c.Assert(s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0), Equals, s.storeAddr(store3)) - follower = s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed) - if seed%2 == 0 { - c.Assert(follower, Equals, s.storeAddr(s.store1)) - } else { - c.Assert(follower, Equals, s.storeAddr(s.store2)) - } - follower2 = s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed+1) - if (seed+1)%2 == 0 { - c.Assert(follower2, Equals, s.storeAddr(s.store1)) - } else { - c.Assert(follower2, Equals, s.storeAddr(s.store2)) - } - c.Assert(follower, Not(Equals), follower2) -} - -func (s *testRegionCacheSuite) TestUpdateLeader3(c *C) { - seed := rand.Uint32() - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - // store2 becomes leader - s.cluster.ChangeLeader(s.region1, s.peer2) - // store2 gone, store3 becomes leader - s.cluster.RemoveStore(s.store2) - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - // tikv-server notifies new leader to pd-server. - s.cluster.ChangeLeader(s.region1, peer3) - // tikv-server reports `NotLeader`(store2 is the leader) - s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, 0) - - // Store2 does not exist any more, causes a reload from PD. - r := s.getRegion(c, []byte("a")) - c.Assert(err, IsNil) - c.Assert(r, NotNil) - c.Assert(r.GetID(), Equals, s.region1) - loc, err = s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - // return resolved store2 address and send fail - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, seed) - c.Assert(err, IsNil) - c.Assert(ctx.Addr, Equals, "store2") - s.cache.OnSendFail(retry.NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) - s.cache.checkAndResolve(nil, func(*Store) bool { return true }) - s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, 0) - addr := s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0) - c.Assert(addr, Equals, "") - addr = s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0) - c.Assert(addr, Equals, s.storeAddr(store3)) - - addr = s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed) - addr2 := s.getAddr(c, []byte("a"), kv.ReplicaReadFollower, seed+1) - c.Assert(addr, Not(Equals), s.storeAddr(store3)) - c.Assert(addr2, Not(Equals), s.storeAddr(store3)) -} - -func (s *testRegionCacheSuite) TestSendFailedButLeaderNotChange(c *C) { - // 3 nodes and no.1 is leader. - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.ChangeLeader(s.region1, s.peer1) - - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer1) - c.Assert(len(ctx.Meta.Peers), Equals, 3) - - // verify follower to be one of store2 and store3 - seed := rand.Uint32() - ctxFollower1, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - ctxFollower2, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Equals, ctxFollower2.Peer.Id) - - // send fail leader switch to 2 - s.cache.OnSendFail(s.bo, ctx, false, nil) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer2) - - // verify follower to be one of store1 and store3 - ctxFollower1, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - ctxFollower2, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed+1) - c.Assert(err, IsNil) - if (seed+1)%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) - - // access 1 it will return NotLeader, leader back to 2 again - s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, ctx.AccessIdx) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer2) - - // verify follower to be one of store1 and store3 - ctxFollower1, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - ctxFollower2, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed+1) - c.Assert(err, IsNil) - if (seed+1)%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) -} - -func (s *testRegionCacheSuite) TestSendFailedInHibernateRegion(c *C) { - // 3 nodes and no.1 is leader. - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.ChangeLeader(s.region1, s.peer1) - - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer1) - c.Assert(len(ctx.Meta.Peers), Equals, 3) - - // verify follower to be one of store2 and store3 - seed := rand.Uint32() - ctxFollower1, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - ctxFollower2, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Equals, ctxFollower2.Peer.Id) - - // send fail leader switch to 2 - s.cache.OnSendFail(s.bo, ctx, false, nil) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer2) - - // verify follower to be one of store1 and store3 - ctxFollower1, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id == s.peer1 || ctxFollower1.Peer.Id == peer3, IsTrue) - ctxFollower2, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed+1) - c.Assert(err, IsNil) - if (seed+1)%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) - - // access 2, it's in hibernate and return 0 leader, so switch to 3 - s.cache.UpdateLeader(loc.Region, nil, ctx.AccessIdx) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, peer3) - - // verify follower to be one of store1 and store2 - ctxFollower1, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - } - ctxFollower2, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer2) - } - c.Assert(ctxFollower1.Peer.Id, Equals, ctxFollower2.Peer.Id) - - // again peer back to 1 - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - s.cache.UpdateLeader(loc.Region, nil, ctx.AccessIdx) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer1) - - // verify follower to be one of store2 and store3 - ctxFollower1, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - ctxFollower2, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed+1) - c.Assert(err, IsNil) - if (seed+1)%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) -} - -func (s *testRegionCacheSuite) TestSendFailInvalidateRegionsInSameStore(c *C) { - // key range: ['' - 'm' - 'z'] - region2 := s.cluster.AllocID() - newPeers := s.cluster.AllocIDs(2) - s.cluster.Split(s.region1, region2, []byte("m"), newPeers, newPeers[0]) - - // Check the two regions. - loc1, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - c.Assert(loc1.Region.id, Equals, s.region1) - loc2, err := s.cache.LocateKey(s.bo, []byte("x")) - c.Assert(err, IsNil) - c.Assert(loc2.Region.id, Equals, region2) - - // Send fail on region1 - ctx, _ := s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadLeader, 0) - s.checkCache(c, 2) - s.cache.OnSendFail(s.bo, ctx, false, errors.New("test error")) - - // Get region2 cache will get nil then reload. - ctx2, err := s.cache.GetTiKVRPCContext(s.bo, loc2.Region, kv.ReplicaReadLeader, 0) - c.Assert(ctx2, IsNil) - c.Assert(err, IsNil) -} - -func (s *testRegionCacheSuite) TestSendFailEnableForwarding(c *C) { - s.cache.enableForwarding = true - - // key range: ['' - 'm' - 'z'] - region2 := s.cluster.AllocID() - newPeers := s.cluster.AllocIDs(2) - s.cluster.Split(s.region1, region2, []byte("m"), newPeers, newPeers[0]) - - var storeState uint32 = uint32(unreachable) - s.cache.testingKnobs.mockRequestLiveness = func(s *Store, bo *retry.Backoffer) livenessState { - return livenessState(atomic.LoadUint32(&storeState)) - } - - // Check the two regions. - loc1, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - c.Assert(loc1.Region.id, Equals, s.region1) - - // Invoke OnSendFail so that the store will be marked as needForwarding - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx, NotNil) - s.cache.OnSendFail(s.bo, ctx, false, errors.New("test error")) - - // ...then on next retry, proxy will be used - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx, NotNil) - c.Assert(ctx.ProxyStore, NotNil) - c.Assert(ctx.ProxyStore.storeID, Equals, s.store2) - - // Proxy will be also applied to other regions whose leader is on the store - loc2, err := s.cache.LocateKey(s.bo, []byte("x")) - c.Assert(err, IsNil) - c.Assert(loc2.Region.id, Equals, region2) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc2.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx, NotNil) - c.Assert(ctx.ProxyStore, NotNil) - c.Assert(ctx.ProxyStore.storeID, Equals, s.store2) - - // Recover the store - atomic.StoreUint32(&storeState, uint32(reachable)) - // The proxy should be unset after several retries - for retry := 0; retry < 15; retry++ { - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - if ctx.ProxyStore == nil { - break - } - time.Sleep(time.Millisecond * 200) - } - c.Assert(ctx.ProxyStore, IsNil) -} - -func (s *testRegionCacheSuite) TestSendFailedInMultipleNode(c *C) { - // 3 nodes and no.1 is leader. - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.ChangeLeader(s.region1, s.peer1) - - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer1) - c.Assert(len(ctx.Meta.Peers), Equals, 3) - - // verify follower to be one of store2 and store3 - seed := rand.Uint32() - ctxFollower1, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - ctxFollower2, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Equals, ctxFollower2.Peer.Id) - - // send fail leader switch to 2 - s.cache.OnSendFail(s.bo, ctx, false, nil) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer2) - - // verify follower to be one of store1 and store3 - ctxFollower1, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - ctxFollower2, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed+1) - c.Assert(err, IsNil) - if (seed+1)%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) - - // send 2 fail leader switch to 3 - s.cache.OnSendFail(s.bo, ctx, false, nil) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, peer3) - - // verify follower to be one of store1 and store2 - ctxFollower1, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - } - c.Assert(ctxFollower1.Peer.Id == s.peer1 || ctxFollower1.Peer.Id == s.peer2, IsTrue) - ctxFollower2, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer1) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer2) - } - c.Assert(ctxFollower1.Peer.Id, Equals, ctxFollower2.Peer.Id) - - // 3 can be access, so switch to 1 - s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer1, StoreId: s.store1}, ctx.AccessIdx) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer1) - - // verify follower to be one of store2 and store3 - ctxFollower1, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed) - c.Assert(err, IsNil) - if seed%2 == 0 { - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower1.Peer.Id, Equals, peer3) - } - ctxFollower2, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, seed+1) - c.Assert(err, IsNil) - if (seed+1)%2 == 0 { - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer2) - } else { - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - } - c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) -} - -func (s *testRegionCacheSuite) TestLabelSelectorTiKVPeer(c *C) { - dc1Label := []*metapb.StoreLabel{ - { - Key: "zone", - Value: "dc-1", - }, - } - dc2Label := []*metapb.StoreLabel{ - { - Key: "zone", - Value: "dc-2", - }, - } - dc3Label := []*metapb.StoreLabel{ - { - Key: "zone", - Value: "dc-3", - }, - } - s.cluster.UpdateStoreLabels(s.store1, dc1Label) - s.cluster.UpdateStoreLabels(s.store2, dc2Label) - - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.UpdateStoreLabels(store3, dc1Label) - // Region have 3 peer, leader located in dc-1, followers located in dc-1, dc-2 - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - seed := rand.Uint32() - - testcases := []struct { - name string - t kv.ReplicaReadType - labels []*metapb.StoreLabel - expectStoreIDRange map[uint64]struct{} - }{ - { - name: "any Peer,located in dc-1", - t: kv.ReplicaReadMixed, - labels: dc1Label, - expectStoreIDRange: map[uint64]struct{}{ - s.store1: {}, - store3: {}, - }, - }, - { - name: "any Peer,located in dc-2", - t: kv.ReplicaReadMixed, - labels: dc2Label, - expectStoreIDRange: map[uint64]struct{}{ - s.store2: {}, - }, - }, - { - name: "only follower,located in dc-1", - t: kv.ReplicaReadFollower, - labels: dc1Label, - expectStoreIDRange: map[uint64]struct{}{ - store3: {}, - }, - }, - { - name: "only leader, shouldn't consider labels", - t: kv.ReplicaReadLeader, - labels: dc2Label, - expectStoreIDRange: map[uint64]struct{}{ - s.store1: {}, - }, - }, - { - name: "no label matching, fallback to leader", - t: kv.ReplicaReadMixed, - labels: dc3Label, - expectStoreIDRange: map[uint64]struct{}{ - s.store1: {}, - }, - }, - } - - for _, testcase := range testcases { - c.Log(testcase.name) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, testcase.t, seed, WithMatchLabels(testcase.labels)) - c.Assert(err, IsNil) - _, exist := testcase.expectStoreIDRange[ctx.Store.storeID] - c.Assert(exist, Equals, true) - } -} - -func (s *testRegionCacheSuite) TestSplit(c *C) { - seed := rand.Uint32() - r := s.getRegion(c, []byte("x")) - c.Assert(r.GetID(), Equals, s.region1) - c.Assert(s.getAddr(c, []byte("x"), kv.ReplicaReadLeader, 0), Equals, s.storeAddr(s.store1)) - c.Assert(s.getAddr(c, []byte("x"), kv.ReplicaReadFollower, seed), Equals, s.storeAddr(s.store2)) - - // split to ['' - 'm' - 'z'] - region2 := s.cluster.AllocID() - newPeers := s.cluster.AllocIDs(2) - s.cluster.Split(s.region1, region2, []byte("m"), newPeers, newPeers[0]) - - // tikv-server reports `NotInRegion` - s.cache.InvalidateCachedRegion(r.VerID()) - s.checkCache(c, 0) - - r = s.getRegion(c, []byte("x")) - c.Assert(r.GetID(), Equals, region2) - c.Assert(s.getAddr(c, []byte("x"), kv.ReplicaReadLeader, 0), Equals, s.storeAddr(s.store1)) - c.Assert(s.getAddr(c, []byte("x"), kv.ReplicaReadFollower, seed), Equals, s.storeAddr(s.store2)) - s.checkCache(c, 1) - - r = s.getRegionWithEndKey(c, []byte("m")) - c.Assert(r.GetID(), Equals, s.region1) - s.checkCache(c, 2) -} - -func (s *testRegionCacheSuite) TestMerge(c *C) { - // key range: ['' - 'm' - 'z'] - region2 := s.cluster.AllocID() - newPeers := s.cluster.AllocIDs(2) - s.cluster.Split(s.region1, region2, []byte("m"), newPeers, newPeers[0]) - - loc, err := s.cache.LocateKey(s.bo, []byte("x")) - c.Assert(err, IsNil) - c.Assert(loc.Region.id, Equals, region2) - - // merge to single region - s.cluster.Merge(s.region1, region2) - - // tikv-server reports `NotInRegion` - s.cache.InvalidateCachedRegion(loc.Region) - s.checkCache(c, 0) - - loc, err = s.cache.LocateKey(s.bo, []byte("x")) - c.Assert(err, IsNil) - c.Assert(loc.Region.id, Equals, s.region1) - s.checkCache(c, 1) -} - -func (s *testRegionCacheSuite) TestReconnect(c *C) { - seed := rand.Uint32() - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - - // connect tikv-server failed, cause drop cache - s.cache.InvalidateCachedRegion(loc.Region) - - r := s.getRegion(c, []byte("a")) - c.Assert(r, NotNil) - c.Assert(r.GetID(), Equals, s.region1) - c.Assert(s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0), Equals, s.storeAddr(s.store1)) - c.Assert(s.getAddr(c, []byte("x"), kv.ReplicaReadFollower, seed), Equals, s.storeAddr(s.store2)) - s.checkCache(c, 1) -} - -func (s *testRegionCacheSuite) TestRegionEpochAheadOfTiKV(c *C) { - // Create a separated region cache to do this test. - pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster)} - cache := NewRegionCache(pdCli) - defer cache.Close() - - region := createSampleRegion([]byte("k1"), []byte("k2")) - region.meta.Id = 1 - region.meta.RegionEpoch = &metapb.RegionEpoch{Version: 10, ConfVer: 10} - cache.insertRegionToCache(region) - - r1 := metapb.Region{Id: 1, RegionEpoch: &metapb.RegionEpoch{Version: 9, ConfVer: 10}} - r2 := metapb.Region{Id: 1, RegionEpoch: &metapb.RegionEpoch{Version: 10, ConfVer: 9}} - - bo := retry.NewBackofferWithVars(context.Background(), 2000000, nil) - - _, err := cache.OnRegionEpochNotMatch(bo, &RPCContext{Region: region.VerID()}, []*metapb.Region{&r1}) - c.Assert(err, IsNil) - _, err = cache.OnRegionEpochNotMatch(bo, &RPCContext{Region: region.VerID()}, []*metapb.Region{&r2}) - c.Assert(err, IsNil) - c.Assert(bo.ErrorsNum(), Equals, 2) -} - -func (s *testRegionCacheSuite) TestRegionEpochOnTiFlash(c *C) { - // add store3 as tiflash - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store1), &metapb.StoreLabel{Key: "engine", Value: "tiflash"}) - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.ChangeLeader(s.region1, peer3) - - // pre-load region cache - loc1, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - c.Assert(loc1.Region.id, Equals, s.region1) - lctx, err := s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(lctx.Peer.Id, Equals, peer3) - - // epoch-not-match on tiflash - ctxTiFlash, err := s.cache.GetTiFlashRPCContext(s.bo, loc1.Region, true) - c.Assert(err, IsNil) - c.Assert(ctxTiFlash.Peer.Id, Equals, s.peer1) - ctxTiFlash.Peer.Role = metapb.PeerRole_Learner - r := ctxTiFlash.Meta - reqSend := NewRegionRequestSender(s.cache, nil) - regionErr := &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{CurrentRegions: []*metapb.Region{r}}} - reqSend.onRegionError(s.bo, ctxTiFlash, nil, regionErr, nil) - - // check leader read should not go to tiflash - lctx, err = s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(lctx.Peer.Id, Not(Equals), s.peer1) -} - -func (s *testRegionCacheSuite) TestRegionDataNotReady(c *C) { - loc1, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - c.Assert(loc1.Region.id, Equals, s.region1) - testcases := []struct { - scope string - readType kv.ReplicaReadType - expectPeerID uint64 - expectOptsLen int - expectSeed uint32 - }{ - { - scope: oracle.GlobalTxnScope, - readType: kv.ReplicaReadFollower, - expectPeerID: s.peer2, - expectOptsLen: 1, - expectSeed: 1, - }, - { - scope: "local", - readType: kv.ReplicaReadFollower, - expectPeerID: s.peer2, - expectOptsLen: 0, - expectSeed: 1, - }, - } - - for _, testcase := range testcases { - fctx, err := s.cache.GetTiKVRPCContext(s.bo, loc1.Region, testcase.readType, 0) - c.Assert(err, IsNil) - c.Assert(fctx.Peer.Id, Equals, testcase.expectPeerID) - reqSend := NewRegionRequestSender(s.cache, nil) - regionErr := &errorpb.Error{DataIsNotReady: &errorpb.DataIsNotReady{}} - var opts []StoreSelectorOption - seed := uint32(0) - s.bo.Reset() - req := &tikvrpc.Request{TxnScope: testcase.scope, Context: kvrpcpb.Context{StaleRead: true}, ReplicaReadSeed: &seed} - retry, err := reqSend.onRegionError(s.bo, fctx, req, regionErr, &opts) - c.Assert(err, IsNil) - c.Assert(retry, IsTrue) - c.Assert(len(opts), Equals, testcase.expectOptsLen) - c.Assert(*req.GetReplicaReadSeed(), Equals, testcase.expectSeed) - } -} - -const regionSplitKeyFormat = "t%08d" - -func createClusterWithStoresAndRegions(regionCnt, storeCount int) *mocktikv.Cluster { - cluster := mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) - _, _, regionID, _ := mocktikv.BootstrapWithMultiStores(cluster, storeCount) - for i := 0; i < regionCnt; i++ { - rawKey := []byte(fmt.Sprintf(regionSplitKeyFormat, i)) - ids := cluster.AllocIDs(4) - // Make leaders equally distributed on the 3 stores. - storeID := ids[0] - peerIDs := ids[1:] - leaderPeerID := peerIDs[i%3] - cluster.SplitRaw(regionID, storeID, rawKey, peerIDs, leaderPeerID) - regionID = ids[0] - } - return cluster -} - -func loadRegionsToCache(cache *RegionCache, regionCnt int) { - for i := 0; i < regionCnt; i++ { - rawKey := []byte(fmt.Sprintf(regionSplitKeyFormat, i)) - cache.LocateKey(retry.NewBackofferWithVars(context.Background(), 1, nil), rawKey) - } -} - -func (s *testRegionCacheSuite) TestListRegionIDsInCache(c *C) { - // ['' - 'm' - 'z'] - region2 := s.cluster.AllocID() - newPeers := s.cluster.AllocIDs(2) - s.cluster.Split(s.region1, region2, []byte("m"), newPeers, newPeers[0]) - - regionIDs, err := s.cache.ListRegionIDsInKeyRange(s.bo, []byte("a"), []byte("z")) - c.Assert(err, IsNil) - c.Assert(regionIDs, DeepEquals, []uint64{s.region1, region2}) - regionIDs, err = s.cache.ListRegionIDsInKeyRange(s.bo, []byte("m"), []byte("z")) - c.Assert(err, IsNil) - c.Assert(regionIDs, DeepEquals, []uint64{region2}) - - regionIDs, err = s.cache.ListRegionIDsInKeyRange(s.bo, []byte("a"), []byte("m")) - c.Assert(err, IsNil) - c.Assert(regionIDs, DeepEquals, []uint64{s.region1, region2}) -} - -func (s *testRegionCacheSuite) TestScanRegions(c *C) { - // Split at "a", "b", "c", "d" - regions := s.cluster.AllocIDs(4) - regions = append([]uint64{s.region1}, regions...) - - peers := [][]uint64{{s.peer1, s.peer2}} - for i := 0; i < 4; i++ { - peers = append(peers, s.cluster.AllocIDs(2)) - } - - for i := 0; i < 4; i++ { - s.cluster.Split(regions[i], regions[i+1], []byte{'a' + byte(i)}, peers[i+1], peers[i+1][0]) - } - - scannedRegions, err := s.cache.scanRegions(s.bo, []byte(""), nil, 100) - c.Assert(err, IsNil) - c.Assert(len(scannedRegions), Equals, 5) - for i := 0; i < 5; i++ { - r := scannedRegions[i] - _, p, _, _ := r.WorkStorePeer(r.getStore()) - - c.Assert(r.meta.Id, Equals, regions[i]) - c.Assert(p.Id, Equals, peers[i][0]) - } - - scannedRegions, err = s.cache.scanRegions(s.bo, []byte("a"), nil, 3) - c.Assert(err, IsNil) - c.Assert(len(scannedRegions), Equals, 3) - for i := 1; i < 4; i++ { - r := scannedRegions[i-1] - _, p, _, _ := r.WorkStorePeer(r.getStore()) - - c.Assert(r.meta.Id, Equals, regions[i]) - c.Assert(p.Id, Equals, peers[i][0]) - } - - scannedRegions, err = s.cache.scanRegions(s.bo, []byte("a1"), nil, 1) - c.Assert(err, IsNil) - c.Assert(len(scannedRegions), Equals, 1) - - r0 := scannedRegions[0] - _, p0, _, _ := r0.WorkStorePeer(r0.getStore()) - c.Assert(r0.meta.Id, Equals, regions[1]) - c.Assert(p0.Id, Equals, peers[1][0]) - - // Test region with no leader - s.cluster.GiveUpLeader(regions[1]) - s.cluster.GiveUpLeader(regions[3]) - scannedRegions, err = s.cache.scanRegions(s.bo, []byte(""), nil, 5) - c.Assert(err, IsNil) - for i := 0; i < 3; i++ { - r := scannedRegions[i] - _, p, _, _ := r.WorkStorePeer(r.getStore()) - - c.Assert(r.meta.Id, Equals, regions[i*2]) - c.Assert(p.Id, Equals, peers[i*2][0]) - } -} - -func (s *testRegionCacheSuite) TestBatchLoadRegions(c *C) { - // Split at "a", "b", "c", "d" - regions := s.cluster.AllocIDs(4) - regions = append([]uint64{s.region1}, regions...) - - peers := [][]uint64{{s.peer1, s.peer2}} - for i := 0; i < 4; i++ { - peers = append(peers, s.cluster.AllocIDs(2)) - } - - for i := 0; i < 4; i++ { - s.cluster.Split(regions[i], regions[i+1], []byte{'a' + byte(i)}, peers[i+1], peers[i+1][0]) - } - - testCases := []struct { - startKey []byte - endKey []byte - limit int - expectKey []byte - expectRegions []uint64 - }{ - {[]byte(""), []byte("a"), 1, []byte("a"), []uint64{regions[0]}}, - {[]byte("a"), []byte("b1"), 2, []byte("c"), []uint64{regions[1], regions[2]}}, - {[]byte("a1"), []byte("d"), 2, []byte("c"), []uint64{regions[1], regions[2]}}, - {[]byte("c"), []byte("c1"), 2, nil, []uint64{regions[3]}}, - {[]byte("d"), nil, 2, nil, []uint64{regions[4]}}, - } - - for _, tc := range testCases { - key, err := s.cache.BatchLoadRegionsFromKey(s.bo, tc.startKey, tc.limit) - c.Assert(err, IsNil) - if tc.expectKey != nil { - c.Assert(key, DeepEquals, tc.expectKey) - } else { - c.Assert(key, HasLen, 0) - } - loadRegions, err := s.cache.BatchLoadRegionsWithKeyRange(s.bo, tc.startKey, tc.endKey, tc.limit) - c.Assert(err, IsNil) - c.Assert(loadRegions, HasLen, len(tc.expectRegions)) - for i := range loadRegions { - c.Assert(loadRegions[i].GetID(), Equals, tc.expectRegions[i]) - } - } - - s.checkCache(c, len(regions)) -} - -func (s *testRegionCacheSuite) TestFollowerReadFallback(c *C) { - // 3 nodes and no.1 is leader. - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.ChangeLeader(s.region1, s.peer1) - - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer1) - c.Assert(len(ctx.Meta.Peers), Equals, 3) - - // verify follower to be store2 and store3 - ctxFollower1, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, 0) - c.Assert(err, IsNil) - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - ctxFollower2, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, 1) - c.Assert(err, IsNil) - c.Assert(ctxFollower2.Peer.Id, Equals, peer3) - c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) - - // send fail on store2, next follower read is going to fallback to store3 - s.cache.OnSendFail(s.bo, ctxFollower1, false, errors.New("test error")) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, peer3) -} - -func (s *testRegionCacheSuite) TestMixedReadFallback(c *C) { - // 3 nodes and no.1 is leader. - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.ChangeLeader(s.region1, s.peer1) - - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer1) - c.Assert(len(ctx.Meta.Peers), Equals, 3) - - // verify follower to be store1, store2 and store3 - ctxFollower1, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadMixed, 0) - c.Assert(err, IsNil) - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer1) - - ctxFollower2, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadMixed, 1) - c.Assert(err, IsNil) - c.Assert(ctxFollower2.Peer.Id, Equals, s.peer2) - - ctxFollower3, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadMixed, 2) - c.Assert(err, IsNil) - c.Assert(ctxFollower3.Peer.Id, Equals, peer3) - - // send fail on store2, next follower read is going to fallback to store3 - s.cache.OnSendFail(s.bo, ctxFollower1, false, errors.New("test error")) - ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadMixed, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.Id, Equals, s.peer2) -} - -func (s *testRegionCacheSuite) TestFollowerMeetEpochNotMatch(c *C) { - // 3 nodes and no.1 is region1 leader. - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.ChangeLeader(s.region1, s.peer1) - - // Check the two regions. - loc1, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - c.Assert(loc1.Region.id, Equals, s.region1) - - reqSend := NewRegionRequestSender(s.cache, nil) - - // follower read failed on store2 - followReqSeed := uint32(0) - ctxFollower1, err := s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadFollower, followReqSeed) - c.Assert(err, IsNil) - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer2) - c.Assert(ctxFollower1.Store.storeID, Equals, s.store2) - - regionErr := &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}} - reqSend.onRegionError(s.bo, ctxFollower1, &tikvrpc.Request{ReplicaReadSeed: &followReqSeed}, regionErr, nil) - c.Assert(followReqSeed, Equals, uint32(1)) - - regionErr = &errorpb.Error{RegionNotFound: &errorpb.RegionNotFound{}} - reqSend.onRegionError(s.bo, ctxFollower1, &tikvrpc.Request{ReplicaReadSeed: &followReqSeed}, regionErr, nil) - c.Assert(followReqSeed, Equals, uint32(2)) -} - -func (s *testRegionCacheSuite) TestMixedMeetEpochNotMatch(c *C) { - // 3 nodes and no.1 is region1 leader. - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(store3)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.ChangeLeader(s.region1, s.peer1) - - // Check the two regions. - loc1, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - c.Assert(loc1.Region.id, Equals, s.region1) - - reqSend := NewRegionRequestSender(s.cache, nil) - - // follower read failed on store1 - followReqSeed := uint32(0) - ctxFollower1, err := s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadMixed, followReqSeed) - c.Assert(err, IsNil) - c.Assert(ctxFollower1.Peer.Id, Equals, s.peer1) - c.Assert(ctxFollower1.Store.storeID, Equals, s.store1) - - regionErr := &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}} - reqSend.onRegionError(s.bo, ctxFollower1, &tikvrpc.Request{ReplicaReadSeed: &followReqSeed}, regionErr, nil) - c.Assert(followReqSeed, Equals, uint32(1)) -} - -func (s *testRegionCacheSuite) TestPeersLenChange(c *C) { - // 2 peers [peer1, peer2] and let peer2 become leader - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, 0) - - // current leader is peer2 in [peer1, peer2] - loc, err = s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - c.Assert(ctx.Peer.StoreId, Equals, s.store2) - - // simulate peer1 became down in kv heartbeat and loaded before response back. - cpMeta := &metapb.Region{ - Id: ctx.Meta.Id, - StartKey: ctx.Meta.StartKey, - EndKey: ctx.Meta.EndKey, - RegionEpoch: ctx.Meta.RegionEpoch, - Peers: make([]*metapb.Peer, len(ctx.Meta.Peers)), - } - copy(cpMeta.Peers, ctx.Meta.Peers) - cpRegion := &pd.Region{ - Meta: cpMeta, - DownPeers: []*metapb.Peer{{Id: s.peer1, StoreId: s.store1}}, - } - filterUnavailablePeers(cpRegion) - region := &Region{meta: cpRegion.Meta} - err = region.init(s.bo, s.cache) - c.Assert(err, IsNil) - s.cache.insertRegionToCache(region) - - // OnSendFail should not panic - s.cache.OnSendFail(retry.NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) -} - -func createSampleRegion(startKey, endKey []byte) *Region { - return &Region{ - meta: &metapb.Region{ - StartKey: startKey, - EndKey: endKey, - }, - } -} - -func (s *testRegionCacheSuite) TestContains(c *C) { - c.Assert(createSampleRegion(nil, nil).Contains([]byte{}), IsTrue) - c.Assert(createSampleRegion(nil, nil).Contains([]byte{10}), IsTrue) - c.Assert(createSampleRegion([]byte{10}, nil).Contains([]byte{}), IsFalse) - c.Assert(createSampleRegion([]byte{10}, nil).Contains([]byte{9}), IsFalse) - c.Assert(createSampleRegion([]byte{10}, nil).Contains([]byte{10}), IsTrue) - c.Assert(createSampleRegion(nil, []byte{10}).Contains([]byte{}), IsTrue) - c.Assert(createSampleRegion(nil, []byte{10}).Contains([]byte{9}), IsTrue) - c.Assert(createSampleRegion(nil, []byte{10}).Contains([]byte{10}), IsFalse) - c.Assert(createSampleRegion([]byte{10}, []byte{20}).Contains([]byte{}), IsFalse) - c.Assert(createSampleRegion([]byte{10}, []byte{20}).Contains([]byte{15}), IsTrue) - c.Assert(createSampleRegion([]byte{10}, []byte{20}).Contains([]byte{30}), IsFalse) -} - -func (s *testRegionCacheSuite) TestContainsByEnd(c *C) { - c.Assert(createSampleRegion(nil, nil).ContainsByEnd([]byte{}), IsFalse) - c.Assert(createSampleRegion(nil, nil).ContainsByEnd([]byte{10}), IsTrue) - c.Assert(createSampleRegion([]byte{10}, nil).ContainsByEnd([]byte{}), IsFalse) - c.Assert(createSampleRegion([]byte{10}, nil).ContainsByEnd([]byte{10}), IsFalse) - c.Assert(createSampleRegion([]byte{10}, nil).ContainsByEnd([]byte{11}), IsTrue) - c.Assert(createSampleRegion(nil, []byte{10}).ContainsByEnd([]byte{}), IsFalse) - c.Assert(createSampleRegion(nil, []byte{10}).ContainsByEnd([]byte{10}), IsTrue) - c.Assert(createSampleRegion(nil, []byte{10}).ContainsByEnd([]byte{11}), IsFalse) - c.Assert(createSampleRegion([]byte{10}, []byte{20}).ContainsByEnd([]byte{}), IsFalse) - c.Assert(createSampleRegion([]byte{10}, []byte{20}).ContainsByEnd([]byte{15}), IsTrue) - c.Assert(createSampleRegion([]byte{10}, []byte{20}).ContainsByEnd([]byte{30}), IsFalse) -} - -func (s *testRegionCacheSuite) TestSwitchPeerWhenNoLeader(c *C) { - var prevCtx *RPCContext - for i := 0; i <= len(s.cluster.GetAllStores()); i++ { - loc, err := s.cache.LocateKey(s.bo, []byte("a")) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) - c.Assert(err, IsNil) - if prevCtx == nil { - c.Assert(i, Equals, 0) - } else { - c.Assert(ctx.AccessIdx, Not(Equals), prevCtx.AccessIdx) - c.Assert(ctx.Peer, Not(DeepEquals), prevCtx.Peer) - } - s.cache.InvalidateCachedRegionWithReason(loc.Region, NoLeader) - c.Assert(s.cache.GetCachedRegionWithRLock(loc.Region).invalidReason, Equals, NoLeader) - prevCtx = ctx - } -} - -func BenchmarkOnRequestFail(b *testing.B) { - /* - This benchmark simulate many concurrent requests call OnSendRequestFail method - after failed on a store, validate that on this scene, requests don't get blocked on the - RegionCache lock. - */ - regionCnt, storeCount := 998, 3 - cluster := createClusterWithStoresAndRegions(regionCnt, storeCount) - cache := NewRegionCache(mocktikv.NewPDClient(cluster)) - defer cache.Close() - loadRegionsToCache(cache, regionCnt) - bo := retry.NewBackofferWithVars(context.Background(), 1, nil) - loc, err := cache.LocateKey(bo, []byte{}) - if err != nil { - b.Fatal(err) - } - region := cache.getRegionByIDFromCache(loc.Region.id) - b.ResetTimer() - regionStore := region.getStore() - store, peer, accessIdx, _ := region.WorkStorePeer(regionStore) - b.RunParallel(func(pb *testing.PB) { - for pb.Next() { - rpcCtx := &RPCContext{ - Region: loc.Region, - Meta: region.meta, - AccessIdx: accessIdx, - Peer: peer, - Store: store, - AccessMode: tiKVOnly, - } - r := cache.GetCachedRegionWithRLock(rpcCtx.Region) - if r != nil { - r.getStore().switchNextTiKVPeer(r, rpcCtx.AccessIdx) - } - } - }) - if len(cache.mu.regions) != regionCnt*2/3 { - b.Fatal(len(cache.mu.regions)) - } -} diff --git a/store/tikv/locate/region_request.go b/store/tikv/locate/region_request.go deleted file mode 100644 index ab7e06a85a146..0000000000000 --- a/store/tikv/locate/region_request.go +++ /dev/null @@ -1,1102 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package locate - -import ( - "bytes" - "context" - "fmt" - "strconv" - "sync" - "sync/atomic" - "time" - - "go.uber.org/zap" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" - - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/errorpb" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/tikv/client" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" -) - -// shuttingDown is a flag to indicate tidb-server is exiting (Ctrl+C signal -// receved for example). If this flag is set, tikv client should not retry on -// network error because tidb-server expect tikv client to exit as soon as possible. -var shuttingDown uint32 - -// StoreShuttingDown atomically stores ShuttingDown into v. -func StoreShuttingDown(v uint32) { - atomic.StoreUint32(&shuttingDown, v) -} - -// LoadShuttingDown atomically loads ShuttingDown. -func LoadShuttingDown() uint32 { - return atomic.LoadUint32(&shuttingDown) -} - -// RegionRequestSender sends KV/Cop requests to tikv server. It handles network -// errors and some region errors internally. -// -// Typically, a KV/Cop request is bind to a region, all keys that are involved -// in the request should be located in the region. -// The sending process begins with looking for the address of leader store's -// address of the target region from cache, and the request is then sent to the -// destination tikv server over TCP connection. -// If region is updated, can be caused by leader transfer, region split, region -// merge, or region balance, tikv server may not able to process request and -// send back a RegionError. -// RegionRequestSender takes care of errors that does not relevant to region -// range, such as 'I/O timeout', 'NotLeader', and 'ServerIsBusy'. If fails to -// send the request to all replicas, a fake rregion error may be returned. -// Caller which receives the error should retry the request. -// -// For other region errors, since region range have changed, the request may need to -// split, so we simply return the error to caller. -type RegionRequestSender struct { - regionCache *RegionCache - client client.Client - storeAddr string - rpcError error - leaderReplicaSelector *replicaSelector - failStoreIDs map[uint64]struct{} - failProxyStoreIDs map[uint64]struct{} - RegionRequestRuntimeStats -} - -// RegionRequestRuntimeStats records the runtime stats of send region requests. -type RegionRequestRuntimeStats struct { - Stats map[tikvrpc.CmdType]*RPCRuntimeStats -} - -// NewRegionRequestRuntimeStats returns a new RegionRequestRuntimeStats. -func NewRegionRequestRuntimeStats() RegionRequestRuntimeStats { - return RegionRequestRuntimeStats{ - Stats: make(map[tikvrpc.CmdType]*RPCRuntimeStats), - } -} - -// RPCRuntimeStats indicates the RPC request count and consume time. -type RPCRuntimeStats struct { - Count int64 - // Send region request consume time. - Consume int64 -} - -// String implements fmt.Stringer interface. -func (r *RegionRequestRuntimeStats) String() string { - var buf bytes.Buffer - for k, v := range r.Stats { - if buf.Len() > 0 { - buf.WriteByte(',') - } - buf.WriteString(fmt.Sprintf("%s:{num_rpc:%d, total_time:%s}", k.String(), v.Count, util.FormatDuration(time.Duration(v.Consume)))) - } - return buf.String() -} - -// Clone returns a copy of itself. -func (r *RegionRequestRuntimeStats) Clone() RegionRequestRuntimeStats { - newRs := NewRegionRequestRuntimeStats() - for cmd, v := range r.Stats { - newRs.Stats[cmd] = &RPCRuntimeStats{ - Count: v.Count, - Consume: v.Consume, - } - } - return newRs -} - -// Merge merges other RegionRequestRuntimeStats. -func (r *RegionRequestRuntimeStats) Merge(rs RegionRequestRuntimeStats) { - for cmd, v := range rs.Stats { - stat, ok := r.Stats[cmd] - if !ok { - r.Stats[cmd] = &RPCRuntimeStats{ - Count: v.Count, - Consume: v.Consume, - } - continue - } - stat.Count += v.Count - stat.Consume += v.Consume - } -} - -// RecordRegionRequestRuntimeStats records request runtime stats. -func RecordRegionRequestRuntimeStats(stats map[tikvrpc.CmdType]*RPCRuntimeStats, cmd tikvrpc.CmdType, d time.Duration) { - stat, ok := stats[cmd] - if !ok { - stats[cmd] = &RPCRuntimeStats{ - Count: 1, - Consume: int64(d), - } - return - } - stat.Count++ - stat.Consume += int64(d) -} - -// NewRegionRequestSender creates a new sender. -func NewRegionRequestSender(regionCache *RegionCache, client client.Client) *RegionRequestSender { - return &RegionRequestSender{ - regionCache: regionCache, - client: client, - } -} - -// GetRegionCache returns the region cache. -func (s *RegionRequestSender) GetRegionCache() *RegionCache { - return s.regionCache -} - -// GetClient returns the RPC client. -func (s *RegionRequestSender) GetClient() client.Client { - return s.client -} - -// SetStoreAddr specifies the dest store address. -func (s *RegionRequestSender) SetStoreAddr(addr string) { - s.storeAddr = addr -} - -// GetStoreAddr returns the dest store address. -func (s *RegionRequestSender) GetStoreAddr() string { - return s.storeAddr -} - -// GetRPCError returns the RPC error. -func (s *RegionRequestSender) GetRPCError() error { - return s.rpcError -} - -// SetRPCError rewrite the rpc error. -func (s *RegionRequestSender) SetRPCError(err error) { - s.rpcError = err -} - -// SendReq sends a request to tikv server. If fails to send the request to all replicas, -// a fake region error may be returned. Caller which receives the error should retry the request. -func (s *RegionRequestSender) SendReq(bo *retry.Backoffer, req *tikvrpc.Request, regionID RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) { - resp, _, err := s.SendReqCtx(bo, req, regionID, timeout, tikvrpc.TiKV) - return resp, err -} - -type replica struct { - store *Store - peer *metapb.Peer - epoch uint32 - attempts int -} - -type replicaSelector struct { - regionCache *RegionCache - region *Region - // replicas contains all TiKV replicas for now and the leader is at the - // head of the slice. - replicas []*replica - // nextReplicaIdx points to the candidate for the next attempt. - nextReplicaIdx int -} - -func newReplicaSelector(regionCache *RegionCache, regionID RegionVerID) (*replicaSelector, error) { - cachedRegion := regionCache.GetCachedRegionWithRLock(regionID) - if cachedRegion == nil || !cachedRegion.isValid() { - return nil, nil - } - regionStore := cachedRegion.getStore() - replicas := make([]*replica, 0, regionStore.accessStoreNum(tiKVOnly)) - for _, storeIdx := range regionStore.accessIndex[tiKVOnly] { - replicas = append(replicas, &replica{ - store: regionStore.stores[storeIdx], - peer: cachedRegion.meta.Peers[storeIdx], - epoch: regionStore.storeEpochs[storeIdx], - attempts: 0, - }) - } - // Move the leader to the first slot. - replicas[regionStore.workTiKVIdx], replicas[0] = replicas[0], replicas[regionStore.workTiKVIdx] - return &replicaSelector{ - regionCache, - cachedRegion, - replicas, - 0, - }, nil -} - -// isExhausted returns true if runs out of all replicas. -func (s *replicaSelector) isExhausted() bool { - return s.nextReplicaIdx >= len(s.replicas) -} - -func (s *replicaSelector) nextReplica() *replica { - if s.isExhausted() { - return nil - } - return s.replicas[s.nextReplicaIdx] -} - -const maxReplicaAttempt = 10 - -// next creates the RPCContext of the current candidate replica. -// It returns a SendError if runs out of all replicas or the cached region is invalidated. -func (s *replicaSelector) next(bo *retry.Backoffer) (*RPCContext, error) { - for { - if !s.region.isValid() { - metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("invalid").Inc() - return nil, nil - } - if s.isExhausted() { - metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("exhausted").Inc() - s.invalidateRegion() - return nil, nil - } - replica := s.replicas[s.nextReplicaIdx] - s.nextReplicaIdx++ - - // Limit the max attempts of each replica to prevent endless retry. - if replica.attempts >= maxReplicaAttempt { - continue - } - replica.attempts++ - - storeFailEpoch := atomic.LoadUint32(&replica.store.epoch) - if storeFailEpoch != replica.epoch { - // TODO(youjiali1995): Is it necessary to invalidate the region? - metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("stale_store").Inc() - s.invalidateRegion() - return nil, nil - } - addr, err := s.regionCache.getStoreAddr(bo, s.region, replica.store) - if err == nil && len(addr) != 0 { - return &RPCContext{ - Region: s.region.VerID(), - Meta: s.region.meta, - Peer: replica.peer, - Store: replica.store, - Addr: addr, - AccessMode: tiKVOnly, - TiKVNum: len(s.replicas), - }, nil - } - } -} - -func (s *replicaSelector) onSendFailure(bo *retry.Backoffer, err error) { - metrics.RegionCacheCounterWithSendFail.Inc() - replica := s.replicas[s.nextReplicaIdx-1] - if replica.store.requestLiveness(bo, s.regionCache) == reachable { - s.rewind() - return - } - - store := replica.store - // invalidate regions in store. - if atomic.CompareAndSwapUint32(&store.epoch, replica.epoch, replica.epoch+1) { - logutil.BgLogger().Info("mark store's regions need be refill", zap.Uint64("id", store.storeID), zap.String("addr", store.addr), zap.Error(err)) - metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() - // schedule a store addr resolve. - store.markNeedCheck(s.regionCache.notifyCheckCh) - } - // TODO(youjiali1995): It's not necessary, but some tests depend on it and it's not easy to fix. - if s.isExhausted() { - s.region.scheduleReload() - } -} - -// OnSendSuccess updates the leader of the cached region since the replicaSelector -// is only used for leader request. It's called when the request is sent to the -// replica successfully. -func (s *replicaSelector) OnSendSuccess() { - // The successful replica is not at the head of replicas which means it's not the - // leader in the cached region, so update leader. - if s.nextReplicaIdx-1 != 0 { - leader := s.replicas[s.nextReplicaIdx-1].peer - if !s.regionCache.switchWorkLeaderToPeer(s.region, leader) { - panic("the store must exist") - } - } -} - -func (s *replicaSelector) rewind() { - s.nextReplicaIdx-- -} - -// updateLeader updates the leader of the cached region. -// If the leader peer isn't found in the region, the region will be invalidated. -func (s *replicaSelector) updateLeader(leader *metapb.Peer) { - if leader == nil { - return - } - for i, replica := range s.replicas { - if isSamePeer(replica.peer, leader) { - if i < s.nextReplicaIdx { - s.nextReplicaIdx-- - } - // Move the leader replica to the front of candidates. - s.replicas[i], s.replicas[s.nextReplicaIdx] = s.replicas[s.nextReplicaIdx], s.replicas[i] - if s.replicas[s.nextReplicaIdx].attempts == maxReplicaAttempt { - // Give the replica one more chance and because the current replica is skipped, it - // won't result in infinite retry. - s.replicas[s.nextReplicaIdx].attempts = maxReplicaAttempt - 1 - } - // Update the workTiKVIdx so that following requests can be sent to the leader immediately. - if !s.regionCache.switchWorkLeaderToPeer(s.region, leader) { - panic("the store must exist") - } - logutil.BgLogger().Debug("switch region leader to specific leader due to kv return NotLeader", - zap.Uint64("regionID", s.region.GetID()), - zap.Uint64("leaderStoreID", leader.GetStoreId())) - return - } - } - // Invalidate the region since the new leader is not in the cached version. - s.region.invalidate(StoreNotFound) -} - -func (s *replicaSelector) invalidateRegion() { - if s.region != nil { - s.region.invalidate(Other) - } -} - -func (s *RegionRequestSender) getRPCContext( - bo *retry.Backoffer, - req *tikvrpc.Request, - regionID RegionVerID, - et tikvrpc.EndpointType, - opts ...StoreSelectorOption, -) (*RPCContext, error) { - switch et { - case tikvrpc.TiKV: - // Now only requests sent to the replica leader will use the replica selector to get - // the RPC context. - // TODO(youjiali1995): make all requests use the replica selector. - if !s.regionCache.enableForwarding && req.ReplicaReadType == kv.ReplicaReadLeader { - if s.leaderReplicaSelector == nil { - selector, err := newReplicaSelector(s.regionCache, regionID) - if selector == nil || err != nil { - return nil, err - } - s.leaderReplicaSelector = selector - } - return s.leaderReplicaSelector.next(bo) - } - - var seed uint32 - if req.ReplicaReadSeed != nil { - seed = *req.ReplicaReadSeed - } - return s.regionCache.GetTiKVRPCContext(bo, regionID, req.ReplicaReadType, seed, opts...) - case tikvrpc.TiFlash: - return s.regionCache.GetTiFlashRPCContext(bo, regionID, true) - case tikvrpc.TiDB: - return &RPCContext{Addr: s.storeAddr}, nil - default: - return nil, errors.Errorf("unsupported storage type: %v", et) - } -} - -func (s *RegionRequestSender) reset() { - s.leaderReplicaSelector = nil - s.failStoreIDs = nil - s.failProxyStoreIDs = nil -} - -// IsFakeRegionError returns true if err is fack region error. -func IsFakeRegionError(err *errorpb.Error) bool { - return err != nil && err.GetEpochNotMatch() != nil && len(err.GetEpochNotMatch().CurrentRegions) == 0 -} - -// SendReqCtx sends a request to tikv server and return response and RPCCtx of this RPC. -func (s *RegionRequestSender) SendReqCtx( - bo *retry.Backoffer, - req *tikvrpc.Request, - regionID RegionVerID, - timeout time.Duration, - et tikvrpc.EndpointType, - opts ...StoreSelectorOption, -) ( - resp *tikvrpc.Response, - rpcCtx *RPCContext, - err error, -) { - if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("regionRequest.SendReqCtx", opentracing.ChildOf(span.Context())) - defer span1.Finish() - bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) - } - - if val, err := util.EvalFailpoint("tikvStoreSendReqResult"); err == nil { - switch val.(string) { - case "timeout": - return nil, nil, errors.New("timeout") - case "GCNotLeader": - if req.Type == tikvrpc.CmdGC { - return &tikvrpc.Response{ - Resp: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}, - }, nil, nil - } - case "GCServerIsBusy": - if req.Type == tikvrpc.CmdGC { - return &tikvrpc.Response{ - Resp: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}}}, - }, nil, nil - } - case "requestTiDBStoreError": - if et == tikvrpc.TiDB { - return nil, nil, tikverr.ErrTiKVServerTimeout - } - case "requestTiFlashError": - if et == tikvrpc.TiFlash { - return nil, nil, tikverr.ErrTiFlashServerTimeout - } - } - } - - // If the MaxExecutionDurationMs is not set yet, we set it to be the RPC timeout duration - // so TiKV can give up the requests whose response TiDB cannot receive due to timeout. - if req.Context.MaxExecutionDurationMs == 0 { - req.Context.MaxExecutionDurationMs = uint64(timeout.Milliseconds()) - } - - s.reset() - tryTimes := 0 - defer func() { - if tryTimes > 0 { - metrics.TiKVRequestRetryTimesHistogram.Observe(float64(tryTimes)) - } - }() - for { - if (tryTimes > 0) && (tryTimes%100 == 0) { - logutil.Logger(bo.GetCtx()).Warn("retry", zap.Uint64("region", regionID.GetID()), zap.Int("times", tryTimes)) - } - - rpcCtx, err = s.getRPCContext(bo, req, regionID, et, opts...) - if err != nil { - return nil, nil, err - } - if rpcCtx != nil { - rpcCtx.tryTimes = tryTimes - } - - if _, err := util.EvalFailpoint("invalidCacheAndRetry"); err == nil { - // cooperate with github.com/pingcap/tidb/store/gcworker/setGcResolveMaxBackoff - if c := bo.GetCtx().Value("injectedBackoff"); c != nil { - resp, err = tikvrpc.GenRegionErrorResp(req, &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}) - return resp, nil, err - } - } - if rpcCtx == nil { - // TODO(youjiali1995): remove it when using the replica selector for all requests. - // If the region is not found in cache, it must be out - // of date and already be cleaned up. We can skip the - // RPC by returning RegionError directly. - - // TODO: Change the returned error to something like "region missing in cache", - // and handle this error like EpochNotMatch, which means to re-split the request and retry. - logutil.Logger(bo.GetCtx()).Debug("throwing pseudo region error due to region not found in cache", zap.Stringer("region", ®ionID)) - resp, err = tikvrpc.GenRegionErrorResp(req, &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}) - return resp, nil, err - } - - logutil.Eventf(bo.GetCtx(), "send %s request to region %d at %s", req.Type, regionID.id, rpcCtx.Addr) - s.storeAddr = rpcCtx.Addr - var retry bool - resp, retry, err = s.sendReqToRegion(bo, rpcCtx, req, timeout) - if err != nil { - return nil, nil, errors.Trace(err) - } - - // recheck whether the session/query is killed during the Next() - boVars := bo.GetVars() - if boVars != nil && boVars.Killed != nil && atomic.LoadUint32(boVars.Killed) == 1 { - return nil, nil, tikverr.ErrQueryInterrupted - } - if val, err := util.EvalFailpoint("mockRetrySendReqToRegion"); err == nil { - if val.(bool) { - retry = true - } - } - if retry { - tryTimes++ - continue - } - - var regionErr *errorpb.Error - regionErr, err = resp.GetRegionError() - if err != nil { - return nil, nil, errors.Trace(err) - } - if regionErr != nil { - retry, err = s.onRegionError(bo, rpcCtx, req, regionErr, &opts) - if err != nil { - return nil, nil, errors.Trace(err) - } - if retry { - tryTimes++ - continue - } - } else { - // Clear the RPC Error since the request is evaluated successfully on a store. - s.rpcError = nil - if s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.OnSendSuccess() - } - } - return resp, rpcCtx, nil - } -} - -// RPCCancellerCtxKey is context key attach rpc send cancelFunc collector to ctx. -type RPCCancellerCtxKey struct{} - -// RPCCanceller is rpc send cancelFunc collector. -type RPCCanceller struct { - sync.Mutex - allocID int - cancels map[int]func() - cancelled bool -} - -// NewRPCanceller creates RPCCanceller with init state. -func NewRPCanceller() *RPCCanceller { - return &RPCCanceller{cancels: make(map[int]func())} -} - -// WithCancel generates new context with cancel func. -func (h *RPCCanceller) WithCancel(ctx context.Context) (context.Context, func()) { - nctx, cancel := context.WithCancel(ctx) - h.Lock() - if h.cancelled { - h.Unlock() - cancel() - return nctx, func() {} - } - id := h.allocID - h.allocID++ - h.cancels[id] = cancel - h.Unlock() - return nctx, func() { - cancel() - h.Lock() - delete(h.cancels, id) - h.Unlock() - } -} - -// CancelAll cancels all inflight rpc context. -func (h *RPCCanceller) CancelAll() { - h.Lock() - for _, c := range h.cancels { - c() - } - h.cancelled = true - h.Unlock() -} - -func (s *RegionRequestSender) sendReqToRegion(bo *retry.Backoffer, rpcCtx *RPCContext, req *tikvrpc.Request, timeout time.Duration) (resp *tikvrpc.Response, retry bool, err error) { - if e := tikvrpc.SetContext(req, rpcCtx.Meta, rpcCtx.Peer); e != nil { - return nil, false, errors.Trace(e) - } - // judge the store limit switch. - if limit := kv.StoreLimit.Load(); limit > 0 { - if err := s.getStoreToken(rpcCtx.Store, limit); err != nil { - return nil, false, err - } - defer s.releaseStoreToken(rpcCtx.Store) - } - - ctx := bo.GetCtx() - if rawHook := ctx.Value(RPCCancellerCtxKey{}); rawHook != nil { - var cancel context.CancelFunc - ctx, cancel = rawHook.(*RPCCanceller).WithCancel(ctx) - defer cancel() - } - - // sendToAddr is the first target address that will receive the request. If proxy is used, sendToAddr will point to - // the proxy that will forward the request to the final target. - sendToAddr := rpcCtx.Addr - if rpcCtx.ProxyStore == nil { - req.ForwardedHost = "" - } else { - req.ForwardedHost = rpcCtx.Addr - sendToAddr = rpcCtx.ProxyAddr - } - - var sessionID uint64 - if v := bo.GetCtx().Value(util.SessionID); v != nil { - sessionID = v.(uint64) - } - - injectFailOnSend := false - if val, e := util.EvalFailpoint("rpcFailOnSend"); e == nil { - inject := true - // Optional filters - if s, ok := val.(string); ok { - if s == "greengc" && !req.IsGreenGCRequest() { - inject = false - } else if s == "write" && !req.IsTxnWriteRequest() { - inject = false - } - } else if sessionID == 0 { - inject = false - } - - if inject { - logutil.Logger(ctx).Info("[failpoint] injected RPC error on send", zap.Stringer("type", req.Type), - zap.Stringer("req", req.Req.(fmt.Stringer)), zap.Stringer("ctx", &req.Context)) - injectFailOnSend = true - err = errors.New("injected RPC error on send") - } - } - - if !injectFailOnSend { - start := time.Now() - resp, err = s.client.SendRequest(ctx, sendToAddr, req, timeout) - if s.Stats != nil { - RecordRegionRequestRuntimeStats(s.Stats, req.Type, time.Since(start)) - if val, err := util.EvalFailpoint("tikvStoreRespResult"); err == nil { - if val.(bool) { - if req.Type == tikvrpc.CmdCop && bo.GetTotalSleep() == 0 { - return &tikvrpc.Response{ - Resp: &coprocessor.Response{RegionError: &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}}, - }, false, nil - } - } - } - } - - if val, e := util.EvalFailpoint("rpcFailOnRecv"); e == nil { - inject := true - // Optional filters - if s, ok := val.(string); ok { - if s == "greengc" && !req.IsGreenGCRequest() { - inject = false - } else if s == "write" && !req.IsTxnWriteRequest() { - inject = false - } - } else if sessionID == 0 { - inject = false - } - - if inject { - logutil.Logger(ctx).Info("[failpoint] injected RPC error on recv", zap.Stringer("type", req.Type), - zap.Stringer("req", req.Req.(fmt.Stringer)), zap.Stringer("ctx", &req.Context)) - err = errors.New("injected RPC error on recv") - resp = nil - } - } - - if val, e := util.EvalFailpoint("rpcContextCancelErr"); e == nil { - if val.(bool) { - ctx1, cancel := context.WithCancel(context.Background()) - cancel() - <-ctx1.Done() - ctx = ctx1 - err = ctx.Err() - resp = nil - } - } - } - - if rpcCtx.ProxyStore != nil { - fromStore := strconv.FormatUint(rpcCtx.ProxyStore.storeID, 10) - toStore := strconv.FormatUint(rpcCtx.Store.storeID, 10) - result := "ok" - if err != nil { - result = "fail" - } - metrics.TiKVForwardRequestCounter.WithLabelValues(fromStore, toStore, req.Type.String(), result).Inc() - } - - if err != nil { - s.rpcError = err - - // Because in rpc logic, context.Cancel() will be transferred to rpcContext.Cancel error. For rpcContext cancel, - // we need to retry the request. But for context cancel active, for example, limitExec gets the required rows, - // we shouldn't retry the request, it will go to backoff and hang in retry logic. - if ctx.Err() != nil && errors.Cause(ctx.Err()) == context.Canceled { - return nil, false, errors.Trace(ctx.Err()) - } - - if val, e := util.EvalFailpoint("noRetryOnRpcError"); e == nil { - if val.(bool) { - return nil, false, err - } - } - if e := s.onSendFail(bo, rpcCtx, err); e != nil { - return nil, false, errors.Trace(e) - } - return nil, true, nil - } - return -} - -func (s *RegionRequestSender) getStoreToken(st *Store, limit int64) error { - // Checking limit is not thread safe, preferring this for avoiding load in loop. - count := st.tokenCount.Load() - if count < limit { - // Adding tokenCount is no thread safe, preferring this for avoiding check in loop. - st.tokenCount.Add(1) - return nil - } - metrics.TiKVStoreLimitErrorCounter.WithLabelValues(st.addr, strconv.FormatUint(st.storeID, 10)).Inc() - return &tikverr.ErrTokenLimit{StoreID: st.storeID} -} - -func (s *RegionRequestSender) releaseStoreToken(st *Store) { - count := st.tokenCount.Load() - // Decreasing tokenCount is no thread safe, preferring this for avoiding check in loop. - if count > 0 { - st.tokenCount.Sub(1) - return - } - logutil.BgLogger().Warn("release store token failed, count equals to 0") -} - -func (s *RegionRequestSender) onSendFail(bo *retry.Backoffer, ctx *RPCContext, err error) error { - if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("regionRequest.onSendFail", opentracing.ChildOf(span.Context())) - defer span1.Finish() - bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) - } - // If it failed because the context is cancelled by ourself, don't retry. - if errors.Cause(err) == context.Canceled { - return errors.Trace(err) - } else if LoadShuttingDown() > 0 { - return tikverr.ErrTiDBShuttingDown - } - if status.Code(errors.Cause(err)) == codes.Canceled { - select { - case <-bo.GetCtx().Done(): - return errors.Trace(err) - default: - // If we don't cancel, but the error code is Canceled, it must be from grpc remote. - // This may happen when tikv is killed and exiting. - // Backoff and retry in this case. - logutil.BgLogger().Warn("receive a grpc cancel signal from remote", zap.Error(err)) - } - } - - if ctx.Meta != nil { - if s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.onSendFailure(bo, err) - } else { - s.regionCache.OnSendFail(bo, ctx, s.NeedReloadRegion(ctx), err) - } - } - - // Retry on send request failure when it's not canceled. - // When a store is not available, the leader of related region should be elected quickly. - // TODO: the number of retry time should be limited:since region may be unavailable - // when some unrecoverable disaster happened. - if ctx.Store != nil && ctx.Store.storeType == tikvrpc.TiFlash { - err = bo.Backoff(retry.BoTiFlashRPC, errors.Errorf("send tiflash request error: %v, ctx: %v, try next peer later", err, ctx)) - } else { - err = bo.Backoff(retry.BoTiKVRPC, errors.Errorf("send tikv request error: %v, ctx: %v, try next peer later", err, ctx)) - } - return errors.Trace(err) -} - -// NeedReloadRegion checks is all peers has sent failed, if so need reload. -func (s *RegionRequestSender) NeedReloadRegion(ctx *RPCContext) (need bool) { - if s.failStoreIDs == nil { - s.failStoreIDs = make(map[uint64]struct{}) - } - if s.failProxyStoreIDs == nil { - s.failProxyStoreIDs = make(map[uint64]struct{}) - } - s.failStoreIDs[ctx.Store.storeID] = struct{}{} - if ctx.ProxyStore != nil { - s.failProxyStoreIDs[ctx.ProxyStore.storeID] = struct{}{} - } - - if ctx.AccessMode == tiKVOnly && len(s.failStoreIDs)+len(s.failProxyStoreIDs) >= ctx.TiKVNum { - need = true - } else if ctx.AccessMode == tiFlashOnly && len(s.failStoreIDs) >= len(ctx.Meta.Peers)-ctx.TiKVNum { - need = true - } else if len(s.failStoreIDs)+len(s.failProxyStoreIDs) >= len(ctx.Meta.Peers) { - need = true - } - - if need { - s.failStoreIDs = nil - s.failProxyStoreIDs = nil - } - return -} - -func regionErrorToLabel(e *errorpb.Error) string { - if e.GetNotLeader() != nil { - return "not_leader" - } else if e.GetRegionNotFound() != nil { - return "region_not_found" - } else if e.GetKeyNotInRegion() != nil { - return "key_not_in_region" - } else if e.GetEpochNotMatch() != nil { - return "epoch_not_match" - } else if e.GetServerIsBusy() != nil { - return "server_is_busy" - } else if e.GetStaleCommand() != nil { - return "stale_command" - } else if e.GetStoreNotMatch() != nil { - return "store_not_match" - } - return "unknown" -} - -func (s *RegionRequestSender) onRegionError(bo *retry.Backoffer, ctx *RPCContext, req *tikvrpc.Request, regionErr *errorpb.Error, opts *[]StoreSelectorOption) (shouldRetry bool, err error) { - if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("tikv.onRegionError", opentracing.ChildOf(span.Context())) - defer span1.Finish() - bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) - } - // Stale Read request will retry the leader or next peer on error, - // if txnScope is global, we will only retry the leader by using the WithLeaderOnly option, - // if txnScope is local, we will retry both other peers and the leader by the incresing seed. - if ctx.tryTimes < 1 && req != nil && req.TxnScope == oracle.GlobalTxnScope && req.GetStaleRead() { - *opts = append(*opts, WithLeaderOnly()) - } - seed := req.GetReplicaReadSeed() - - // NOTE: Please add the region error handler in the same order of errorpb.Error. - metrics.TiKVRegionErrorCounter.WithLabelValues(regionErrorToLabel(regionErr)).Inc() - - if notLeader := regionErr.GetNotLeader(); notLeader != nil { - // Retry if error is `NotLeader`. - logutil.BgLogger().Debug("tikv reports `NotLeader` retry later", - zap.String("notLeader", notLeader.String()), - zap.String("ctx", ctx.String())) - - if s.leaderReplicaSelector != nil { - leader := notLeader.GetLeader() - if leader == nil { - // The region may be during transferring leader. - if err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("no leader, ctx: %v", ctx)); err != nil { - return false, errors.Trace(err) - } - } else { - s.leaderReplicaSelector.updateLeader(notLeader.GetLeader()) - } - return true, nil - } else if notLeader.GetLeader() == nil { - // The peer doesn't know who is the current leader. Generally it's because - // the Raft group is in an election, but it's possible that the peer is - // isolated and removed from the Raft group. So it's necessary to reload - // the region from PD. - s.regionCache.InvalidateCachedRegionWithReason(ctx.Region, NoLeader) - if err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil { - return false, errors.Trace(err) - } - return false, nil - } else { - // don't backoff if a new leader is returned. - s.regionCache.UpdateLeader(ctx.Region, notLeader.GetLeader(), ctx.AccessIdx) - return true, nil - } - } - - // This peer is removed from the region. Invalidate the region since it's too stale. - if regionErr.GetRegionNotFound() != nil { - if seed != nil { - logutil.BgLogger().Debug("tikv reports `RegionNotFound` in follow-reader", - zap.Stringer("ctx", ctx), zap.Uint32("seed", *seed)) - *seed = *seed + 1 - } - s.regionCache.InvalidateCachedRegion(ctx.Region) - return false, nil - } - - if regionErr.GetKeyNotInRegion() != nil { - logutil.BgLogger().Debug("tikv reports `KeyNotInRegion`", zap.Stringer("ctx", ctx)) - s.regionCache.InvalidateCachedRegion(ctx.Region) - return false, nil - } - - if epochNotMatch := regionErr.GetEpochNotMatch(); epochNotMatch != nil { - logutil.BgLogger().Debug("tikv reports `EpochNotMatch` retry later", - zap.Stringer("EpochNotMatch", epochNotMatch), - zap.Stringer("ctx", ctx)) - if seed != nil { - *seed = *seed + 1 - } - retry, err := s.regionCache.OnRegionEpochNotMatch(bo, ctx, epochNotMatch.CurrentRegions) - if !retry && s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.invalidateRegion() - } - return retry, errors.Trace(err) - } - - if regionErr.GetServerIsBusy() != nil { - logutil.BgLogger().Warn("tikv reports `ServerIsBusy` retry later", - zap.String("reason", regionErr.GetServerIsBusy().GetReason()), - zap.Stringer("ctx", ctx)) - if ctx != nil && ctx.Store != nil && ctx.Store.storeType == tikvrpc.TiFlash { - err = bo.Backoff(retry.BoTiFlashServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) - } else { - err = bo.Backoff(retry.BoTiKVServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) - } - if err != nil { - return false, errors.Trace(err) - } - if s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.rewind() - } - return true, nil - } - - // StaleCommand error indicates the request is sent to the old leader and its term is changed. - // We can't know whether the request is committed or not, so it's an undetermined error too, - // but we don't handle it now. - if regionErr.GetStaleCommand() != nil { - logutil.BgLogger().Debug("tikv reports `StaleCommand`", zap.Stringer("ctx", ctx)) - if s.leaderReplicaSelector != nil { - // Needn't backoff because the new leader should be elected soon - // and the leaderReplicaSelector will try the next peer. - } else { - err = bo.Backoff(retry.BoStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) - if err != nil { - return false, errors.Trace(err) - } - } - return true, nil - } - - if storeNotMatch := regionErr.GetStoreNotMatch(); storeNotMatch != nil { - // store not match - logutil.BgLogger().Debug("tikv reports `StoreNotMatch` retry later", - zap.Stringer("storeNotMatch", storeNotMatch), - zap.Stringer("ctx", ctx)) - ctx.Store.markNeedCheck(s.regionCache.notifyCheckCh) - s.regionCache.InvalidateCachedRegion(ctx.Region) - return false, nil - } - - if regionErr.GetRaftEntryTooLarge() != nil { - logutil.BgLogger().Warn("tikv reports `RaftEntryTooLarge`", zap.Stringer("ctx", ctx)) - return false, errors.New(regionErr.String()) - } - - if regionErr.GetMaxTimestampNotSynced() != nil { - logutil.BgLogger().Debug("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) - err = bo.Backoff(retry.BoMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) - if err != nil { - return false, errors.Trace(err) - } - if s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.rewind() - } - return true, nil - } - - // A read request may be sent to a peer which has not been initialized yet, we should retry in this case. - if regionErr.GetRegionNotInitialized() != nil { - logutil.BgLogger().Debug("tikv reports `RegionNotInitialized` retry later", - zap.Uint64("store-id", ctx.Store.storeID), - zap.Uint64("region-id", regionErr.GetRegionNotInitialized().GetRegionId()), - zap.Stringer("ctx", ctx)) - err = bo.Backoff(retry.BoMaxRegionNotInitialized, errors.Errorf("region not initialized")) - if err != nil { - return false, errors.Trace(err) - } - if seed != nil { - *seed = *seed + 1 - } - return true, nil - } - - // The read-index can't be handled timely because the region is splitting or merging. - if regionErr.GetReadIndexNotReady() != nil { - logutil.BgLogger().Debug("tikv reports `ReadIndexNotReady` retry later", - zap.Uint64("store-id", ctx.Store.storeID), - zap.Uint64("region-id", regionErr.GetRegionNotInitialized().GetRegionId()), - zap.Stringer("ctx", ctx)) - if seed != nil { - *seed = *seed + 1 - } - // The region can't provide service until split or merge finished, so backoff. - err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("read index not ready, ctx: %v", ctx)) - if err != nil { - return false, errors.Trace(err) - } - if s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.rewind() - } - return true, nil - } - - if regionErr.GetProposalInMergingMode() != nil { - logutil.BgLogger().Debug("tikv reports `ProposalInMergingMode`", zap.Stringer("ctx", ctx)) - // The region is merging and it can't provide service until merge finished, so backoff. - err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("region is merging, ctx: %v", ctx)) - if err != nil { - return false, errors.Trace(err) - } - if s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.rewind() - } - return true, nil - } - - // A stale read request may be sent to a peer which the data is not ready yet, we should retry in this case. - // This error is specific to stale read and the target replica is randomly selected. If the request is sent - // to the leader, the data must be ready, so we don't backoff here. - if regionErr.GetDataIsNotReady() != nil { - logutil.BgLogger().Warn("tikv reports `DataIsNotReady` retry later", - zap.Uint64("store-id", ctx.Store.storeID), - zap.Uint64("peer-id", regionErr.GetDataIsNotReady().GetPeerId()), - zap.Uint64("region-id", regionErr.GetDataIsNotReady().GetRegionId()), - zap.Uint64("safe-ts", regionErr.GetDataIsNotReady().GetSafeTs()), - zap.Stringer("ctx", ctx)) - err = bo.Backoff(retry.BoMaxDataNotReady, errors.Errorf("data is not ready")) - if err != nil { - return false, errors.Trace(err) - } - if seed != nil { - *seed = *seed + 1 - } - return true, nil - } - - logutil.BgLogger().Debug("tikv reports region failed", - zap.Stringer("regionErr", regionErr), - zap.Stringer("ctx", ctx)) - - if s.leaderReplicaSelector != nil { - // Try the next replica. - return true, nil - } - - // When the request is sent to TiDB, there is no region in the request, so the region id will be 0. - // So when region id is 0, there is no business with region cache. - if ctx.Region.id != 0 { - s.regionCache.InvalidateCachedRegion(ctx.Region) - } - // For other errors, we only drop cache here. - // Because caller may need to re-split the request. - return false, nil -} diff --git a/store/tikv/locate/region_request_test.go b/store/tikv/locate/region_request_test.go deleted file mode 100644 index ac67115928637..0000000000000 --- a/store/tikv/locate/region_request_test.go +++ /dev/null @@ -1,1187 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package locate - -import ( - "context" - "fmt" - "net" - "sync" - "sync/atomic" - "time" - "unsafe" - - . "github.com/pingcap/check" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/errorpb" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/mpp" - "github.com/pingcap/kvproto/pkg/tikvpb" - "github.com/pingcap/tidb/store/tikv/client" - tikvclient "github.com/pingcap/tidb/store/tikv/client" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/retry" - - "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "google.golang.org/grpc" -) - -type testRegionRequestToSingleStoreSuite struct { - cluster *mocktikv.Cluster - store uint64 - peer uint64 - region uint64 - cache *RegionCache - bo *retry.Backoffer - regionRequestSender *RegionRequestSender - mvccStore mocktikv.MVCCStore -} - -type testRegionRequestToThreeStoresSuite struct { - OneByOneSuite - cluster *mocktikv.Cluster - storeIDs []uint64 - peerIDs []uint64 - regionID uint64 - leaderPeer uint64 - cache *RegionCache - bo *retry.Backoffer - regionRequestSender *RegionRequestSender - mvccStore mocktikv.MVCCStore -} - -var _ = Suite(&testRegionRequestToSingleStoreSuite{}) -var _ = Suite(&testRegionRequestToThreeStoresSuite{}) - -func (s *testRegionRequestToSingleStoreSuite) SetUpTest(c *C) { - s.cluster = mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) - s.store, s.peer, s.region = mocktikv.BootstrapWithSingleStore(s.cluster) - pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster)} - s.cache = NewRegionCache(pdCli) - s.bo = retry.NewNoopBackoff(context.Background()) - s.mvccStore = mocktikv.MustNewMVCCStore() - client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) - s.regionRequestSender = NewRegionRequestSender(s.cache, client) -} - -func (s *testRegionRequestToThreeStoresSuite) SetUpTest(c *C) { - s.cluster = mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) - s.storeIDs, s.peerIDs, s.regionID, s.leaderPeer = mocktikv.BootstrapWithMultiStores(s.cluster, 3) - pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster)} - s.cache = NewRegionCache(pdCli) - s.bo = retry.NewNoopBackoff(context.Background()) - s.mvccStore = mocktikv.MustNewMVCCStore() - client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) - s.regionRequestSender = NewRegionRequestSender(s.cache, client) -} - -func (s *testRegionRequestToSingleStoreSuite) TearDownTest(c *C) { - s.cache.Close() -} - -func (s *testRegionRequestToThreeStoresSuite) TearDownTest(c *C) { - s.cache.Close() -} - -type fnClient struct { - fn func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) -} - -func (f *fnClient) Close() error { - return nil -} - -func (f *fnClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - return f.fn(ctx, addr, req, timeout) -} - -func (s *testRegionRequestToThreeStoresSuite) TestGetRPCContext(c *C) { - // Load the bootstrapped region into the cache. - _, err := s.cache.BatchLoadRegionsFromKey(s.bo, []byte{}, 1) - c.Assert(err, IsNil) - - var seed uint32 - var regionID = RegionVerID{s.regionID, 0, 0} - - req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{}, kv.ReplicaReadLeader, &seed) - rpcCtx, err := s.regionRequestSender.getRPCContext(s.bo, req, regionID, tikvrpc.TiKV) - c.Assert(err, IsNil) - c.Assert(rpcCtx.Peer.Id, Equals, s.leaderPeer) - - req.ReplicaReadType = kv.ReplicaReadFollower - rpcCtx, err = s.regionRequestSender.getRPCContext(s.bo, req, regionID, tikvrpc.TiKV) - c.Assert(err, IsNil) - c.Assert(rpcCtx.Peer.Id, Not(Equals), s.leaderPeer) - - req.ReplicaReadType = kv.ReplicaReadMixed - rpcCtx, err = s.regionRequestSender.getRPCContext(s.bo, req, regionID, tikvrpc.TiKV) - c.Assert(err, IsNil) - c.Assert(rpcCtx.Peer.Id, Equals, s.leaderPeer) - - seed = 1 - rpcCtx, err = s.regionRequestSender.getRPCContext(s.bo, req, regionID, tikvrpc.TiKV) - c.Assert(err, IsNil) - c.Assert(rpcCtx.Peer.Id, Not(Equals), s.leaderPeer) -} - -func (s *testRegionRequestToSingleStoreSuite) TestOnRegionError(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - - // test stale command retry. - func() { - oc := s.regionRequestSender.client - defer func() { - s.regionRequestSender.client = oc - }() - s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { - staleResp := &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{ - RegionError: &errorpb.Error{StaleCommand: &errorpb.StaleCommand{}}, - }} - return staleResp, nil - }} - bo := retry.NewBackofferWithVars(context.Background(), 5, nil) - resp, err := s.regionRequestSender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp, NotNil) - regionErr, _ := resp.GetRegionError() - c.Assert(regionErr, NotNil) - }() -} - -func (s *testRegionRequestToThreeStoresSuite) TestStoreTokenLimit(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, &kvrpcpb.PrewriteRequest{}, kvrpcpb.Context{}) - region, err := s.cache.LocateRegionByID(s.bo, s.regionID) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - oldStoreLimit := kv.StoreLimit.Load() - kv.StoreLimit.Store(500) - s.cache.getStoreByStoreID(s.storeIDs[0]).tokenCount.Store(500) - // cause there is only one region in this cluster, regionID maps this leader. - resp, err := s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) - c.Assert(err, NotNil) - c.Assert(resp, IsNil) - e, ok := errors.Cause(err).(*tikverr.ErrTokenLimit) - c.Assert(ok, IsTrue) - c.Assert(e.StoreID, Equals, uint64(1)) - kv.StoreLimit.Store(oldStoreLimit) -} - -func (s *testRegionRequestToSingleStoreSuite) TestOnSendFailedWithStoreRestart(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - resp, err := s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp.Resp, NotNil) - c.Assert(s.regionRequestSender.rpcError, IsNil) - - // stop store. - s.cluster.StopStore(s.store) - _, err = s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) - c.Assert(err, NotNil) - // The RPC error shouldn't be nil since it failed to sent the request. - c.Assert(s.regionRequestSender.rpcError, NotNil) - - // start store. - s.cluster.StartStore(s.store) - - // locate region again is needed - // since last request on the region failed and region's info had been cleared. - region, err = s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - c.Assert(s.regionRequestSender.rpcError, NotNil) - resp, err = s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp.Resp, NotNil) - // The RPC error should be nil since it's evaluated successfully. - c.Assert(s.regionRequestSender.rpcError, IsNil) -} - -func (s *testRegionRequestToSingleStoreSuite) TestOnSendFailedWithCloseKnownStoreThenUseNewOne(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - - // add new store2 and make store2 as leader. - store2 := s.cluster.AllocID() - peer2 := s.cluster.AllocID() - s.cluster.AddStore(store2, fmt.Sprintf("store%d", store2)) - s.cluster.AddPeer(s.region, store2, peer2) - s.cluster.ChangeLeader(s.region, peer2) - - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - resp, err := s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp.Resp, NotNil) - - // stop store2 and make store1 as new leader. - s.cluster.StopStore(store2) - s.cluster.ChangeLeader(s.region, s.peer) - - // send to store2 fail and send to new leader store1. - bo2 := retry.NewBackofferWithVars(context.Background(), 100, nil) - resp, err = s.regionRequestSender.SendReq(bo2, req, region.Region, time.Second) - c.Assert(err, IsNil) - regionErr, err := resp.GetRegionError() - c.Assert(err, IsNil) - c.Assert(regionErr, IsNil) - c.Assert(resp.Resp, NotNil) -} - -func (s *testRegionRequestToSingleStoreSuite) TestSendReqCtx(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - resp, ctx, err := s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) - c.Assert(err, IsNil) - c.Assert(resp.Resp, NotNil) - c.Assert(ctx, NotNil) - req.ReplicaRead = true - resp, ctx, err = s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) - c.Assert(err, IsNil) - c.Assert(resp.Resp, NotNil) - c.Assert(ctx, NotNil) -} - -func (s *testRegionRequestToSingleStoreSuite) TestOnSendFailedWithCancelled(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - resp, err := s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp.Resp, NotNil) - - // set store to cancel state. - s.cluster.CancelStore(s.store) - // locate region again is needed - // since last request on the region failed and region's info had been cleared. - _, err = s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) - c.Assert(err, NotNil) - c.Assert(errors.Cause(err), Equals, context.Canceled) - - // set store to normal state. - s.cluster.UnCancelStore(s.store) - region, err = s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - resp, err = s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp.Resp, NotNil) -} - -func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionWhenCtxCanceled(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - - sender := s.regionRequestSender - bo, cancel := s.bo.Fork() - cancel() - // Call SendKVReq with a canceled context. - _, err = sender.SendReq(bo, req, region.Region, time.Second) - // Check this kind of error won't cause region cache drop. - c.Assert(errors.Cause(err), Equals, context.Canceled) - c.Assert(sender.regionCache.getRegionByIDFromCache(s.region), NotNil) -} - -// cancelContextClient wraps rpcClient and always cancels context before sending requests. -type cancelContextClient struct { - client.Client - redirectAddr string -} - -func (c *cancelContextClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - childCtx, cancel := context.WithCancel(ctx) - cancel() - return c.Client.SendRequest(childCtx, c.redirectAddr, req, timeout) -} - -// mockTikvGrpcServer mock a tikv gprc server for testing. -type mockTikvGrpcServer struct{} - -// KvGet commands with mvcc/txn supported. -func (s *mockTikvGrpcServer) KvGet(context.Context, *kvrpcpb.GetRequest) (*kvrpcpb.GetResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvScan(context.Context, *kvrpcpb.ScanRequest) (*kvrpcpb.ScanResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvPrewrite(context.Context, *kvrpcpb.PrewriteRequest) (*kvrpcpb.PrewriteResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvCommit(context.Context, *kvrpcpb.CommitRequest) (*kvrpcpb.CommitResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvImport(context.Context, *kvrpcpb.ImportRequest) (*kvrpcpb.ImportResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvCleanup(context.Context, *kvrpcpb.CleanupRequest) (*kvrpcpb.CleanupResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvBatchGet(context.Context, *kvrpcpb.BatchGetRequest) (*kvrpcpb.BatchGetResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvBatchRollback(context.Context, *kvrpcpb.BatchRollbackRequest) (*kvrpcpb.BatchRollbackResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvScanLock(context.Context, *kvrpcpb.ScanLockRequest) (*kvrpcpb.ScanLockResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvResolveLock(context.Context, *kvrpcpb.ResolveLockRequest) (*kvrpcpb.ResolveLockResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvPessimisticLock(context.Context, *kvrpcpb.PessimisticLockRequest) (*kvrpcpb.PessimisticLockResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KVPessimisticRollback(context.Context, *kvrpcpb.PessimisticRollbackRequest) (*kvrpcpb.PessimisticRollbackResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvCheckTxnStatus(ctx context.Context, in *kvrpcpb.CheckTxnStatusRequest) (*kvrpcpb.CheckTxnStatusResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvCheckSecondaryLocks(ctx context.Context, in *kvrpcpb.CheckSecondaryLocksRequest) (*kvrpcpb.CheckSecondaryLocksResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvTxnHeartBeat(ctx context.Context, in *kvrpcpb.TxnHeartBeatRequest) (*kvrpcpb.TxnHeartBeatResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvGC(context.Context, *kvrpcpb.GCRequest) (*kvrpcpb.GCResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) KvDeleteRange(context.Context, *kvrpcpb.DeleteRangeRequest) (*kvrpcpb.DeleteRangeResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawGet(context.Context, *kvrpcpb.RawGetRequest) (*kvrpcpb.RawGetResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawBatchGet(context.Context, *kvrpcpb.RawBatchGetRequest) (*kvrpcpb.RawBatchGetResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawPut(context.Context, *kvrpcpb.RawPutRequest) (*kvrpcpb.RawPutResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawBatchPut(context.Context, *kvrpcpb.RawBatchPutRequest) (*kvrpcpb.RawBatchPutResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawDelete(context.Context, *kvrpcpb.RawDeleteRequest) (*kvrpcpb.RawDeleteResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawBatchDelete(context.Context, *kvrpcpb.RawBatchDeleteRequest) (*kvrpcpb.RawBatchDeleteResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawScan(context.Context, *kvrpcpb.RawScanRequest) (*kvrpcpb.RawScanResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawDeleteRange(context.Context, *kvrpcpb.RawDeleteRangeRequest) (*kvrpcpb.RawDeleteRangeResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawBatchScan(context.Context, *kvrpcpb.RawBatchScanRequest) (*kvrpcpb.RawBatchScanResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawGetKeyTTL(context.Context, *kvrpcpb.RawGetKeyTTLRequest) (*kvrpcpb.RawGetKeyTTLResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) UnsafeDestroyRange(context.Context, *kvrpcpb.UnsafeDestroyRangeRequest) (*kvrpcpb.UnsafeDestroyRangeResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RegisterLockObserver(context.Context, *kvrpcpb.RegisterLockObserverRequest) (*kvrpcpb.RegisterLockObserverResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) CheckLockObserver(context.Context, *kvrpcpb.CheckLockObserverRequest) (*kvrpcpb.CheckLockObserverResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RemoveLockObserver(context.Context, *kvrpcpb.RemoveLockObserverRequest) (*kvrpcpb.RemoveLockObserverResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) PhysicalScanLock(context.Context, *kvrpcpb.PhysicalScanLockRequest) (*kvrpcpb.PhysicalScanLockResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) Coprocessor(context.Context, *coprocessor.Request) (*coprocessor.Response, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) BatchCoprocessor(*coprocessor.BatchRequest, tikvpb.Tikv_BatchCoprocessorServer) error { - return errors.New("unreachable") -} -func (s *mockTikvGrpcServer) RawCoprocessor(context.Context, *kvrpcpb.RawCoprocessorRequest) (*kvrpcpb.RawCoprocessorResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) DispatchMPPTask(context.Context, *mpp.DispatchTaskRequest) (*mpp.DispatchTaskResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) EstablishMPPConnection(*mpp.EstablishMPPConnectionRequest, tikvpb.Tikv_EstablishMPPConnectionServer) error { - return errors.New("unreachable") -} -func (s *mockTikvGrpcServer) CancelMPPTask(context.Context, *mpp.CancelTaskRequest) (*mpp.CancelTaskResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) Raft(tikvpb.Tikv_RaftServer) error { - return errors.New("unreachable") -} -func (s *mockTikvGrpcServer) BatchRaft(tikvpb.Tikv_BatchRaftServer) error { - return errors.New("unreachable") -} -func (s *mockTikvGrpcServer) Snapshot(tikvpb.Tikv_SnapshotServer) error { - return errors.New("unreachable") -} -func (s *mockTikvGrpcServer) MvccGetByKey(context.Context, *kvrpcpb.MvccGetByKeyRequest) (*kvrpcpb.MvccGetByKeyResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) MvccGetByStartTs(context.Context, *kvrpcpb.MvccGetByStartTsRequest) (*kvrpcpb.MvccGetByStartTsResponse, error) { - return nil, errors.New("unreachable") -} -func (s *mockTikvGrpcServer) SplitRegion(context.Context, *kvrpcpb.SplitRegionRequest) (*kvrpcpb.SplitRegionResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) CoprocessorStream(*coprocessor.Request, tikvpb.Tikv_CoprocessorStreamServer) error { - return errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) BatchCommands(tikvpb.Tikv_BatchCommandsServer) error { - return errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) ReadIndex(context.Context, *kvrpcpb.ReadIndexRequest) (*kvrpcpb.ReadIndexResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) CheckLeader(context.Context, *kvrpcpb.CheckLeaderRequest) (*kvrpcpb.CheckLeaderResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) GetStoreSafeTS(context.Context, *kvrpcpb.StoreSafeTSRequest) (*kvrpcpb.StoreSafeTSResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) RawCompareAndSwap(context.Context, *kvrpcpb.RawCASRequest) (*kvrpcpb.RawCASResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionForGrpcWhenCtxCanceled(c *C) { - // prepare a mock tikv grpc server - addr := "localhost:56341" - lis, err := net.Listen("tcp", addr) - c.Assert(err, IsNil) - server := grpc.NewServer() - tikvpb.RegisterTikvServer(server, &mockTikvGrpcServer{}) - wg := &sync.WaitGroup{} - wg.Add(1) - go func() { - server.Serve(lis) - wg.Done() - }() - - client := tikvclient.NewRPCClient(config.Security{}) - sender := NewRegionRequestSender(s.cache, client) - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - - bo, cancel := s.bo.Fork() - cancel() - _, err = sender.SendReq(bo, req, region.Region, 3*time.Second) - c.Assert(errors.Cause(err), Equals, context.Canceled) - c.Assert(s.cache.getRegionByIDFromCache(s.region), NotNil) - - // Just for covering error code = codes.Canceled. - client1 := &cancelContextClient{ - Client: tikvclient.NewRPCClient(config.Security{}), - redirectAddr: addr, - } - sender = NewRegionRequestSender(s.cache, client1) - sender.SendReq(s.bo, req, region.Region, 3*time.Second) - - // cleanup - server.Stop() - wg.Wait() -} - -func (s *testRegionRequestToSingleStoreSuite) TestOnMaxTimestampNotSyncedError(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, &kvrpcpb.PrewriteRequest{}) - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - - // test retry for max timestamp not synced - func() { - oc := s.regionRequestSender.client - defer func() { - s.regionRequestSender.client = oc - }() - count := 0 - s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { - count++ - var resp *tikvrpc.Response - if count < 3 { - resp = &tikvrpc.Response{Resp: &kvrpcpb.PrewriteResponse{ - RegionError: &errorpb.Error{MaxTimestampNotSynced: &errorpb.MaxTimestampNotSynced{}}, - }} - } else { - resp = &tikvrpc.Response{Resp: &kvrpcpb.PrewriteResponse{}} - } - return resp, nil - }} - bo := retry.NewBackofferWithVars(context.Background(), 5, nil) - resp, err := s.regionRequestSender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp, NotNil) - }() -} - -func (s *testRegionRequestToThreeStoresSuite) TestSwitchPeerWhenNoLeader(c *C) { - var leaderAddr string - s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { - if leaderAddr == "" { - leaderAddr = addr - } - // Returns OK when switches to a different peer. - if leaderAddr != addr { - return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{}}, nil - } - return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{ - RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}, - }}, nil - }} - - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - - bo := retry.NewBackofferWithVars(context.Background(), 5, nil) - loc, err := s.cache.LocateKey(s.bo, []byte("key")) - c.Assert(err, IsNil) - resp, err := s.regionRequestSender.SendReq(bo, req, loc.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp, NotNil) -} - -func (s *testRegionRequestToThreeStoresSuite) loadAndGetLeaderStore(c *C) (*Store, string) { - region, err := s.regionRequestSender.regionCache.findRegionByKey(s.bo, []byte("a"), false) - c.Assert(err, IsNil) - leaderStore, leaderPeer, _, _ := region.WorkStorePeer(region.getStore()) - c.Assert(leaderPeer.Id, Equals, s.leaderPeer) - leaderAddr, err := s.regionRequestSender.regionCache.getStoreAddr(s.bo, region, leaderStore) - c.Assert(err, IsNil) - return leaderStore, leaderAddr -} - -func (s *testRegionRequestToThreeStoresSuite) TestForwarding(c *C) { - s.regionRequestSender.regionCache.enableForwarding = true - - // First get the leader's addr from region cache - leaderStore, leaderAddr := s.loadAndGetLeaderStore(c) - - bo := retry.NewBackoffer(context.Background(), 10000) - - // Simulate that the leader is network-partitioned but can be accessed by forwarding via a follower - innerClient := s.regionRequestSender.client - s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - if addr == leaderAddr { - return nil, errors.New("simulated rpc error") - } - // MockTiKV doesn't support forwarding. Simulate forwarding here. - if len(req.ForwardedHost) != 0 { - addr = req.ForwardedHost - } - return innerClient.SendRequest(ctx, addr, req, timeout) - }} - var storeState uint32 = uint32(unreachable) - s.regionRequestSender.regionCache.testingKnobs.mockRequestLiveness = func(s *Store, bo *retry.Backoffer) livenessState { - return livenessState(atomic.LoadUint32(&storeState)) - } - - loc, err := s.regionRequestSender.regionCache.LocateKey(bo, []byte("k")) - c.Assert(err, IsNil) - c.Assert(loc.Region.GetID(), Equals, s.regionID) - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("k"), - Value: []byte("v1"), - }) - resp, ctx, err := s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV) - c.Assert(err, IsNil) - regionErr, err := resp.GetRegionError() - c.Assert(err, IsNil) - c.Assert(regionErr, IsNil) - c.Assert(resp.Resp.(*kvrpcpb.RawPutResponse).Error, Equals, "") - c.Assert(ctx.Addr, Equals, leaderAddr) - c.Assert(ctx.ProxyStore, NotNil) - c.Assert(ctx.ProxyAddr, Not(Equals), leaderAddr) - c.Assert(ctx.ProxyAccessIdx, Not(Equals), ctx.AccessIdx) - c.Assert(err, IsNil) - - // Simulate recovering to normal - s.regionRequestSender.client = innerClient - atomic.StoreUint32(&storeState, uint32(reachable)) - start := time.Now() - for { - if atomic.LoadInt32(&leaderStore.needForwarding) == 0 { - break - } - if time.Since(start) > 3*time.Second { - c.Fatal("store didn't recover to normal in time") - } - time.Sleep(time.Millisecond * 200) - } - atomic.StoreUint32(&storeState, uint32(unreachable)) - - req = tikvrpc.NewRequest(tikvrpc.CmdRawGet, &kvrpcpb.RawGetRequest{Key: []byte("k")}) - resp, ctx, err = s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV) - c.Assert(err, IsNil) - regionErr, err = resp.GetRegionError() - c.Assert(err, IsNil) - c.Assert(regionErr, IsNil) - c.Assert(resp.Resp.(*kvrpcpb.RawGetResponse).Value, BytesEquals, []byte("v1")) - c.Assert(ctx.ProxyStore, IsNil) - - // Simulate server down - s.regionRequestSender.client = &fnClient{fn: func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - if addr == leaderAddr || req.ForwardedHost == leaderAddr { - return nil, errors.New("simulated rpc error") - } - - // MockTiKV doesn't support forwarding. Simulate forwarding here. - if len(req.ForwardedHost) != 0 { - addr = req.ForwardedHost - } - return innerClient.SendRequest(ctx, addr, req, timeout) - }} - // The leader is changed after a store is down. - newLeaderPeerID := s.peerIDs[0] - if newLeaderPeerID == s.leaderPeer { - newLeaderPeerID = s.peerIDs[1] - } - - c.Assert(newLeaderPeerID, Not(Equals), s.leaderPeer) - s.cluster.ChangeLeader(s.regionID, newLeaderPeerID) - - req = tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("k"), - Value: []byte("v2"), - }) - resp, ctx, err = s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV) - c.Assert(err, IsNil) - regionErr, err = resp.GetRegionError() - c.Assert(err, IsNil) - // After several retries, the region will be marked as needReload. - // Then SendReqCtx will throw a pseudo EpochNotMatch to tell the caller to reload the region. - c.Assert(regionErr.EpochNotMatch, NotNil) - c.Assert(ctx, IsNil) - c.Assert(len(s.regionRequestSender.failStoreIDs), Equals, 0) - c.Assert(len(s.regionRequestSender.failProxyStoreIDs), Equals, 0) - region := s.regionRequestSender.regionCache.GetCachedRegionWithRLock(loc.Region) - c.Assert(region, NotNil) - c.Assert(region.checkNeedReload(), IsTrue) - - loc, err = s.regionRequestSender.regionCache.LocateKey(bo, []byte("k")) - c.Assert(err, IsNil) - req = tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("k"), - Value: []byte("v2"), - }) - resp, ctx, err = s.regionRequestSender.SendReqCtx(bo, req, loc.Region, time.Second, tikvrpc.TiKV) - c.Assert(err, IsNil) - regionErr, err = resp.GetRegionError() - c.Assert(err, IsNil) - c.Assert(regionErr, IsNil) - c.Assert(resp.Resp.(*kvrpcpb.RawPutResponse).Error, Equals, "") - // Leader changed - c.Assert(ctx.Store.storeID, Not(Equals), leaderStore.storeID) - c.Assert(ctx.ProxyStore, IsNil) -} - -func (s *testRegionRequestToSingleStoreSuite) TestGetRegionByIDFromCache(c *C) { - region, err := s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - - // test kv epochNotMatch return empty regions - s.cache.OnRegionEpochNotMatch(s.bo, &RPCContext{Region: region.Region, Store: &Store{storeID: s.store}}, []*metapb.Region{}) - c.Assert(err, IsNil) - r := s.cache.getRegionByIDFromCache(s.region) - c.Assert(r, IsNil) - - // refill cache - region, err = s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - - // test kv load new region with new start-key and new epoch - v2 := region.Region.confVer + 1 - r2 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: region.Region.ver, ConfVer: v2}, StartKey: []byte{1}} - st := &Store{storeID: s.store} - s.cache.insertRegionToCache(&Region{meta: &r2, store: unsafe.Pointer(st), lastAccess: time.Now().Unix()}) - region, err = s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - c.Assert(region.Region.confVer, Equals, v2) - c.Assert(region.Region.ver, Equals, region.Region.ver) - - v3 := region.Region.confVer + 1 - r3 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: v3, ConfVer: region.Region.confVer}, StartKey: []byte{2}} - st = &Store{storeID: s.store} - s.cache.insertRegionToCache(&Region{meta: &r3, store: unsafe.Pointer(st), lastAccess: time.Now().Unix()}) - region, err = s.cache.LocateRegionByID(s.bo, s.region) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - c.Assert(region.Region.confVer, Equals, region.Region.confVer) - c.Assert(region.Region.ver, Equals, v3) -} - -func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector(c *C) { - regionLoc, err := s.cache.LocateRegionByID(s.bo, s.regionID) - c.Assert(err, IsNil) - c.Assert(regionLoc, NotNil) - region := s.cache.GetCachedRegionWithRLock(regionLoc.Region) - regionStore := region.getStore() - - // Create a fake region and change its leader to the last peer. - regionStore = regionStore.clone() - regionStore.workTiKVIdx = AccessIndex(len(regionStore.stores) - 1) - sidx, _ := regionStore.accessStore(tiKVOnly, regionStore.workTiKVIdx) - regionStore.stores[sidx].epoch++ - regionStore.storeEpochs[sidx]++ - // Add a TiFlash peer to the region. - peer := &metapb.Peer{Id: s.cluster.AllocID(), StoreId: s.cluster.AllocID()} - regionStore.accessIndex[tiFlashOnly] = append(regionStore.accessIndex[tiFlashOnly], len(regionStore.stores)) - regionStore.stores = append(regionStore.stores, &Store{storeID: peer.StoreId, storeType: tikvrpc.TiFlash}) - regionStore.storeEpochs = append(regionStore.storeEpochs, 0) - - region = &Region{ - meta: region.GetMeta(), - } - region.lastAccess = time.Now().Unix() - region.meta.Peers = append(region.meta.Peers, peer) - atomic.StorePointer(®ion.store, unsafe.Pointer(regionStore)) - - cache := NewRegionCache(s.cache.pdClient) - defer cache.Close() - cache.insertRegionToCache(region) - - // Verify creating the replicaSelector. - replicaSelector, err := newReplicaSelector(cache, regionLoc.Region) - c.Assert(replicaSelector, NotNil) - c.Assert(err, IsNil) - c.Assert(replicaSelector.region, Equals, region) - // Should only contains TiKV stores. - c.Assert(len(replicaSelector.replicas), Equals, regionStore.accessStoreNum(tiKVOnly)) - c.Assert(len(replicaSelector.replicas), Equals, len(regionStore.stores)-1) - c.Assert(replicaSelector.nextReplicaIdx == 0, IsTrue) - c.Assert(replicaSelector.isExhausted(), IsFalse) - - // Verify that the store matches the peer and epoch. - for _, replica := range replicaSelector.replicas { - c.Assert(replica.store.storeID, Equals, replica.peer.GetStoreId()) - c.Assert(replica.peer, Equals, region.getPeerOnStore(replica.store.storeID)) - c.Assert(replica.attempts == 0, IsTrue) - - for i, store := range regionStore.stores { - if replica.store == store { - c.Assert(replica.epoch, Equals, regionStore.storeEpochs[i]) - } - } - } - // Verify that the leader replica is at the head of replicas. - leaderStore, leaderPeer, _, _ := region.WorkStorePeer(regionStore) - leaderReplica := replicaSelector.replicas[0] - c.Assert(leaderReplica.store, Equals, leaderStore) - c.Assert(leaderReplica.peer, Equals, leaderPeer) - - assertRPCCtxEqual := func(rpcCtx *RPCContext, replica *replica) { - c.Assert(rpcCtx.Store, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].store) - c.Assert(rpcCtx.Peer, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].peer) - c.Assert(rpcCtx.Addr, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].store.addr) - c.Assert(rpcCtx.AccessMode, Equals, tiKVOnly) - } - - // Verify the correctness of next() - for i := 0; i < len(replicaSelector.replicas); i++ { - rpcCtx, err := replicaSelector.next(s.bo) - c.Assert(rpcCtx, NotNil) - c.Assert(err, IsNil) - c.Assert(rpcCtx.Region, Equals, regionLoc.Region) - c.Assert(rpcCtx.Meta, Equals, region.meta) - replica := replicaSelector.replicas[replicaSelector.nextReplicaIdx-1] - assertRPCCtxEqual(rpcCtx, replica) - c.Assert(replica.attempts, Equals, 1) - c.Assert(replicaSelector.nextReplicaIdx, Equals, i+1) - } - c.Assert(replicaSelector.isExhausted(), IsTrue) - rpcCtx, err := replicaSelector.next(s.bo) - c.Assert(rpcCtx, IsNil) - c.Assert(err, IsNil) - // The region should be invalidated if runs out of all replicas. - c.Assert(replicaSelector.region.isValid(), IsFalse) - - region.lastAccess = time.Now().Unix() - replicaSelector, err = newReplicaSelector(cache, regionLoc.Region) - c.Assert(err, IsNil) - c.Assert(replicaSelector, NotNil) - cache.testingKnobs.mockRequestLiveness = func(s *Store, bo *retry.Backoffer) livenessState { - return reachable - } - for i := 0; i < maxReplicaAttempt; i++ { - rpcCtx, err := replicaSelector.next(s.bo) - c.Assert(rpcCtx, NotNil) - c.Assert(err, IsNil) - nextIdx := replicaSelector.nextReplicaIdx - // Verify that retry the same store if it's reachable. - replicaSelector.onSendFailure(s.bo, nil) - c.Assert(nextIdx, Equals, replicaSelector.nextReplicaIdx+1) - c.Assert(replicaSelector.nextReplica().attempts, Equals, i+1) - } - // Verify the maxReplicaAttempt limit for each replica. - rpcCtx, err = replicaSelector.next(s.bo) - c.Assert(err, IsNil) - assertRPCCtxEqual(rpcCtx, replicaSelector.replicas[1]) - c.Assert(replicaSelector.nextReplicaIdx, Equals, 2) - - // Verify updating leader. - replicaSelector, _ = newReplicaSelector(cache, regionLoc.Region) - replicaSelector.next(s.bo) - // The leader is the 3rd replica. After updating leader, it should be the next. - leader := replicaSelector.replicas[2] - replicaSelector.updateLeader(leader.peer) - c.Assert(replicaSelector.nextReplica(), Equals, leader) - c.Assert(replicaSelector.nextReplicaIdx, Equals, 1) - rpcCtx, _ = replicaSelector.next(s.bo) - assertRPCCtxEqual(rpcCtx, leader) - // Verify the regionStore is updated and the workTiKVIdx points to the leader. - regionStore = region.getStore() - leaderStore, leaderPeer, _, _ = region.WorkStorePeer(regionStore) - c.Assert(leaderStore, Equals, leader.store) - c.Assert(leaderPeer, Equals, leader.peer) - - replicaSelector, _ = newReplicaSelector(cache, regionLoc.Region) - replicaSelector.next(s.bo) - replicaSelector.next(s.bo) - replicaSelector.next(s.bo) - c.Assert(replicaSelector.isExhausted(), IsTrue) - // The leader is the 1st replica. After updating leader, it should be the next and - // the currnet replica is skipped. - leader = replicaSelector.replicas[0] - replicaSelector.updateLeader(leader.peer) - // The leader should be the next replica. - c.Assert(replicaSelector.nextReplica(), Equals, leader) - c.Assert(replicaSelector.nextReplicaIdx, Equals, 2) - rpcCtx, _ = replicaSelector.next(s.bo) - c.Assert(replicaSelector.isExhausted(), IsTrue) - assertRPCCtxEqual(rpcCtx, leader) - // Verify the regionStore is updated and the workTiKVIdx points to the leader. - regionStore = region.getStore() - leaderStore, leaderPeer, _, _ = region.WorkStorePeer(regionStore) - c.Assert(leaderStore, Equals, leader.store) - c.Assert(leaderPeer, Equals, leader.peer) - - // Give the leader one more chance even if it exceeds the maxReplicaAttempt. - replicaSelector, _ = newReplicaSelector(cache, regionLoc.Region) - leader = replicaSelector.replicas[0] - leader.attempts = maxReplicaAttempt - replicaSelector.updateLeader(leader.peer) - c.Assert(leader.attempts, Equals, maxReplicaAttempt-1) - rpcCtx, _ = replicaSelector.next(s.bo) - assertRPCCtxEqual(rpcCtx, leader) - c.Assert(leader.attempts, Equals, maxReplicaAttempt) - - // Invalidate the region if the leader is not in the region. - region.lastAccess = time.Now().Unix() - replicaSelector.updateLeader(&metapb.Peer{Id: s.cluster.AllocID(), StoreId: s.cluster.AllocID()}) - c.Assert(region.isValid(), IsFalse) - // Don't try next replica if the region is invalidated. - rpcCtx, err = replicaSelector.next(s.bo) - c.Assert(rpcCtx, IsNil) - c.Assert(err, IsNil) - - // Verify on send success. - region.lastAccess = time.Now().Unix() - replicaSelector, _ = newReplicaSelector(cache, regionLoc.Region) - replicaSelector.next(s.bo) - rpcCtx, err = replicaSelector.next(s.bo) - c.Assert(err, IsNil) - replicaSelector.OnSendSuccess() - // Verify the regionStore is updated and the workTiKVIdx points to the leader. - leaderStore, leaderPeer, _, _ = region.WorkStorePeer(region.getStore()) - c.Assert(leaderStore, Equals, rpcCtx.Store) - c.Assert(leaderPeer, Equals, rpcCtx.Peer) -} - -// TODO(youjiali1995): Remove duplicated tests. This test may be duplicated with other -// tests but it's a dedicated one to test sending requests with the replica selector. -func (s *testRegionRequestToThreeStoresSuite) TestSendReqWithReplicaSelector(c *C) { - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: []byte("key"), - Value: []byte("value"), - }) - region, err := s.cache.LocateRegionByID(s.bo, s.regionID) - c.Assert(err, IsNil) - c.Assert(region, NotNil) - - reloadRegion := func() { - s.regionRequestSender.leaderReplicaSelector.region.invalidate(Other) - region, _ = s.cache.LocateRegionByID(s.bo, s.regionID) - } - - hasFakeRegionError := func(resp *tikvrpc.Response) bool { - if resp == nil { - return false - } - regionErr, err := resp.GetRegionError() - if err != nil { - return false - } - return IsFakeRegionError(regionErr) - } - - // Normal - bo := retry.NewBackoffer(context.Background(), -1) - sender := s.regionRequestSender - resp, err := sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp, NotNil) - c.Assert(bo.GetTotalBackoffTimes() == 0, IsTrue) - - // Switch to the next Peer due to store failure and the leader is on the next peer. - bo = retry.NewBackoffer(context.Background(), -1) - s.cluster.ChangeLeader(s.regionID, s.peerIDs[1]) - s.cluster.StopStore(s.storeIDs[0]) - resp, err = sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp, NotNil) - c.Assert(sender.leaderReplicaSelector.nextReplicaIdx, Equals, 2) - c.Assert(bo.GetTotalBackoffTimes() == 1, IsTrue) - s.cluster.StartStore(s.storeIDs[0]) - - // Leader is updated because of send success, so no backoff. - bo = retry.NewBackoffer(context.Background(), -1) - resp, err = sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp, NotNil) - c.Assert(sender.leaderReplicaSelector.nextReplicaIdx, Equals, 1) - c.Assert(bo.GetTotalBackoffTimes() == 0, IsTrue) - - // Switch to the next peer due to leader failure but the new leader is not elected. - // Region will be invalidated due to store epoch changed. - reloadRegion() - s.cluster.StopStore(s.storeIDs[1]) - bo = retry.NewBackoffer(context.Background(), -1) - resp, err = sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(hasFakeRegionError(resp), IsTrue) - c.Assert(sender.leaderReplicaSelector.isExhausted(), IsFalse) - c.Assert(bo.GetTotalBackoffTimes(), Equals, 1) - s.cluster.StartStore(s.storeIDs[1]) - - // Leader is changed. No backoff. - reloadRegion() - s.cluster.ChangeLeader(s.regionID, s.peerIDs[0]) - bo = retry.NewBackoffer(context.Background(), -1) - resp, err = sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(resp, NotNil) - c.Assert(bo.GetTotalBackoffTimes(), Equals, 0) - - // No leader. Backoff for each replica and runs out all replicas. - s.cluster.GiveUpLeader(s.regionID) - bo = retry.NewBackoffer(context.Background(), -1) - resp, err = sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(hasFakeRegionError(resp), IsTrue) - c.Assert(bo.GetTotalBackoffTimes(), Equals, 3) - c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) - c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) - s.cluster.ChangeLeader(s.regionID, s.peerIDs[0]) - - // The leader store is alive but can't provide service. - // Region will be invalidated due to running out of all replicas. - s.regionRequestSender.regionCache.testingKnobs.mockRequestLiveness = func(s *Store, bo *retry.Backoffer) livenessState { - return reachable - } - reloadRegion() - s.cluster.StopStore(s.storeIDs[0]) - bo = retry.NewBackoffer(context.Background(), -1) - resp, err = sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(hasFakeRegionError(resp), IsTrue) - c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) - c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) - c.Assert(bo.GetTotalBackoffTimes(), Equals, maxReplicaAttempt+2) - s.cluster.StartStore(s.storeIDs[0]) - - // Verify that retry the same replica when meets ServerIsBusy/MaxTimestampNotSynced/ReadIndexNotReady/ProposalInMergingMode. - for _, regionErr := range []*errorpb.Error{ - // ServerIsBusy takes too much time to test. - // {ServerIsBusy: &errorpb.ServerIsBusy{}}, - {MaxTimestampNotSynced: &errorpb.MaxTimestampNotSynced{}}, - {ReadIndexNotReady: &errorpb.ReadIndexNotReady{}}, - {ProposalInMergingMode: &errorpb.ProposalInMergingMode{}}, - } { - func() { - oc := sender.client - defer func() { - sender.client = oc - }() - s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { - // Return the specific region error when accesses the leader. - if addr == s.cluster.GetStore(s.storeIDs[0]).Address { - return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: regionErr}}, nil - } - // Return the not leader error when accesses followers. - return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: &errorpb.Error{ - NotLeader: &errorpb.NotLeader{ - RegionId: region.Region.id, Leader: &metapb.Peer{Id: s.peerIDs[0], StoreId: s.storeIDs[0]}, - }}}}, nil - - }} - reloadRegion() - bo = retry.NewBackoffer(context.Background(), -1) - resp, err := sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(hasFakeRegionError(resp), IsTrue) - c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) - c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) - c.Assert(bo.GetTotalBackoffTimes(), Equals, maxReplicaAttempt+2) - }() - } - - // Verify switch to the next peer immediately when meets StaleCommand. - reloadRegion() - func() { - oc := sender.client - defer func() { - sender.client = oc - }() - s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { - return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: &errorpb.Error{StaleCommand: &errorpb.StaleCommand{}}}}, nil - }} - reloadRegion() - bo = retry.NewBackoffer(context.Background(), -1) - resp, err := sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(hasFakeRegionError(resp), IsTrue) - c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) - c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) - c.Assert(bo.GetTotalBackoffTimes(), Equals, 0) - }() - - // Verify don't invalidate region when meets unknown region errors. - reloadRegion() - func() { - oc := sender.client - defer func() { - sender.client = oc - }() - s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { - return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: &errorpb.Error{Message: ""}}}, nil - }} - reloadRegion() - bo = retry.NewBackoffer(context.Background(), -1) - resp, err := sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(hasFakeRegionError(resp), IsTrue) - c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) - c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) - c.Assert(bo.GetTotalBackoffTimes(), Equals, 0) - }() - - // Verify invalidate region when meets StoreNotMatch/RegionNotFound/EpochNotMatch/NotLeader and can't find the leader in region. - for i, regionErr := range []*errorpb.Error{ - {StoreNotMatch: &errorpb.StoreNotMatch{}}, - {RegionNotFound: &errorpb.RegionNotFound{}}, - {EpochNotMatch: &errorpb.EpochNotMatch{}}, - {NotLeader: &errorpb.NotLeader{Leader: &metapb.Peer{}}}} { - func() { - oc := sender.client - defer func() { - sender.client = oc - }() - s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { - return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: regionErr}}, nil - - }} - reloadRegion() - bo = retry.NewBackoffer(context.Background(), -1) - resp, err := sender.SendReq(bo, req, region.Region, time.Second) - - // Return a sendError when meets NotLeader and can't find the leader in the region. - if i == 3 { - c.Assert(err, IsNil) - c.Assert(hasFakeRegionError(resp), IsTrue) - } else { - c.Assert(err, IsNil) - c.Assert(resp, NotNil) - regionErr, _ := resp.GetRegionError() - c.Assert(regionErr, NotNil) - } - c.Assert(sender.leaderReplicaSelector.isExhausted(), IsFalse) - c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) - c.Assert(bo.GetTotalBackoffTimes(), Equals, 0) - }() - } - - // Runs out of all replicas and then returns a send error. - s.regionRequestSender.regionCache.testingKnobs.mockRequestLiveness = func(s *Store, bo *retry.Backoffer) livenessState { - return unreachable - } - reloadRegion() - for _, store := range s.storeIDs { - s.cluster.StopStore(store) - } - bo = retry.NewBackoffer(context.Background(), -1) - resp, err = sender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, IsNil) - c.Assert(hasFakeRegionError(resp), IsTrue) - c.Assert(bo.GetTotalBackoffTimes() == 3, IsTrue) - c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) - for _, store := range s.storeIDs { - s.cluster.StartStore(store) - } -} diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go deleted file mode 100644 index 008cd181dcb85..0000000000000 --- a/store/tikv/lock_resolver.go +++ /dev/null @@ -1,1002 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "container/list" - "context" - "encoding/hex" - "fmt" - "math" - "sync" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/config" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - pd "github.com/tikv/pd/client" - "go.uber.org/zap" -) - -// ResolvedCacheSize is max number of cached txn status. -const ResolvedCacheSize = 2048 - -// bigTxnThreshold : transaction involves keys exceed this threshold can be treated as `big transaction`. -const bigTxnThreshold = 16 - -// LockResolver resolves locks and also caches resolved txn status. -type LockResolver struct { - store *KVStore - mu struct { - sync.RWMutex - // resolved caches resolved txns (FIFO, txn id -> txnStatus). - resolved map[uint64]TxnStatus - recentResolved *list.List - } - testingKnobs struct { - meetLock func(locks []*Lock) - } -} - -func newLockResolver(store *KVStore) *LockResolver { - r := &LockResolver{ - store: store, - } - r.mu.resolved = make(map[uint64]TxnStatus) - r.mu.recentResolved = list.New() - return r -} - -// NewLockResolver is exported for other pkg to use, suppress unused warning. -var _ = NewLockResolver - -// NewLockResolver creates a LockResolver. -// It is exported for other pkg to use. For instance, binlog service needs -// to determine a transaction's commit state. -func NewLockResolver(etcdAddrs []string, security config.Security, opts ...pd.ClientOption) (*LockResolver, error) { - pdCli, err := pd.NewClient(etcdAddrs, pd.SecurityOption{ - CAPath: security.ClusterSSLCA, - CertPath: security.ClusterSSLCert, - KeyPath: security.ClusterSSLKey, - }, opts...) - if err != nil { - return nil, errors.Trace(err) - } - pdCli = util.InterceptedPDClient{Client: pdCli} - uuid := fmt.Sprintf("tikv-%v", pdCli.GetClusterID(context.TODO())) - - tlsConfig, err := security.ToTLSConfig() - if err != nil { - return nil, errors.Trace(err) - } - - spkv, err := NewEtcdSafePointKV(etcdAddrs, tlsConfig) - if err != nil { - return nil, errors.Trace(err) - } - - s, err := NewKVStore(uuid, locate.NewCodeCPDClient(pdCli), spkv, client.NewRPCClient(security)) - if err != nil { - return nil, errors.Trace(err) - } - return s.lockResolver, nil -} - -// TxnStatus represents a txn's final status. It should be Lock or Commit or Rollback. -type TxnStatus struct { - ttl uint64 - commitTS uint64 - action kvrpcpb.Action - primaryLock *kvrpcpb.LockInfo -} - -// IsCommitted returns true if the txn's final status is Commit. -func (s TxnStatus) IsCommitted() bool { return s.ttl == 0 && s.commitTS > 0 } - -// CommitTS returns the txn's commitTS. It is valid iff `IsCommitted` is true. -func (s TxnStatus) CommitTS() uint64 { return s.commitTS } - -// TTL returns the TTL of the transaction if the transaction is still alive. -func (s TxnStatus) TTL() uint64 { return s.ttl } - -// Action returns what the CheckTxnStatus request have done to the transaction. -func (s TxnStatus) Action() kvrpcpb.Action { return s.action } - -// StatusCacheable checks whether the transaction status is certain.True will be -// returned if its status is certain: -// If transaction is already committed, the result could be cached. -// Otherwise: -// If l.LockType is pessimistic lock type: -// - if its primary lock is pessimistic too, the check txn status result should not be cached. -// - if its primary lock is prewrite lock type, the check txn status could be cached. -// If l.lockType is prewrite lock type: -// - always cache the check txn status result. -// For prewrite locks, their primary keys should ALWAYS be the correct one and will NOT change. -func (s TxnStatus) StatusCacheable() bool { - if s.IsCommitted() { - return true - } - if s.ttl == 0 { - if s.action == kvrpcpb.Action_NoAction || - s.action == kvrpcpb.Action_LockNotExistRollback || - s.action == kvrpcpb.Action_TTLExpireRollback { - return true - } - } - return false -} - -// By default, locks after 3000ms is considered unusual (the client created the -// lock might be dead). Other client may cleanup this kind of lock. -// For locks created recently, we will do backoff and retry. -var defaultLockTTL uint64 = 3000 - -// ttl = ttlFactor * sqrt(writeSizeInMiB) -var ttlFactor = 6000 - -// Lock represents a lock from tikv server. -type Lock struct { - Key []byte - Primary []byte - TxnID uint64 - TTL uint64 - TxnSize uint64 - LockType kvrpcpb.Op - UseAsyncCommit bool - LockForUpdateTS uint64 - MinCommitTS uint64 -} - -func (l *Lock) String() string { - buf := bytes.NewBuffer(make([]byte, 0, 128)) - buf.WriteString("key: ") - buf.WriteString(hex.EncodeToString(l.Key)) - buf.WriteString(", primary: ") - buf.WriteString(hex.EncodeToString(l.Primary)) - return fmt.Sprintf("%s, txnStartTS: %d, lockForUpdateTS:%d, minCommitTs:%d, ttl: %d, type: %s, UseAsyncCommit: %t", - buf.String(), l.TxnID, l.LockForUpdateTS, l.MinCommitTS, l.TTL, l.LockType, l.UseAsyncCommit) -} - -// NewLock creates a new *Lock. -func NewLock(l *kvrpcpb.LockInfo) *Lock { - return &Lock{ - Key: l.GetKey(), - Primary: l.GetPrimaryLock(), - TxnID: l.GetLockVersion(), - TTL: l.GetLockTtl(), - TxnSize: l.GetTxnSize(), - LockType: l.LockType, - UseAsyncCommit: l.UseAsyncCommit, - LockForUpdateTS: l.LockForUpdateTs, - MinCommitTS: l.MinCommitTs, - } -} - -func (lr *LockResolver) saveResolved(txnID uint64, status TxnStatus) { - lr.mu.Lock() - defer lr.mu.Unlock() - - if _, ok := lr.mu.resolved[txnID]; ok { - return - } - lr.mu.resolved[txnID] = status - lr.mu.recentResolved.PushBack(txnID) - if len(lr.mu.resolved) > ResolvedCacheSize { - front := lr.mu.recentResolved.Front() - delete(lr.mu.resolved, front.Value.(uint64)) - lr.mu.recentResolved.Remove(front) - } -} - -func (lr *LockResolver) getResolved(txnID uint64) (TxnStatus, bool) { - lr.mu.RLock() - defer lr.mu.RUnlock() - - s, ok := lr.mu.resolved[txnID] - return s, ok -} - -// BatchResolveLocks resolve locks in a batch. -// Used it in gcworker only! -func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc locate.RegionVerID) (bool, error) { - if len(locks) == 0 { - return true, nil - } - - metrics.LockResolverCountWithBatchResolve.Inc() - - // The GCWorker kill all ongoing transactions, because it must make sure all - // locks have been cleaned before GC. - expiredLocks := locks - - txnInfos := make(map[uint64]uint64) - startTime := time.Now() - for _, l := range expiredLocks { - if _, ok := txnInfos[l.TxnID]; ok { - continue - } - metrics.LockResolverCountWithExpired.Inc() - - // Use currentTS = math.MaxUint64 means rollback the txn, no matter the lock is expired or not! - status, err := lr.getTxnStatus(bo, l.TxnID, l.Primary, 0, math.MaxUint64, true, false, l) - if err != nil { - return false, err - } - - // If the transaction uses async commit, CheckTxnStatus will reject rolling back the primary lock. - // Then we need to check the secondary locks to determine the final status of the transaction. - if status.primaryLock != nil && status.primaryLock.UseAsyncCommit { - resolveData, err := lr.checkAllSecondaries(bo, l, &status) - if err == nil { - txnInfos[l.TxnID] = resolveData.commitTs - continue - } - if _, ok := errors.Cause(err).(*nonAsyncCommitLock); ok { - status, err = lr.getTxnStatus(bo, l.TxnID, l.Primary, 0, math.MaxUint64, true, true, l) - if err != nil { - return false, err - } - } else { - return false, err - } - } - - if status.ttl > 0 { - logutil.BgLogger().Error("BatchResolveLocks fail to clean locks, this result is not expected!") - return false, errors.New("TiDB ask TiKV to rollback locks but it doesn't, the protocol maybe wrong") - } - - txnInfos[l.TxnID] = status.commitTS - } - logutil.BgLogger().Info("BatchResolveLocks: lookup txn status", - zap.Duration("cost time", time.Since(startTime)), - zap.Int("num of txn", len(txnInfos))) - - listTxnInfos := make([]*kvrpcpb.TxnInfo, 0, len(txnInfos)) - for txnID, status := range txnInfos { - listTxnInfos = append(listTxnInfos, &kvrpcpb.TxnInfo{ - Txn: txnID, - Status: status, - }) - } - - req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, &kvrpcpb.ResolveLockRequest{TxnInfos: listTxnInfos}) - startTime = time.Now() - resp, err := lr.store.SendReq(bo, req, loc, client.ReadTimeoutShort) - if err != nil { - return false, errors.Trace(err) - } - - regionErr, err := resp.GetRegionError() - if err != nil { - return false, errors.Trace(err) - } - - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return false, errors.Trace(err) - } - return false, nil - } - - if resp.Resp == nil { - return false, errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.ResolveLockResponse) - if keyErr := cmdResp.GetError(); keyErr != nil { - return false, errors.Errorf("unexpected resolve err: %s", keyErr) - } - - logutil.BgLogger().Info("BatchResolveLocks: resolve locks in a batch", - zap.Duration("cost time", time.Since(startTime)), - zap.Int("num of locks", len(expiredLocks))) - return true, nil -} - -// ResolveLocks tries to resolve Locks. The resolving process is in 3 steps: -// 1) Use the `lockTTL` to pick up all expired locks. Only locks that are too -// old are considered orphan locks and will be handled later. If all locks -// are expired then all locks will be resolved so the returned `ok` will be -// true, otherwise caller should sleep a while before retry. -// 2) For each lock, query the primary key to get txn(which left the lock)'s -// commit status. -// 3) Send `ResolveLock` cmd to the lock's region to resolve all locks belong to -// the same transaction. -func (lr *LockResolver) ResolveLocks(bo *Backoffer, callerStartTS uint64, locks []*Lock) (int64, []uint64 /*pushed*/, error) { - return lr.resolveLocks(bo, callerStartTS, locks, false, false) -} - -// ResolveLocksLite resolves locks while preventing scan whole region. -func (lr *LockResolver) ResolveLocksLite(bo *Backoffer, callerStartTS uint64, locks []*Lock) (int64, []uint64 /*pushed*/, error) { - return lr.resolveLocks(bo, callerStartTS, locks, false, true) -} - -func (lr *LockResolver) resolveLocks(bo *Backoffer, callerStartTS uint64, locks []*Lock, forWrite bool, lite bool) (int64, []uint64 /*pushed*/, error) { - if lr.testingKnobs.meetLock != nil { - lr.testingKnobs.meetLock(locks) - } - var msBeforeTxnExpired txnExpireTime - if len(locks) == 0 { - return msBeforeTxnExpired.value(), nil, nil - } - - if forWrite { - metrics.LockResolverCountWithResolveForWrite.Inc() - } else { - metrics.LockResolverCountWithResolve.Inc() - } - - var pushFail bool - // TxnID -> []Region, record resolved Regions. - // TODO: Maybe put it in LockResolver and share by all txns. - cleanTxns := make(map[uint64]map[locate.RegionVerID]struct{}) - var pushed []uint64 - // pushed is only used in the read operation. - if !forWrite { - pushed = make([]uint64, 0, len(locks)) - } - - var resolve func(*Lock, bool) error - resolve = func(l *Lock, forceSyncCommit bool) error { - status, err := lr.getTxnStatusFromLock(bo, l, callerStartTS, forceSyncCommit) - if err != nil { - return err - } - - if status.ttl == 0 { - metrics.LockResolverCountWithExpired.Inc() - // If the lock is committed or rollbacked, resolve lock. - cleanRegions, exists := cleanTxns[l.TxnID] - if !exists { - cleanRegions = make(map[locate.RegionVerID]struct{}) - cleanTxns[l.TxnID] = cleanRegions - } - - if status.primaryLock != nil && !forceSyncCommit && status.primaryLock.UseAsyncCommit && !exists { - err = lr.resolveLockAsync(bo, l, status) - if _, ok := errors.Cause(err).(*nonAsyncCommitLock); ok { - err = resolve(l, true) - } - } else if l.LockType == kvrpcpb.Op_PessimisticLock { - err = lr.resolvePessimisticLock(bo, l, cleanRegions) - } else { - err = lr.resolveLock(bo, l, status, lite, cleanRegions) - } - if err != nil { - return err - } - } else { - metrics.LockResolverCountWithNotExpired.Inc() - // If the lock is valid, the txn may be a pessimistic transaction. - // Update the txn expire time. - msBeforeLockExpired := lr.store.GetOracle().UntilExpired(l.TxnID, status.ttl, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) - msBeforeTxnExpired.update(msBeforeLockExpired) - if forWrite { - // Write conflict detected! - // If it's a optimistic conflict and current txn is earlier than the lock owner, - // abort current transaction. - // This could avoids the deadlock scene of two large transaction. - if l.LockType != kvrpcpb.Op_PessimisticLock && l.TxnID > callerStartTS { - metrics.LockResolverCountWithWriteConflict.Inc() - return tikverr.NewErrWriteConfictWithArgs(callerStartTS, l.TxnID, status.commitTS, l.Key) - } - } else { - if status.action != kvrpcpb.Action_MinCommitTSPushed { - pushFail = true - return nil - } - pushed = append(pushed, l.TxnID) - } - } - return nil - } - - for _, l := range locks { - err := resolve(l, false) - if err != nil { - msBeforeTxnExpired.update(0) - err = errors.Trace(err) - return msBeforeTxnExpired.value(), nil, err - } - } - if pushFail { - // If any of the lock fails to push minCommitTS, don't return the pushed array. - pushed = nil - } - - if msBeforeTxnExpired.value() > 0 && len(pushed) == 0 { - // If len(pushed) > 0, the caller will not block on the locks, it push the minCommitTS instead. - metrics.LockResolverCountWithWaitExpired.Inc() - } - return msBeforeTxnExpired.value(), pushed, nil -} - -func (lr *LockResolver) resolveLocksForWrite(bo *Backoffer, callerStartTS uint64, locks []*Lock) (int64, error) { - msBeforeTxnExpired, _, err := lr.resolveLocks(bo, callerStartTS, locks, true, false) - return msBeforeTxnExpired, err -} - -type txnExpireTime struct { - initialized bool - txnExpire int64 -} - -func (t *txnExpireTime) update(lockExpire int64) { - if lockExpire <= 0 { - lockExpire = 0 - } - if !t.initialized { - t.txnExpire = lockExpire - t.initialized = true - return - } - if lockExpire < t.txnExpire { - t.txnExpire = lockExpire - } -} - -func (t *txnExpireTime) value() int64 { - if !t.initialized { - return 0 - } - return t.txnExpire -} - -// GetTxnStatus queries tikv-server for a txn's status (commit/rollback). -// If the primary key is still locked, it will launch a Rollback to abort it. -// To avoid unnecessarily aborting too many txns, it is wiser to wait a few -// seconds before calling it after Prewrite. -func (lr *LockResolver) GetTxnStatus(txnID uint64, callerStartTS uint64, primary []byte) (TxnStatus, error) { - var status TxnStatus - bo := retry.NewBackoffer(context.Background(), cleanupMaxBackoff) - currentTS, err := lr.store.GetOracle().GetLowResolutionTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) - if err != nil { - return status, err - } - return lr.getTxnStatus(bo, txnID, primary, callerStartTS, currentTS, true, false, nil) -} - -func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStartTS uint64, forceSyncCommit bool) (TxnStatus, error) { - var currentTS uint64 - var err error - var status TxnStatus - - if l.TTL == 0 { - // NOTE: l.TTL = 0 is a special protocol!!! - // When the pessimistic txn prewrite meets locks of a txn, it should resolve the lock **unconditionally**. - // In this case, TiKV use lock TTL = 0 to notify TiDB, and TiDB should resolve the lock! - // Set currentTS to max uint64 to make the lock expired. - currentTS = math.MaxUint64 - } else { - currentTS, err = lr.store.GetOracle().GetLowResolutionTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) - if err != nil { - return TxnStatus{}, err - } - } - - rollbackIfNotExist := false - if _, err := util.EvalFailpoint("getTxnStatusDelay"); err == nil { - time.Sleep(100 * time.Millisecond) - } - for { - status, err = lr.getTxnStatus(bo, l.TxnID, l.Primary, callerStartTS, currentTS, rollbackIfNotExist, forceSyncCommit, l) - if err == nil { - return status, nil - } - // If the error is something other than txnNotFoundErr, throw the error (network - // unavailable, tikv down, backoff timeout etc) to the caller. - if _, ok := errors.Cause(err).(txnNotFoundErr); !ok { - return TxnStatus{}, err - } - - if _, err := util.EvalFailpoint("txnNotFoundRetTTL"); err == nil { - return TxnStatus{ttl: l.TTL, action: kvrpcpb.Action_NoAction}, nil - } - - // Handle txnNotFound error. - // getTxnStatus() returns it when the secondary locks exist while the primary lock doesn't. - // This is likely to happen in the concurrently prewrite when secondary regions - // success before the primary region. - if err := bo.Backoff(retry.BoTxnNotFound, err); err != nil { - logutil.Logger(bo.GetCtx()).Warn("getTxnStatusFromLock backoff fail", zap.Error(err)) - } - - if lr.store.GetOracle().UntilExpired(l.TxnID, l.TTL, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) <= 0 { - logutil.Logger(bo.GetCtx()).Warn("lock txn not found, lock has expired", - zap.Uint64("CallerStartTs", callerStartTS), - zap.Stringer("lock str", l)) - if l.LockType == kvrpcpb.Op_PessimisticLock { - if _, err := util.EvalFailpoint("txnExpireRetTTL"); err == nil { - return TxnStatus{action: kvrpcpb.Action_LockNotExistDoNothing}, - errors.New("error txn not found and lock expired") - } - } - // For pessimistic lock resolving, if the primary lock does not exist and rollbackIfNotExist is true, - // The Action_LockNotExistDoNothing will be returned as the status. - rollbackIfNotExist = true - } else { - if l.LockType == kvrpcpb.Op_PessimisticLock { - return TxnStatus{ttl: l.TTL}, nil - } - } - } -} - -type txnNotFoundErr struct { - *kvrpcpb.TxnNotFound -} - -func (e txnNotFoundErr) Error() string { - return e.TxnNotFound.String() -} - -// getTxnStatus sends the CheckTxnStatus request to the TiKV server. -// When rollbackIfNotExist is false, the caller should be careful with the txnNotFoundErr error. -func (lr *LockResolver) getTxnStatus(bo *Backoffer, txnID uint64, primary []byte, - callerStartTS, currentTS uint64, rollbackIfNotExist bool, forceSyncCommit bool, lockInfo *Lock) (TxnStatus, error) { - if s, ok := lr.getResolved(txnID); ok { - return s, nil - } - - metrics.LockResolverCountWithQueryTxnStatus.Inc() - - // CheckTxnStatus may meet the following cases: - // 1. LOCK - // 1.1 Lock expired -- orphan lock, fail to update TTL, crash recovery etc. - // 1.2 Lock TTL -- active transaction holding the lock. - // 2. NO LOCK - // 2.1 Txn Committed - // 2.2 Txn Rollbacked -- rollback itself, rollback by others, GC tomb etc. - // 2.3 No lock -- pessimistic lock rollback, concurrence prewrite. - - var status TxnStatus - resolvingPessimisticLock := lockInfo != nil && lockInfo.LockType == kvrpcpb.Op_PessimisticLock - req := tikvrpc.NewRequest(tikvrpc.CmdCheckTxnStatus, &kvrpcpb.CheckTxnStatusRequest{ - PrimaryKey: primary, - LockTs: txnID, - CallerStartTs: callerStartTS, - CurrentTs: currentTS, - RollbackIfNotExist: rollbackIfNotExist, - ForceSyncCommit: forceSyncCommit, - ResolvingPessimisticLock: resolvingPessimisticLock, - }) - for { - loc, err := lr.store.GetRegionCache().LocateKey(bo, primary) - if err != nil { - return status, errors.Trace(err) - } - resp, err := lr.store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) - if err != nil { - return status, errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return status, errors.Trace(err) - } - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return status, errors.Trace(err) - } - continue - } - if resp.Resp == nil { - return status, errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.CheckTxnStatusResponse) - if keyErr := cmdResp.GetError(); keyErr != nil { - txnNotFound := keyErr.GetTxnNotFound() - if txnNotFound != nil { - return status, txnNotFoundErr{txnNotFound} - } - - err = errors.Errorf("unexpected err: %s, tid: %v", keyErr, txnID) - logutil.BgLogger().Error("getTxnStatus error", zap.Error(err)) - return status, err - } - status.action = cmdResp.Action - status.primaryLock = cmdResp.LockInfo - - if status.primaryLock != nil && status.primaryLock.UseAsyncCommit && !forceSyncCommit { - if !lr.store.GetOracle().IsExpired(txnID, cmdResp.LockTtl, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) { - status.ttl = cmdResp.LockTtl - } - } else if cmdResp.LockTtl != 0 { - status.ttl = cmdResp.LockTtl - } else { - if cmdResp.CommitVersion == 0 { - metrics.LockResolverCountWithQueryTxnStatusRolledBack.Inc() - } else { - metrics.LockResolverCountWithQueryTxnStatusCommitted.Inc() - } - - status.commitTS = cmdResp.CommitVersion - if status.StatusCacheable() { - lr.saveResolved(txnID, status) - } - } - - return status, nil - } -} - -// asyncResolveData is data contributed by multiple goroutines when resolving locks using the async commit protocol. All -// data should be protected by the mutex field. -type asyncResolveData struct { - mutex sync.Mutex - // If any key has been committed (missingLock is true), then this is the commit ts. In that case, all locks should - // be committed with the same commit timestamp. If no locks have been committed (missingLock is false), then we will - // use max(all min commit ts) from all locks; i.e., it is the commit ts we should use. Note that a secondary lock's - // commit ts may or may not be the same as the primary lock's min commit ts. - commitTs uint64 - keys [][]byte - missingLock bool -} - -type nonAsyncCommitLock struct{} - -func (*nonAsyncCommitLock) Error() string { - return "CheckSecondaryLocks receives a non-async-commit lock" -} - -// addKeys adds the keys from locks to data, keeping other fields up to date. startTS and commitTS are for the -// transaction being resolved. -// -// In the async commit protocol when checking locks, we send a list of keys to check and get back a list of locks. There -// will be a lock for every key which is locked. If there are fewer locks than keys, then a lock is missing because it -// has been committed, rolled back, or was never locked. -// -// In this function, locks is the list of locks, and expected is the number of keys. asyncResolveData.missingLock will be -// set to true if the lengths don't match. If the lengths do match, then the locks are added to asyncResolveData.locks -// and will need to be resolved by the caller. -func (data *asyncResolveData) addKeys(locks []*kvrpcpb.LockInfo, expected int, startTS uint64, commitTS uint64) error { - data.mutex.Lock() - defer data.mutex.Unlock() - - // Check locks to see if any have been committed or rolled back. - if len(locks) < expected { - logutil.BgLogger().Debug("addKeys: lock has been committed or rolled back", zap.Uint64("commit ts", commitTS), zap.Uint64("start ts", startTS)) - // A lock is missing - the transaction must either have been rolled back or committed. - if !data.missingLock { - // commitTS == 0 => lock has been rolled back. - if commitTS != 0 && commitTS < data.commitTs { - return errors.Errorf("commit TS must be greater or equal to min commit TS: commit ts: %v, min commit ts: %v", commitTS, data.commitTs) - } - data.commitTs = commitTS - } - data.missingLock = true - - if data.commitTs != commitTS { - return errors.Errorf("commit TS mismatch in async commit recovery: %v and %v", data.commitTs, commitTS) - } - - // We do not need to resolve the remaining locks because TiKV will have resolved them as appropriate. - return nil - } - - logutil.BgLogger().Debug("addKeys: all locks present", zap.Uint64("start ts", startTS)) - // Save all locks to be resolved. - for _, lockInfo := range locks { - if lockInfo.LockVersion != startTS { - err := errors.Errorf("unexpected timestamp, expected: %v, found: %v", startTS, lockInfo.LockVersion) - logutil.BgLogger().Error("addLocks error", zap.Error(err)) - return err - } - if !lockInfo.UseAsyncCommit { - return &nonAsyncCommitLock{} - } - if !data.missingLock && lockInfo.MinCommitTs > data.commitTs { - data.commitTs = lockInfo.MinCommitTs - } - data.keys = append(data.keys, lockInfo.Key) - } - - return nil -} - -func (lr *LockResolver) checkSecondaries(bo *Backoffer, txnID uint64, curKeys [][]byte, curRegionID locate.RegionVerID, shared *asyncResolveData) error { - checkReq := &kvrpcpb.CheckSecondaryLocksRequest{ - Keys: curKeys, - StartVersion: txnID, - } - req := tikvrpc.NewRequest(tikvrpc.CmdCheckSecondaryLocks, checkReq) - metrics.LockResolverCountWithQueryCheckSecondaryLocks.Inc() - resp, err := lr.store.SendReq(bo, req, curRegionID, client.ReadTimeoutShort) - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - - logutil.BgLogger().Debug("checkSecondaries: region error, regrouping", zap.Uint64("txn id", txnID), zap.Uint64("region", curRegionID.GetID())) - - // If regions have changed, then we might need to regroup the keys. Since this should be rare and for the sake - // of simplicity, we will resolve regions sequentially. - regions, _, err := lr.store.GetRegionCache().GroupKeysByRegion(bo, curKeys, nil) - if err != nil { - return errors.Trace(err) - } - for regionID, keys := range regions { - // Recursion will terminate because the resolve request succeeds or the Backoffer reaches its limit. - if err = lr.checkSecondaries(bo, txnID, keys, regionID, shared); err != nil { - return err - } - } - return nil - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - - checkResp := resp.Resp.(*kvrpcpb.CheckSecondaryLocksResponse) - return shared.addKeys(checkResp.Locks, len(curKeys), txnID, checkResp.CommitTs) -} - -// resolveLockAsync resolves l assuming it was locked using the async commit protocol. -func (lr *LockResolver) resolveLockAsync(bo *Backoffer, l *Lock, status TxnStatus) error { - metrics.LockResolverCountWithResolveAsync.Inc() - - resolveData, err := lr.checkAllSecondaries(bo, l, &status) - if err != nil { - return err - } - - status.commitTS = resolveData.commitTs - - resolveData.keys = append(resolveData.keys, l.Primary) - keysByRegion, _, err := lr.store.GetRegionCache().GroupKeysByRegion(bo, resolveData.keys, nil) - if err != nil { - return errors.Trace(err) - } - - logutil.BgLogger().Info("resolve async commit", zap.Uint64("startTS", l.TxnID), zap.Uint64("commitTS", status.commitTS)) - - errChan := make(chan error, len(keysByRegion)) - // Resolve every lock in the transaction. - for region, locks := range keysByRegion { - curLocks := locks - curRegion := region - go func() { - errChan <- lr.resolveRegionLocks(bo, l, curRegion, curLocks, status) - }() - } - - var errs []string - for range keysByRegion { - err1 := <-errChan - if err1 != nil { - errs = append(errs, err1.Error()) - } - } - - if len(errs) > 0 { - return errors.Errorf("async commit recovery (sending ResolveLock) finished with errors: %v", errs) - } - - return nil -} - -// checkAllSecondaries checks the secondary locks of an async commit transaction to find out the final -// status of the transaction -func (lr *LockResolver) checkAllSecondaries(bo *Backoffer, l *Lock, status *TxnStatus) (*asyncResolveData, error) { - regions, _, err := lr.store.GetRegionCache().GroupKeysByRegion(bo, status.primaryLock.Secondaries, nil) - if err != nil { - return nil, errors.Trace(err) - } - - shared := asyncResolveData{ - mutex: sync.Mutex{}, - commitTs: status.primaryLock.MinCommitTs, - keys: [][]byte{}, - missingLock: false, - } - - errChan := make(chan error, len(regions)) - checkBo, cancel := bo.Fork() - defer cancel() - for regionID, keys := range regions { - curRegionID := regionID - curKeys := keys - - go func() { - errChan <- lr.checkSecondaries(checkBo, l.TxnID, curKeys, curRegionID, &shared) - }() - } - - for range regions { - err := <-errChan - if err != nil { - return nil, err - } - } - - return &shared, nil -} - -// resolveRegionLocks is essentially the same as resolveLock, but we resolve all keys in the same region at the same time. -func (lr *LockResolver) resolveRegionLocks(bo *Backoffer, l *Lock, region locate.RegionVerID, keys [][]byte, status TxnStatus) error { - lreq := &kvrpcpb.ResolveLockRequest{ - StartVersion: l.TxnID, - } - if status.IsCommitted() { - lreq.CommitVersion = status.CommitTS() - } - lreq.Keys = keys - req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, lreq) - - resp, err := lr.store.SendReq(bo, req, region, client.ReadTimeoutShort) - if err != nil { - return errors.Trace(err) - } - - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - - logutil.BgLogger().Info("resolveRegionLocks region error, regrouping", zap.String("lock", l.String()), zap.Uint64("region", region.GetID())) - - // Regroup locks. - regions, _, err := lr.store.GetRegionCache().GroupKeysByRegion(bo, keys, nil) - if err != nil { - return errors.Trace(err) - } - for regionID, keys := range regions { - // Recursion will terminate because the resolve request succeeds or the Backoffer reaches its limit. - if err = lr.resolveRegionLocks(bo, l, regionID, keys, status); err != nil { - return err - } - } - return nil - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.ResolveLockResponse) - if keyErr := cmdResp.GetError(); keyErr != nil { - err = errors.Errorf("unexpected resolve err: %s, lock: %v", keyErr, l) - logutil.BgLogger().Error("resolveLock error", zap.Error(err)) - } - - return nil -} - -func (lr *LockResolver) resolveLock(bo *Backoffer, l *Lock, status TxnStatus, lite bool, cleanRegions map[locate.RegionVerID]struct{}) error { - metrics.LockResolverCountWithResolveLocks.Inc() - resolveLite := lite || l.TxnSize < bigTxnThreshold - for { - loc, err := lr.store.GetRegionCache().LocateKey(bo, l.Key) - if err != nil { - return errors.Trace(err) - } - if _, ok := cleanRegions[loc.Region]; ok { - return nil - } - lreq := &kvrpcpb.ResolveLockRequest{ - StartVersion: l.TxnID, - } - if status.IsCommitted() { - lreq.CommitVersion = status.CommitTS() - } else { - logutil.BgLogger().Info("resolveLock rollback", zap.String("lock", l.String())) - } - - if resolveLite { - // Only resolve specified keys when it is a small transaction, - // prevent from scanning the whole region in this case. - metrics.LockResolverCountWithResolveLockLite.Inc() - lreq.Keys = [][]byte{l.Key} - } - req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, lreq) - resp, err := lr.store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - continue - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.ResolveLockResponse) - if keyErr := cmdResp.GetError(); keyErr != nil { - err = errors.Errorf("unexpected resolve err: %s, lock: %v", keyErr, l) - logutil.BgLogger().Error("resolveLock error", zap.Error(err)) - return err - } - if !resolveLite { - cleanRegions[loc.Region] = struct{}{} - } - return nil - } -} - -func (lr *LockResolver) resolvePessimisticLock(bo *Backoffer, l *Lock, cleanRegions map[locate.RegionVerID]struct{}) error { - metrics.LockResolverCountWithResolveLocks.Inc() - for { - loc, err := lr.store.GetRegionCache().LocateKey(bo, l.Key) - if err != nil { - return errors.Trace(err) - } - if _, ok := cleanRegions[loc.Region]; ok { - return nil - } - forUpdateTS := l.LockForUpdateTS - if forUpdateTS == 0 { - forUpdateTS = math.MaxUint64 - } - pessimisticRollbackReq := &kvrpcpb.PessimisticRollbackRequest{ - StartVersion: l.TxnID, - ForUpdateTs: forUpdateTS, - Keys: [][]byte{l.Key}, - } - req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticRollback, pessimisticRollbackReq) - resp, err := lr.store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - continue - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.PessimisticRollbackResponse) - if keyErr := cmdResp.GetErrors(); len(keyErr) > 0 { - err = errors.Errorf("unexpected resolve pessimistic lock err: %s, lock: %v", keyErr[0], l) - logutil.Logger(bo.GetCtx()).Error("resolveLock error", zap.Error(err)) - return err - } - return nil - } -} diff --git a/store/tikv/logutil/hex.go b/store/tikv/logutil/hex.go deleted file mode 100644 index cc657d311cad4..0000000000000 --- a/store/tikv/logutil/hex.go +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package logutil - -import ( - "bytes" - "encoding/hex" - "fmt" - "io" - "reflect" - "strings" - - "github.com/golang/protobuf/proto" -) - -// Hex defines a fmt.Stringer for proto.Message. -// We can't define the String() method on proto.Message, but we can wrap it. -func Hex(msg proto.Message) fmt.Stringer { - return hexStringer{msg} -} - -type hexStringer struct { - proto.Message -} - -func (h hexStringer) String() string { - val := reflect.ValueOf(h.Message) - var w bytes.Buffer - prettyPrint(&w, val) - return w.String() -} - -func prettyPrint(w io.Writer, val reflect.Value) { - tp := val.Type() - switch val.Kind() { - case reflect.Slice: - elemType := tp.Elem() - if elemType.Kind() == reflect.Uint8 { - fmt.Fprintf(w, "%s", hex.EncodeToString(val.Bytes())) - } else { - fmt.Fprintf(w, "%s", val.Interface()) - } - case reflect.Struct: - fmt.Fprintf(w, "{") - for i := 0; i < val.NumField(); i++ { - fv := val.Field(i) - ft := tp.Field(i) - if strings.HasPrefix(ft.Name, "XXX_") { - continue - } - if i != 0 { - fmt.Fprintf(w, " ") - } - fmt.Fprintf(w, "%s:", ft.Name) - prettyPrint(w, fv) - } - fmt.Fprintf(w, "}") - case reflect.Ptr: - if val.IsNil() { - fmt.Fprintf(w, "%v", val.Interface()) - } else { - prettyPrint(w, reflect.Indirect(val)) - } - default: - fmt.Fprintf(w, "%v", val.Interface()) - } -} diff --git a/store/tikv/logutil/log.go b/store/tikv/logutil/log.go deleted file mode 100644 index 9ddf5b98f644a..0000000000000 --- a/store/tikv/logutil/log.go +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package logutil - -import ( - "context" - - "github.com/pingcap/log" - "go.uber.org/zap" -) - -// BgLogger returns the default global logger. -func BgLogger() *zap.Logger { - return log.L() -} - -// Logger gets a contextual logger from current context. -// contextual logger will output common fields from context. -func Logger(ctx context.Context) *zap.Logger { - if ctxlogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { - return ctxlogger - } - return log.L() -} - -type ctxLogKeyType struct{} - -// CtxLogKey is the key to retrieve logger from context. -// It can be assigned to another value. -var CtxLogKey interface{} = ctxLogKeyType{} diff --git a/store/tikv/logutil/tracing.go b/store/tikv/logutil/tracing.go deleted file mode 100644 index e45f1a586bf5a..0000000000000 --- a/store/tikv/logutil/tracing.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package logutil - -import ( - "context" - "fmt" - - "github.com/opentracing/opentracing-go" - "github.com/opentracing/opentracing-go/log" -) - -// TraceEventKey presents the TraceEventKey in span log. -var TraceEventKey = "event" - -// Event records event in current tracing span. -func Event(ctx context.Context, event string) { - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span.LogFields(log.String(TraceEventKey, event)) - } -} - -// Eventf records event in current tracing span with format support. -func Eventf(ctx context.Context, format string, args ...interface{}) { - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span.LogFields(log.String(TraceEventKey, fmt.Sprintf(format, args...))) - } -} - -// SetTag sets tag kv-pair in current tracing span -func SetTag(ctx context.Context, key string, value interface{}) { - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span.SetTag(key, value) - } -} diff --git a/store/tikv/metrics/metrics.go b/store/tikv/metrics/metrics.go deleted file mode 100644 index de6a6f60b1a26..0000000000000 --- a/store/tikv/metrics/metrics.go +++ /dev/null @@ -1,590 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package metrics - -import ( - "github.com/prometheus/client_golang/prometheus" - dto "github.com/prometheus/client_model/go" -) - -// Client metrics. -var ( - TiKVTxnCmdHistogram *prometheus.HistogramVec - TiKVBackoffHistogram *prometheus.HistogramVec - TiKVSendReqHistogram *prometheus.HistogramVec - TiKVCoprocessorHistogram prometheus.Histogram - TiKVLockResolverCounter *prometheus.CounterVec - TiKVRegionErrorCounter *prometheus.CounterVec - TiKVTxnWriteKVCountHistogram prometheus.Histogram - TiKVTxnWriteSizeHistogram prometheus.Histogram - TiKVRawkvCmdHistogram *prometheus.HistogramVec - TiKVRawkvSizeHistogram *prometheus.HistogramVec - TiKVTxnRegionsNumHistogram *prometheus.HistogramVec - TiKVLoadSafepointCounter *prometheus.CounterVec - TiKVSecondaryLockCleanupFailureCounter *prometheus.CounterVec - TiKVRegionCacheCounter *prometheus.CounterVec - TiKVLocalLatchWaitTimeHistogram prometheus.Histogram - TiKVStatusDuration *prometheus.HistogramVec - TiKVStatusCounter *prometheus.CounterVec - TiKVBatchWaitDuration prometheus.Histogram - TiKVBatchSendLatency prometheus.Histogram - TiKVBatchWaitOverLoad prometheus.Counter - TiKVBatchPendingRequests *prometheus.HistogramVec - TiKVBatchRequests *prometheus.HistogramVec - TiKVBatchClientUnavailable prometheus.Histogram - TiKVBatchClientWaitEstablish prometheus.Histogram - TiKVRangeTaskStats *prometheus.GaugeVec - TiKVRangeTaskPushDuration *prometheus.HistogramVec - TiKVTokenWaitDuration prometheus.Histogram - TiKVTxnHeartBeatHistogram *prometheus.HistogramVec - TiKVPessimisticLockKeysDuration prometheus.Histogram - TiKVTTLLifeTimeReachCounter prometheus.Counter - TiKVNoAvailableConnectionCounter prometheus.Counter - TiKVTwoPCTxnCounter *prometheus.CounterVec - TiKVAsyncCommitTxnCounter *prometheus.CounterVec - TiKVOnePCTxnCounter *prometheus.CounterVec - TiKVStoreLimitErrorCounter *prometheus.CounterVec - TiKVGRPCConnTransientFailureCounter *prometheus.CounterVec - TiKVPanicCounter *prometheus.CounterVec - TiKVForwardRequestCounter *prometheus.CounterVec - TiKVTSFutureWaitDuration prometheus.Histogram - TiKVSafeTSUpdateCounter *prometheus.CounterVec - TiKVSafeTSUpdateStats *prometheus.GaugeVec - TiKVReplicaSelectorFailureCounter *prometheus.CounterVec - TiKVRequestRetryTimesHistogram prometheus.Histogram - TiKVTxnCommitBackoffSeconds prometheus.Histogram - TiKVTxnCommitBackoffCount prometheus.Histogram - TiKVSmallReadDuration prometheus.Histogram -) - -// Label constants. -const ( - LblType = "type" - LblResult = "result" - LblStore = "store" - LblCommit = "commit" - LblAbort = "abort" - LblRollback = "rollback" - LblBatchGet = "batch_get" - LblGet = "get" - LblLockKeys = "lock_keys" - LabelBatchRecvLoop = "batch-recv-loop" - LabelBatchSendLoop = "batch-send-loop" - LblAddress = "address" - LblFromStore = "from_store" - LblToStore = "to_store" -) - -func initMetrics(namespace, subsystem string) { - TiKVTxnCmdHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "txn_cmd_duration_seconds", - Help: "Bucketed histogram of processing time of txn cmds.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days - }, []string{LblType}) - - TiKVBackoffHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "backoff_seconds", - Help: "total backoff seconds of a single backoffer.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days - }, []string{LblType}) - - TiKVSendReqHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "request_seconds", - Help: "Bucketed histogram of sending request duration.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days - }, []string{LblType, LblStore}) - - TiKVCoprocessorHistogram = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "cop_duration_seconds", - Help: "Run duration of a single coprocessor task, includes backoff time.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days - }) - - TiKVLockResolverCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "lock_resolver_actions_total", - Help: "Counter of lock resolver actions.", - }, []string{LblType}) - - TiKVRegionErrorCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "region_err_total", - Help: "Counter of region errors.", - }, []string{LblType}) - - TiKVTxnWriteKVCountHistogram = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "txn_write_kv_num", - Help: "Count of kv pairs to write in a transaction.", - Buckets: prometheus.ExponentialBuckets(1, 4, 17), // 1 ~ 4G - }) - - TiKVTxnWriteSizeHistogram = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "txn_write_size_bytes", - Help: "Size of kv pairs to write in a transaction.", - Buckets: prometheus.ExponentialBuckets(16, 4, 17), // 16Bytes ~ 64GB - }) - - TiKVRawkvCmdHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "rawkv_cmd_seconds", - Help: "Bucketed histogram of processing time of rawkv cmds.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days - }, []string{LblType}) - - TiKVRawkvSizeHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "rawkv_kv_size_bytes", - Help: "Size of key/value to put, in bytes.", - Buckets: prometheus.ExponentialBuckets(1, 2, 30), // 1Byte ~ 512MB - }, []string{LblType}) - - TiKVTxnRegionsNumHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "txn_regions_num", - Help: "Number of regions in a transaction.", - Buckets: prometheus.ExponentialBuckets(1, 2, 25), // 1 ~ 16M - }, []string{LblType}) - - TiKVLoadSafepointCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "load_safepoint_total", - Help: "Counter of load safepoint.", - }, []string{LblType}) - - TiKVSecondaryLockCleanupFailureCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "lock_cleanup_task_total", - Help: "failure statistic of secondary lock cleanup task.", - }, []string{LblType}) - - TiKVRegionCacheCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "region_cache_operations_total", - Help: "Counter of region cache.", - }, []string{LblType, LblResult}) - - TiKVLocalLatchWaitTimeHistogram = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "local_latch_wait_seconds", - Help: "Wait time of a get local latch.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s - }) - - TiKVStatusDuration = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "kv_status_api_duration", - Help: "duration for kv status api.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s - }, []string{"store"}) - - TiKVStatusCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "kv_status_api_count", - Help: "Counter of access kv status api.", - }, []string{LblResult}) - - TiKVBatchWaitDuration = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_wait_duration", - Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s - Help: "batch wait duration", - }) - - TiKVBatchSendLatency = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_send_latency", - Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s - Help: "batch send latency", - }) - - TiKVBatchWaitOverLoad = prometheus.NewCounter( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_wait_overload", - Help: "event of tikv transport layer overload", - }) - - TiKVBatchPendingRequests = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_pending_requests", - Buckets: prometheus.ExponentialBuckets(1, 2, 8), - Help: "number of requests pending in the batch channel", - }, []string{"store"}) - - TiKVBatchRequests = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_requests", - Buckets: prometheus.ExponentialBuckets(1, 2, 8), - Help: "number of requests in one batch", - }, []string{"store"}) - - TiKVBatchClientUnavailable = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_client_unavailable_seconds", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days - Help: "batch client unavailable", - }) - - TiKVBatchClientWaitEstablish = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_client_wait_connection_establish", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days - Help: "batch client wait new connection establish", - }) - - TiKVRangeTaskStats = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "range_task_stats", - Help: "stat of range tasks", - }, []string{LblType, LblResult}) - - TiKVRangeTaskPushDuration = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "range_task_push_duration", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s - Help: "duration to push sub tasks to range task workers", - }, []string{LblType}) - - TiKVTokenWaitDuration = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_executor_token_wait_duration", - Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s - Help: "tidb txn token wait duration to process batches", - }) - - TiKVTxnHeartBeatHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "txn_heart_beat", - Help: "Bucketed histogram of the txn_heartbeat request duration.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s - }, []string{LblType}) - - TiKVPessimisticLockKeysDuration = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "pessimistic_lock_keys_duration", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 24), // 1ms ~ 8389s - Help: "tidb txn pessimistic lock keys duration", - }) - - TiKVTTLLifeTimeReachCounter = prometheus.NewCounter( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "ttl_lifetime_reach_total", - Help: "Counter of ttlManager live too long.", - }) - - TiKVNoAvailableConnectionCounter = prometheus.NewCounter( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "batch_client_no_available_connection_total", - Help: "Counter of no available batch client.", - }) - - TiKVTwoPCTxnCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "commit_txn_counter", - Help: "Counter of 2PC transactions.", - }, []string{LblType}) - - TiKVAsyncCommitTxnCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "async_commit_txn_counter", - Help: "Counter of async commit transactions.", - }, []string{LblType}) - - TiKVOnePCTxnCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "one_pc_txn_counter", - Help: "Counter of 1PC transactions.", - }, []string{LblType}) - - TiKVStoreLimitErrorCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "get_store_limit_token_error", - 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}) - - TiKVForwardRequestCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "forward_request_counter", - Help: "Counter of tikv request being forwarded through another node", - }, []string{LblFromStore, LblToStore, LblType, LblResult}) - - TiKVTSFutureWaitDuration = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "ts_future_wait_seconds", - Help: "Bucketed histogram of seconds cost for waiting timestamp future.", - Buckets: prometheus.ExponentialBuckets(0.000005, 2, 30), // 5us ~ 2560s - }) - - TiKVSafeTSUpdateCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "safets_update_counter", - Help: "Counter of tikv safe_ts being updated.", - }, []string{LblResult, LblStore}) - - TiKVSafeTSUpdateStats = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "safets_update_stats", - Help: "stat of tikv updating safe_ts stats", - }, []string{LblStore}) - TiKVReplicaSelectorFailureCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "replica_selector_failure_counter", - Help: "Counter of the reason why the replica selector cannot yield a potential leader.", - }, []string{LblType}) - TiKVRequestRetryTimesHistogram = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "request_retry_times", - Help: "Bucketed histogram of how many times a region request retries.", - Buckets: []float64{1, 2, 3, 4, 8, 16, 32, 64, 128, 256}, - }) - TiKVTxnCommitBackoffSeconds = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "txn_commit_backoff_seconds", - Help: "Bucketed histogram of the total backoff duration in committing a transaction.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 2097s - }) - TiKVTxnCommitBackoffCount = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "txn_commit_backoff_count", - Help: "Bucketed histogram of the backoff count in committing a transaction.", - Buckets: prometheus.ExponentialBuckets(1, 2, 12), // 1 ~ 2048 - }) - - // TiKVSmallReadDuration uses to collect small request read duration. - TiKVSmallReadDuration = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: "sli", // Always use "sli" to make it compatible with TiDB. - Name: "tikv_small_read_duration", - Help: "Read time of TiKV small read.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 28), // 0.5ms ~ 74h - }) - - initShortcuts() -} - -func init() { - initMetrics("tikv", "client_go") -} - -// InitMetrics initializes metrics variables with given namespace and subsystem name. -func InitMetrics(namespace, subsystem string) { - initMetrics(namespace, subsystem) -} - -// RegisterMetrics registers all metrics variables. -// Note: to change default namespace and subsystem name, call `InitMetrics` before registering. -func RegisterMetrics() { - prometheus.MustRegister(TiKVTxnCmdHistogram) - prometheus.MustRegister(TiKVBackoffHistogram) - prometheus.MustRegister(TiKVSendReqHistogram) - prometheus.MustRegister(TiKVCoprocessorHistogram) - prometheus.MustRegister(TiKVLockResolverCounter) - prometheus.MustRegister(TiKVRegionErrorCounter) - prometheus.MustRegister(TiKVTxnWriteKVCountHistogram) - prometheus.MustRegister(TiKVTxnWriteSizeHistogram) - prometheus.MustRegister(TiKVRawkvCmdHistogram) - prometheus.MustRegister(TiKVRawkvSizeHistogram) - prometheus.MustRegister(TiKVTxnRegionsNumHistogram) - prometheus.MustRegister(TiKVLoadSafepointCounter) - prometheus.MustRegister(TiKVSecondaryLockCleanupFailureCounter) - prometheus.MustRegister(TiKVRegionCacheCounter) - prometheus.MustRegister(TiKVLocalLatchWaitTimeHistogram) - prometheus.MustRegister(TiKVStatusDuration) - prometheus.MustRegister(TiKVStatusCounter) - prometheus.MustRegister(TiKVBatchWaitDuration) - prometheus.MustRegister(TiKVBatchSendLatency) - prometheus.MustRegister(TiKVBatchWaitOverLoad) - prometheus.MustRegister(TiKVBatchPendingRequests) - prometheus.MustRegister(TiKVBatchRequests) - prometheus.MustRegister(TiKVBatchClientUnavailable) - prometheus.MustRegister(TiKVBatchClientWaitEstablish) - prometheus.MustRegister(TiKVRangeTaskStats) - prometheus.MustRegister(TiKVRangeTaskPushDuration) - prometheus.MustRegister(TiKVTokenWaitDuration) - prometheus.MustRegister(TiKVTxnHeartBeatHistogram) - prometheus.MustRegister(TiKVPessimisticLockKeysDuration) - prometheus.MustRegister(TiKVTTLLifeTimeReachCounter) - prometheus.MustRegister(TiKVNoAvailableConnectionCounter) - prometheus.MustRegister(TiKVTwoPCTxnCounter) - prometheus.MustRegister(TiKVAsyncCommitTxnCounter) - prometheus.MustRegister(TiKVOnePCTxnCounter) - prometheus.MustRegister(TiKVStoreLimitErrorCounter) - prometheus.MustRegister(TiKVGRPCConnTransientFailureCounter) - prometheus.MustRegister(TiKVPanicCounter) - prometheus.MustRegister(TiKVForwardRequestCounter) - prometheus.MustRegister(TiKVTSFutureWaitDuration) - prometheus.MustRegister(TiKVSafeTSUpdateCounter) - prometheus.MustRegister(TiKVSafeTSUpdateStats) - prometheus.MustRegister(TiKVReplicaSelectorFailureCounter) - prometheus.MustRegister(TiKVRequestRetryTimesHistogram) - prometheus.MustRegister(TiKVTxnCommitBackoffSeconds) - prometheus.MustRegister(TiKVTxnCommitBackoffCount) - prometheus.MustRegister(TiKVSmallReadDuration) -} - -// readCounter reads the value of a prometheus.Counter. -// Returns -1 when failing to read the value. -func readCounter(m prometheus.Counter) int64 { - // Actually, it's not recommended to read the value of prometheus metric types directly: - // https://github.com/prometheus/client_golang/issues/486#issuecomment-433345239 - pb := &dto.Metric{} - // It's impossible to return an error though. - if err := m.Write(pb); err != nil { - return -1 - } - return int64(pb.GetCounter().GetValue()) -} - -// TxnCommitCounter is the counter of transactions committed with -// different protocols, i.e. 2PC, async-commit, 1PC. -type TxnCommitCounter struct { - TwoPC int64 `json:"twoPC"` - AsyncCommit int64 `json:"asyncCommit"` - OnePC int64 `json:"onePC"` -} - -// Sub returns the difference of two counters. -func (c TxnCommitCounter) Sub(rhs TxnCommitCounter) TxnCommitCounter { - new := TxnCommitCounter{} - new.TwoPC = c.TwoPC - rhs.TwoPC - new.AsyncCommit = c.AsyncCommit - rhs.AsyncCommit - new.OnePC = c.OnePC - rhs.OnePC - return new -} - -// GetTxnCommitCounter gets the TxnCommitCounter. -func GetTxnCommitCounter() TxnCommitCounter { - return TxnCommitCounter{ - TwoPC: readCounter(TwoPCTxnCounterOk), - AsyncCommit: readCounter(AsyncCommitTxnCounterOk), - OnePC: readCounter(OnePCTxnCounterOk), - } -} - -const smallTxnAffectRow = 20 - -// ObserveReadSLI observes the read SLI metric. -func ObserveReadSLI(readKeys uint64, readTime float64) { - if readKeys <= smallTxnAffectRow && readKeys != 0 && readTime != 0 { - TiKVSmallReadDuration.Observe(readTime) - } -} diff --git a/store/tikv/metrics/shortcuts.go b/store/tikv/metrics/shortcuts.go deleted file mode 100644 index 76463bcb9d628..0000000000000 --- a/store/tikv/metrics/shortcuts.go +++ /dev/null @@ -1,185 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package metrics - -import "github.com/prometheus/client_golang/prometheus" - -// Shortcuts for performance improvement. -var ( - TxnCmdHistogramWithCommit prometheus.Observer - TxnCmdHistogramWithRollback prometheus.Observer - TxnCmdHistogramWithBatchGet prometheus.Observer - TxnCmdHistogramWithGet prometheus.Observer - TxnCmdHistogramWithLockKeys prometheus.Observer - - RawkvCmdHistogramWithGet prometheus.Observer - RawkvCmdHistogramWithBatchGet prometheus.Observer - RawkvCmdHistogramWithBatchPut prometheus.Observer - RawkvCmdHistogramWithDelete prometheus.Observer - RawkvCmdHistogramWithBatchDelete prometheus.Observer - RawkvCmdHistogramWithRawScan prometheus.Observer - RawkvCmdHistogramWithRawReversScan prometheus.Observer - RawkvSizeHistogramWithKey prometheus.Observer - RawkvSizeHistogramWithValue prometheus.Observer - - BackoffHistogramRPC prometheus.Observer - BackoffHistogramLock prometheus.Observer - BackoffHistogramLockFast prometheus.Observer - BackoffHistogramPD prometheus.Observer - BackoffHistogramRegionMiss prometheus.Observer - BackoffHistogramRegionScheduling prometheus.Observer - BackoffHistogramServerBusy prometheus.Observer - BackoffHistogramStaleCmd prometheus.Observer - BackoffHistogramDataNotReady prometheus.Observer - BackoffHistogramEmpty prometheus.Observer - - TxnRegionsNumHistogramWithSnapshot prometheus.Observer - TxnRegionsNumHistogramPrewrite prometheus.Observer - TxnRegionsNumHistogramCommit prometheus.Observer - TxnRegionsNumHistogramCleanup prometheus.Observer - TxnRegionsNumHistogramPessimisticLock prometheus.Observer - TxnRegionsNumHistogramPessimisticRollback prometheus.Observer - TxnRegionsNumHistogramWithCoprocessor prometheus.Observer - TxnRegionsNumHistogramWithBatchCoprocessor prometheus.Observer - - LockResolverCountWithBatchResolve prometheus.Counter - LockResolverCountWithExpired prometheus.Counter - LockResolverCountWithNotExpired prometheus.Counter - LockResolverCountWithWaitExpired prometheus.Counter - LockResolverCountWithResolve prometheus.Counter - LockResolverCountWithResolveForWrite prometheus.Counter - LockResolverCountWithResolveAsync prometheus.Counter - LockResolverCountWithWriteConflict prometheus.Counter - LockResolverCountWithQueryTxnStatus prometheus.Counter - LockResolverCountWithQueryTxnStatusCommitted prometheus.Counter - LockResolverCountWithQueryTxnStatusRolledBack prometheus.Counter - LockResolverCountWithQueryCheckSecondaryLocks prometheus.Counter - LockResolverCountWithResolveLocks prometheus.Counter - LockResolverCountWithResolveLockLite prometheus.Counter - - RegionCacheCounterWithInvalidateRegionFromCacheOK prometheus.Counter - RegionCacheCounterWithSendFail prometheus.Counter - RegionCacheCounterWithGetRegionByIDOK prometheus.Counter - RegionCacheCounterWithGetRegionByIDError prometheus.Counter - RegionCacheCounterWithGetRegionOK prometheus.Counter - RegionCacheCounterWithGetRegionError prometheus.Counter - RegionCacheCounterWithScanRegionsOK prometheus.Counter - RegionCacheCounterWithScanRegionsError prometheus.Counter - RegionCacheCounterWithGetStoreOK prometheus.Counter - RegionCacheCounterWithGetStoreError prometheus.Counter - RegionCacheCounterWithInvalidateStoreRegionsOK prometheus.Counter - - TxnHeartBeatHistogramOK prometheus.Observer - TxnHeartBeatHistogramError prometheus.Observer - - StatusCountWithOK prometheus.Counter - StatusCountWithError prometheus.Counter - - SecondaryLockCleanupFailureCounterCommit prometheus.Counter - SecondaryLockCleanupFailureCounterRollback prometheus.Counter - - TwoPCTxnCounterOk prometheus.Counter - TwoPCTxnCounterError prometheus.Counter - - AsyncCommitTxnCounterOk prometheus.Counter - AsyncCommitTxnCounterError prometheus.Counter - - OnePCTxnCounterOk prometheus.Counter - OnePCTxnCounterError prometheus.Counter - OnePCTxnCounterFallback prometheus.Counter -) - -func initShortcuts() { - TxnCmdHistogramWithCommit = TiKVTxnCmdHistogram.WithLabelValues(LblCommit) - TxnCmdHistogramWithRollback = TiKVTxnCmdHistogram.WithLabelValues(LblRollback) - TxnCmdHistogramWithBatchGet = TiKVTxnCmdHistogram.WithLabelValues(LblBatchGet) - TxnCmdHistogramWithGet = TiKVTxnCmdHistogram.WithLabelValues(LblGet) - TxnCmdHistogramWithLockKeys = TiKVTxnCmdHistogram.WithLabelValues(LblLockKeys) - - RawkvCmdHistogramWithGet = TiKVRawkvCmdHistogram.WithLabelValues("get") - RawkvCmdHistogramWithBatchGet = TiKVRawkvCmdHistogram.WithLabelValues("batch_get") - RawkvCmdHistogramWithBatchPut = TiKVRawkvCmdHistogram.WithLabelValues("batch_put") - RawkvCmdHistogramWithDelete = TiKVRawkvCmdHistogram.WithLabelValues("delete") - RawkvCmdHistogramWithBatchDelete = TiKVRawkvCmdHistogram.WithLabelValues("batch_delete") - RawkvCmdHistogramWithRawScan = TiKVRawkvCmdHistogram.WithLabelValues("raw_scan") - RawkvCmdHistogramWithRawReversScan = TiKVRawkvCmdHistogram.WithLabelValues("raw_reverse_scan") - RawkvSizeHistogramWithKey = TiKVRawkvSizeHistogram.WithLabelValues("key") - RawkvSizeHistogramWithValue = TiKVRawkvSizeHistogram.WithLabelValues("value") - - BackoffHistogramRPC = TiKVBackoffHistogram.WithLabelValues("tikvRPC") - BackoffHistogramLock = TiKVBackoffHistogram.WithLabelValues("txnLock") - BackoffHistogramLockFast = TiKVBackoffHistogram.WithLabelValues("tikvLockFast") - BackoffHistogramPD = TiKVBackoffHistogram.WithLabelValues("pdRPC") - BackoffHistogramRegionMiss = TiKVBackoffHistogram.WithLabelValues("regionMiss") - BackoffHistogramRegionScheduling = TiKVBackoffHistogram.WithLabelValues("regionScheduling") - BackoffHistogramServerBusy = TiKVBackoffHistogram.WithLabelValues("serverBusy") - BackoffHistogramStaleCmd = TiKVBackoffHistogram.WithLabelValues("staleCommand") - BackoffHistogramDataNotReady = TiKVBackoffHistogram.WithLabelValues("dataNotReady") - BackoffHistogramEmpty = TiKVBackoffHistogram.WithLabelValues("") - - TxnRegionsNumHistogramWithSnapshot = TiKVTxnRegionsNumHistogram.WithLabelValues("snapshot") - TxnRegionsNumHistogramPrewrite = TiKVTxnRegionsNumHistogram.WithLabelValues("2pc_prewrite") - TxnRegionsNumHistogramCommit = TiKVTxnRegionsNumHistogram.WithLabelValues("2pc_commit") - TxnRegionsNumHistogramCleanup = TiKVTxnRegionsNumHistogram.WithLabelValues("2pc_cleanup") - TxnRegionsNumHistogramPessimisticLock = TiKVTxnRegionsNumHistogram.WithLabelValues("2pc_pessimistic_lock") - TxnRegionsNumHistogramPessimisticRollback = TiKVTxnRegionsNumHistogram.WithLabelValues("2pc_pessimistic_rollback") - TxnRegionsNumHistogramWithCoprocessor = TiKVTxnRegionsNumHistogram.WithLabelValues("coprocessor") - TxnRegionsNumHistogramWithBatchCoprocessor = TiKVTxnRegionsNumHistogram.WithLabelValues("batch_coprocessor") - - LockResolverCountWithBatchResolve = TiKVLockResolverCounter.WithLabelValues("batch_resolve") - LockResolverCountWithExpired = TiKVLockResolverCounter.WithLabelValues("expired") - LockResolverCountWithNotExpired = TiKVLockResolverCounter.WithLabelValues("not_expired") - LockResolverCountWithWaitExpired = TiKVLockResolverCounter.WithLabelValues("wait_expired") - LockResolverCountWithResolve = TiKVLockResolverCounter.WithLabelValues("resolve") - LockResolverCountWithResolveForWrite = TiKVLockResolverCounter.WithLabelValues("resolve_for_write") - LockResolverCountWithResolveAsync = TiKVLockResolverCounter.WithLabelValues("resolve_async_commit") - LockResolverCountWithWriteConflict = TiKVLockResolverCounter.WithLabelValues("write_conflict") - LockResolverCountWithQueryTxnStatus = TiKVLockResolverCounter.WithLabelValues("query_txn_status") - LockResolverCountWithQueryTxnStatusCommitted = TiKVLockResolverCounter.WithLabelValues("query_txn_status_committed") - LockResolverCountWithQueryTxnStatusRolledBack = TiKVLockResolverCounter.WithLabelValues("query_txn_status_rolled_back") - LockResolverCountWithQueryCheckSecondaryLocks = TiKVLockResolverCounter.WithLabelValues("query_check_secondary_locks") - LockResolverCountWithResolveLocks = TiKVLockResolverCounter.WithLabelValues("query_resolve_locks") - LockResolverCountWithResolveLockLite = TiKVLockResolverCounter.WithLabelValues("query_resolve_lock_lite") - - RegionCacheCounterWithInvalidateRegionFromCacheOK = TiKVRegionCacheCounter.WithLabelValues("invalidate_region_from_cache", "ok") - RegionCacheCounterWithSendFail = TiKVRegionCacheCounter.WithLabelValues("send_fail", "ok") - RegionCacheCounterWithGetRegionByIDOK = TiKVRegionCacheCounter.WithLabelValues("get_region_by_id", "ok") - RegionCacheCounterWithGetRegionByIDError = TiKVRegionCacheCounter.WithLabelValues("get_region_by_id", "err") - RegionCacheCounterWithGetRegionOK = TiKVRegionCacheCounter.WithLabelValues("get_region", "ok") - RegionCacheCounterWithGetRegionError = TiKVRegionCacheCounter.WithLabelValues("get_region", "err") - RegionCacheCounterWithScanRegionsOK = TiKVRegionCacheCounter.WithLabelValues("scan_regions", "ok") - RegionCacheCounterWithScanRegionsError = TiKVRegionCacheCounter.WithLabelValues("scan_regions", "err") - RegionCacheCounterWithGetStoreOK = TiKVRegionCacheCounter.WithLabelValues("get_store", "ok") - RegionCacheCounterWithGetStoreError = TiKVRegionCacheCounter.WithLabelValues("get_store", "err") - RegionCacheCounterWithInvalidateStoreRegionsOK = TiKVRegionCacheCounter.WithLabelValues("invalidate_store_regions", "ok") - - TxnHeartBeatHistogramOK = TiKVTxnHeartBeatHistogram.WithLabelValues("ok") - TxnHeartBeatHistogramError = TiKVTxnHeartBeatHistogram.WithLabelValues("err") - - StatusCountWithOK = TiKVStatusCounter.WithLabelValues("ok") - StatusCountWithError = TiKVStatusCounter.WithLabelValues("err") - - SecondaryLockCleanupFailureCounterCommit = TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("commit") - SecondaryLockCleanupFailureCounterRollback = TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("rollback") - - TwoPCTxnCounterOk = TiKVTwoPCTxnCounter.WithLabelValues("ok") - TwoPCTxnCounterError = TiKVTwoPCTxnCounter.WithLabelValues("err") - - AsyncCommitTxnCounterOk = TiKVAsyncCommitTxnCounter.WithLabelValues("ok") - AsyncCommitTxnCounterError = TiKVAsyncCommitTxnCounter.WithLabelValues("err") - - OnePCTxnCounterOk = TiKVOnePCTxnCounter.WithLabelValues("ok") - OnePCTxnCounterError = TiKVOnePCTxnCounter.WithLabelValues("err") - OnePCTxnCounterFallback = TiKVOnePCTxnCounter.WithLabelValues("fallback") -} diff --git a/store/tikv/mockstore/cluster/cluster.go b/store/tikv/mockstore/cluster/cluster.go deleted file mode 100644 index ef6f20953715f..0000000000000 --- a/store/tikv/mockstore/cluster/cluster.go +++ /dev/null @@ -1,44 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package cluster - -import ( - "time" - - "github.com/pingcap/kvproto/pkg/metapb" -) - -// Cluster simulates a TiKV cluster. -// It can be used to change cluster states in tests. -type Cluster interface { - // AllocID creates an unique ID in cluster. The ID could be used as either - // StoreID, RegionID, or PeerID. - AllocID() uint64 - // GetRegionByKey returns the Region and its leader whose range contains the key. - GetRegionByKey(key []byte) (*metapb.Region, *metapb.Peer) - // GetAllStores returns all Stores' meta. - GetAllStores() []*metapb.Store - // ScheduleDelay schedules a delay event for a transaction on a region. - ScheduleDelay(startTS, regionID uint64, dur time.Duration) - // Split splits a Region at the key (encoded) and creates new Region. - Split(regionID, newRegionID uint64, key []byte, peerIDs []uint64, leaderPeerID uint64) - // SplitRaw splits a Region at the key (not encoded) and creates new Region. - SplitRaw(regionID, newRegionID uint64, rawKey []byte, peerIDs []uint64, leaderPeerID uint64) *metapb.Region - // SplitKeys evenly splits the start, end key into "count" regions. - SplitKeys(start, end []byte, count int) - // AddStore adds a new Store to the cluster. - AddStore(storeID uint64, addr string, labels ...*metapb.StoreLabel) - // RemoveStore removes a Store from the cluster. - RemoveStore(storeID uint64) -} diff --git a/store/tikv/mockstore/deadlock/deadlock.go b/store/tikv/mockstore/deadlock/deadlock.go deleted file mode 100644 index 5f0d781427f4f..0000000000000 --- a/store/tikv/mockstore/deadlock/deadlock.go +++ /dev/null @@ -1,130 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package deadlock - -import ( - "fmt" - "sync" -) - -// Detector detects deadlock. -type Detector struct { - waitForMap map[uint64]*txnList - lock sync.Mutex -} - -type txnList struct { - txns []txnKeyHashPair -} - -type txnKeyHashPair struct { - txn uint64 - keyHash uint64 -} - -// NewDetector creates a new Detector. -func NewDetector() *Detector { - return &Detector{ - waitForMap: map[uint64]*txnList{}, - } -} - -// ErrDeadlock is returned when deadlock is detected. -type ErrDeadlock struct { - KeyHash uint64 -} - -func (e *ErrDeadlock) Error() string { - return fmt.Sprintf("deadlock(%d)", e.KeyHash) -} - -// Detect detects deadlock for the sourceTxn on a locked key. -func (d *Detector) Detect(sourceTxn, waitForTxn, keyHash uint64) *ErrDeadlock { - d.lock.Lock() - err := d.doDetect(sourceTxn, waitForTxn) - if err == nil { - d.register(sourceTxn, waitForTxn, keyHash) - } - d.lock.Unlock() - return err -} - -func (d *Detector) doDetect(sourceTxn, waitForTxn uint64) *ErrDeadlock { - list := d.waitForMap[waitForTxn] - if list == nil { - return nil - } - for _, nextTarget := range list.txns { - if nextTarget.txn == sourceTxn { - return &ErrDeadlock{KeyHash: nextTarget.keyHash} - } - if err := d.doDetect(sourceTxn, nextTarget.txn); err != nil { - return err - } - } - return nil -} - -func (d *Detector) register(sourceTxn, waitForTxn, keyHash uint64) { - list := d.waitForMap[sourceTxn] - pair := txnKeyHashPair{txn: waitForTxn, keyHash: keyHash} - if list == nil { - d.waitForMap[sourceTxn] = &txnList{txns: []txnKeyHashPair{pair}} - return - } - for _, tar := range list.txns { - if tar.txn == waitForTxn && tar.keyHash == keyHash { - return - } - } - list.txns = append(list.txns, pair) -} - -// CleanUp removes the wait for entry for the transaction. -func (d *Detector) CleanUp(txn uint64) { - d.lock.Lock() - delete(d.waitForMap, txn) - d.lock.Unlock() -} - -// CleanUpWaitFor removes a key in the wait for entry for the transaction. -func (d *Detector) CleanUpWaitFor(txn, waitForTxn, keyHash uint64) { - pair := txnKeyHashPair{txn: waitForTxn, keyHash: keyHash} - d.lock.Lock() - l := d.waitForMap[txn] - if l != nil { - for i, tar := range l.txns { - if tar == pair { - l.txns = append(l.txns[:i], l.txns[i+1:]...) - break - } - } - if len(l.txns) == 0 { - delete(d.waitForMap, txn) - } - } - d.lock.Unlock() - -} - -// Expire removes entries with TS smaller than minTS. -func (d *Detector) Expire(minTS uint64) { - d.lock.Lock() - for ts := range d.waitForMap { - if ts < minTS { - delete(d.waitForMap, ts) - } - } - d.lock.Unlock() -} diff --git a/store/tikv/mockstore/deadlock/deadlock_test.go b/store/tikv/mockstore/deadlock/deadlock_test.go deleted file mode 100644 index 6135650d617f3..0000000000000 --- a/store/tikv/mockstore/deadlock/deadlock_test.go +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package deadlock - -import ( - "testing" - - . "github.com/pingcap/check" -) - -func TestT(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&testDeadlockSuite{}) - -type testDeadlockSuite struct{} - -func (s *testDeadlockSuite) TestDeadlock(c *C) { - detector := NewDetector() - err := detector.Detect(1, 2, 100) - c.Assert(err, IsNil) - err = detector.Detect(2, 3, 200) - c.Assert(err, IsNil) - err = detector.Detect(3, 1, 300) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "deadlock(200)") - detector.CleanUp(2) - list2 := detector.waitForMap[2] - c.Assert(list2, IsNil) - - // After cycle is broken, no deadlock now. - err = detector.Detect(3, 1, 300) - c.Assert(err, IsNil) - list3 := detector.waitForMap[3] - c.Assert(list3.txns, HasLen, 1) - - // Different keyHash grows the list. - err = detector.Detect(3, 1, 400) - c.Assert(err, IsNil) - c.Assert(list3.txns, HasLen, 2) - - // Same waitFor and key hash doesn't grow the list. - err = detector.Detect(3, 1, 400) - c.Assert(err, IsNil) - c.Assert(list3.txns, HasLen, 2) - - detector.CleanUpWaitFor(3, 1, 300) - c.Assert(list3.txns, HasLen, 1) - detector.CleanUpWaitFor(3, 1, 400) - list3 = detector.waitForMap[3] - c.Assert(list3, IsNil) - detector.Expire(1) - c.Assert(detector.waitForMap, HasLen, 1) - detector.Expire(2) - c.Assert(detector.waitForMap, HasLen, 0) -} diff --git a/store/tikv/mockstore/mocktikv/cluster.go b/store/tikv/mockstore/mocktikv/cluster.go deleted file mode 100644 index 5d962564568f3..0000000000000 --- a/store/tikv/mockstore/mocktikv/cluster.go +++ /dev/null @@ -1,686 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "bytes" - "context" - "math" - "sort" - "sync" - "time" - - "github.com/golang/protobuf/proto" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - pd "github.com/tikv/pd/client" -) - -// Cluster simulates a TiKV cluster. It focuses on management and the change of -// meta data. A Cluster mainly includes following 3 kinds of meta data: -// 1) Region: A Region is a fragment of TiKV's data whose range is [start, end). -// The data of a Region is duplicated to multiple Peers and distributed in -// multiple Stores. -// 2) Peer: A Peer is a replica of a Region's data. All peers of a Region form -// a group, each group elects a Leader to provide services. -// 3) Store: A Store is a storage/service node. Try to think it as a TiKV server -// process. Only the store with request's Region's leader Peer could respond -// to client's request. -type Cluster struct { - sync.RWMutex - id uint64 - stores map[uint64]*Store - regions map[uint64]*Region - - mvccStore MVCCStore - - // delayEvents is used to control the execution sequence of rpc requests for test. - delayEvents map[delayKey]time.Duration - delayMu sync.Mutex -} - -type delayKey struct { - startTS uint64 - regionID uint64 -} - -// NewCluster creates an empty cluster. It needs to be bootstrapped before -// providing service. -func NewCluster(mvccStore MVCCStore) *Cluster { - return &Cluster{ - stores: make(map[uint64]*Store), - regions: make(map[uint64]*Region), - delayEvents: make(map[delayKey]time.Duration), - mvccStore: mvccStore, - } -} - -// AllocID creates an unique ID in cluster. The ID could be used as either -// StoreID, RegionID, or PeerID. -func (c *Cluster) AllocID() uint64 { - c.Lock() - defer c.Unlock() - - return c.allocID() -} - -// AllocIDs creates multiple IDs. -func (c *Cluster) AllocIDs(n int) []uint64 { - c.Lock() - defer c.Unlock() - - var ids []uint64 - for len(ids) < n { - ids = append(ids, c.allocID()) - } - return ids -} - -func (c *Cluster) allocID() uint64 { - c.id++ - return c.id -} - -// GetAllRegions gets all the regions in the cluster. -func (c *Cluster) GetAllRegions() []*Region { - regions := make([]*Region, 0, len(c.regions)) - for _, region := range c.regions { - regions = append(regions, region) - } - return regions -} - -// GetStore returns a Store's meta. -func (c *Cluster) GetStore(storeID uint64) *metapb.Store { - c.RLock() - defer c.RUnlock() - - if store := c.stores[storeID]; store != nil { - return proto.Clone(store.meta).(*metapb.Store) - } - return nil -} - -// GetAllStores returns all Stores' meta. -func (c *Cluster) GetAllStores() []*metapb.Store { - c.RLock() - defer c.RUnlock() - - stores := make([]*metapb.Store, 0, len(c.stores)) - for _, store := range c.stores { - stores = append(stores, proto.Clone(store.meta).(*metapb.Store)) - } - return stores -} - -// StopStore stops a store with storeID. -func (c *Cluster) StopStore(storeID uint64) { - c.Lock() - defer c.Unlock() - - if store := c.stores[storeID]; store != nil { - store.meta.State = metapb.StoreState_Offline - } -} - -// StartStore starts a store with storeID. -func (c *Cluster) StartStore(storeID uint64) { - c.Lock() - defer c.Unlock() - - if store := c.stores[storeID]; store != nil { - store.meta.State = metapb.StoreState_Up - } -} - -// CancelStore makes the store with cancel state true. -func (c *Cluster) CancelStore(storeID uint64) { - c.Lock() - defer c.Unlock() - - // A store returns context.Cancelled Error when cancel is true. - if store := c.stores[storeID]; store != nil { - store.cancel = true - } -} - -// UnCancelStore makes the store with cancel state false. -func (c *Cluster) UnCancelStore(storeID uint64) { - c.Lock() - defer c.Unlock() - - if store := c.stores[storeID]; store != nil { - store.cancel = false - } -} - -// GetStoreByAddr returns a Store's meta by an addr. -func (c *Cluster) GetStoreByAddr(addr string) *metapb.Store { - c.RLock() - defer c.RUnlock() - - for _, s := range c.stores { - if s.meta.GetAddress() == addr { - return proto.Clone(s.meta).(*metapb.Store) - } - } - return nil -} - -// GetAndCheckStoreByAddr checks and returns a Store's meta by an addr -func (c *Cluster) GetAndCheckStoreByAddr(addr string) (ss []*metapb.Store, err error) { - c.RLock() - defer c.RUnlock() - - for _, s := range c.stores { - if s.cancel { - err = context.Canceled - return - } - if s.meta.GetAddress() == addr { - ss = append(ss, proto.Clone(s.meta).(*metapb.Store)) - } - } - return -} - -// AddStore add a new Store to the cluster. -func (c *Cluster) AddStore(storeID uint64, addr string, labels ...*metapb.StoreLabel) { - c.Lock() - defer c.Unlock() - - c.stores[storeID] = newStore(storeID, addr, labels...) -} - -// RemoveStore removes a Store from the cluster. -func (c *Cluster) RemoveStore(storeID uint64) { - c.Lock() - defer c.Unlock() - - delete(c.stores, storeID) -} - -// MarkTombstone marks store as tombstone. -func (c *Cluster) MarkTombstone(storeID uint64) { - c.Lock() - defer c.Unlock() - nm := *c.stores[storeID].meta - nm.State = metapb.StoreState_Tombstone - c.stores[storeID].meta = &nm -} - -// UpdateStoreAddr updates store address for cluster. -func (c *Cluster) UpdateStoreAddr(storeID uint64, addr string, labels ...*metapb.StoreLabel) { - c.Lock() - defer c.Unlock() - c.stores[storeID] = newStore(storeID, addr, labels...) -} - -// GetRegion returns a Region's meta and leader ID. -func (c *Cluster) GetRegion(regionID uint64) (*metapb.Region, uint64) { - c.RLock() - defer c.RUnlock() - - r := c.regions[regionID] - if r == nil { - return nil, 0 - } - return proto.Clone(r.Meta).(*metapb.Region), r.leader -} - -// GetRegionByKey returns the Region and its leader whose range contains the key. -func (c *Cluster) GetRegionByKey(key []byte) (*metapb.Region, *metapb.Peer) { - c.RLock() - defer c.RUnlock() - - return c.getRegionByKeyNoLock(key) -} - -// getRegionByKeyNoLock returns the Region and its leader whose range contains the key without Lock. -func (c *Cluster) getRegionByKeyNoLock(key []byte) (*metapb.Region, *metapb.Peer) { - for _, r := range c.regions { - if regionContains(r.Meta.StartKey, r.Meta.EndKey, key) { - return proto.Clone(r.Meta).(*metapb.Region), proto.Clone(r.leaderPeer()).(*metapb.Peer) - } - } - return nil, nil -} - -// GetPrevRegionByKey returns the previous Region and its leader whose range contains the key. -func (c *Cluster) GetPrevRegionByKey(key []byte) (*metapb.Region, *metapb.Peer) { - c.RLock() - defer c.RUnlock() - - currentRegion, _ := c.getRegionByKeyNoLock(key) - if len(currentRegion.StartKey) == 0 { - return nil, nil - } - for _, r := range c.regions { - if bytes.Equal(r.Meta.EndKey, currentRegion.StartKey) { - return proto.Clone(r.Meta).(*metapb.Region), proto.Clone(r.leaderPeer()).(*metapb.Peer) - } - } - return nil, nil -} - -// GetRegionByID returns the Region and its leader whose ID is regionID. -func (c *Cluster) GetRegionByID(regionID uint64) (*metapb.Region, *metapb.Peer) { - c.RLock() - defer c.RUnlock() - - for _, r := range c.regions { - if r.Meta.GetId() == regionID { - return proto.Clone(r.Meta).(*metapb.Region), proto.Clone(r.leaderPeer()).(*metapb.Peer) - } - } - return nil, nil -} - -// ScanRegions returns at most `limit` regions from given `key` and their leaders. -func (c *Cluster) ScanRegions(startKey, endKey []byte, limit int) []*pd.Region { - c.RLock() - defer c.RUnlock() - - regions := make([]*Region, 0, len(c.regions)) - for _, region := range c.regions { - regions = append(regions, region) - } - - sort.Slice(regions, func(i, j int) bool { - return bytes.Compare(regions[i].Meta.GetStartKey(), regions[j].Meta.GetStartKey()) < 0 - }) - - startPos := sort.Search(len(regions), func(i int) bool { - if len(regions[i].Meta.GetEndKey()) == 0 { - return true - } - return bytes.Compare(regions[i].Meta.GetEndKey(), startKey) > 0 - }) - regions = regions[startPos:] - if len(endKey) > 0 { - endPos := sort.Search(len(regions), func(i int) bool { - return bytes.Compare(regions[i].Meta.GetStartKey(), endKey) >= 0 - }) - if endPos > 0 { - regions = regions[:endPos] - } - } - if limit > 0 && len(regions) > limit { - regions = regions[:limit] - } - - result := make([]*pd.Region, 0, len(regions)) - for _, region := range regions { - leader := region.leaderPeer() - if leader == nil { - leader = &metapb.Peer{} - } else { - leader = proto.Clone(leader).(*metapb.Peer) - } - - r := &pd.Region{ - Meta: proto.Clone(region.Meta).(*metapb.Region), - Leader: leader, - } - result = append(result, r) - } - - return result -} - -// Bootstrap creates the first Region. The Stores should be in the Cluster before -// bootstrap. -func (c *Cluster) Bootstrap(regionID uint64, storeIDs, peerIDs []uint64, leaderPeerID uint64) { - c.Lock() - defer c.Unlock() - - if len(storeIDs) != len(peerIDs) { - panic("len(storeIDs) != len(peerIDs)") - } - c.regions[regionID] = newRegion(regionID, storeIDs, peerIDs, leaderPeerID) -} - -// AddPeer adds a new Peer for the Region on the Store. -func (c *Cluster) AddPeer(regionID, storeID, peerID uint64) { - c.Lock() - defer c.Unlock() - - c.regions[regionID].addPeer(peerID, storeID) -} - -// RemovePeer removes the Peer from the Region. Note that if the Peer is leader, -// the Region will have no leader before calling ChangeLeader(). -func (c *Cluster) RemovePeer(regionID, storeID uint64) { - c.Lock() - defer c.Unlock() - - c.regions[regionID].removePeer(storeID) -} - -// ChangeLeader sets the Region's leader Peer. Caller should guarantee the Peer -// exists. -func (c *Cluster) ChangeLeader(regionID, leaderPeerID uint64) { - c.Lock() - defer c.Unlock() - - c.regions[regionID].changeLeader(leaderPeerID) -} - -// GiveUpLeader sets the Region's leader to 0. The Region will have no leader -// before calling ChangeLeader(). -func (c *Cluster) GiveUpLeader(regionID uint64) { - c.ChangeLeader(regionID, 0) -} - -// Split splits a Region at the key (encoded) and creates new Region. -func (c *Cluster) Split(regionID, newRegionID uint64, key []byte, peerIDs []uint64, leaderPeerID uint64) { - c.SplitRaw(regionID, newRegionID, NewMvccKey(key), peerIDs, leaderPeerID) -} - -// SplitRaw splits a Region at the key (not encoded) and creates new Region. -func (c *Cluster) SplitRaw(regionID, newRegionID uint64, rawKey []byte, peerIDs []uint64, leaderPeerID uint64) *metapb.Region { - c.Lock() - defer c.Unlock() - - newRegion := c.regions[regionID].split(newRegionID, rawKey, peerIDs, leaderPeerID) - c.regions[newRegionID] = newRegion - // The mocktikv should return a deep copy of meta info to avoid data race - meta := proto.Clone(newRegion.Meta) - return meta.(*metapb.Region) -} - -// Merge merges 2 regions, their key ranges should be adjacent. -func (c *Cluster) Merge(regionID1, regionID2 uint64) { - c.Lock() - defer c.Unlock() - - c.regions[regionID1].merge(c.regions[regionID2].Meta.GetEndKey()) - delete(c.regions, regionID2) -} - -// SplitKeys evenly splits the start, end key into "count" regions. -// Only works for single store. -func (c *Cluster) SplitKeys(start, end []byte, count int) { - c.splitRange(c.mvccStore, NewMvccKey(start), NewMvccKey(end), count) -} - -// ScheduleDelay schedules a delay event for a transaction on a region. -func (c *Cluster) ScheduleDelay(startTS, regionID uint64, dur time.Duration) { - c.delayMu.Lock() - c.delayEvents[delayKey{startTS: startTS, regionID: regionID}] = dur - c.delayMu.Unlock() -} - -// UpdateStoreLabels merge the target and owned labels together -func (c *Cluster) UpdateStoreLabels(storeID uint64, labels []*metapb.StoreLabel) { - c.Lock() - defer c.Unlock() - c.stores[storeID].mergeLabels(labels) -} - -func (c *Cluster) handleDelay(startTS, regionID uint64) { - key := delayKey{startTS: startTS, regionID: regionID} - c.delayMu.Lock() - dur, ok := c.delayEvents[key] - if ok { - delete(c.delayEvents, key) - } - c.delayMu.Unlock() - if ok { - time.Sleep(dur) - } -} - -func (c *Cluster) splitRange(mvccStore MVCCStore, start, end MvccKey, count int) { - c.Lock() - defer c.Unlock() - c.evacuateOldRegionRanges(start, end) - regionPairs := c.getEntriesGroupByRegions(mvccStore, start, end, count) - c.createNewRegions(regionPairs, start, end) -} - -// getEntriesGroupByRegions groups the key value pairs into splitted regions. -func (c *Cluster) getEntriesGroupByRegions(mvccStore MVCCStore, start, end MvccKey, count int) [][]Pair { - startTS := uint64(math.MaxUint64) - limit := math.MaxInt32 - pairs := mvccStore.Scan(start.Raw(), end.Raw(), limit, startTS, kvrpcpb.IsolationLevel_SI, nil) - regionEntriesSlice := make([][]Pair, 0, count) - quotient := len(pairs) / count - remainder := len(pairs) % count - i := 0 - for i < len(pairs) { - regionEntryCount := quotient - if remainder > 0 { - remainder-- - regionEntryCount++ - } - regionEntries := pairs[i : i+regionEntryCount] - regionEntriesSlice = append(regionEntriesSlice, regionEntries) - i += regionEntryCount - } - return regionEntriesSlice -} - -func (c *Cluster) createNewRegions(regionPairs [][]Pair, start, end MvccKey) { - for i := range regionPairs { - peerID := c.allocID() - newRegion := newRegion(c.allocID(), []uint64{c.firstStoreID()}, []uint64{peerID}, peerID) - var regionStartKey, regionEndKey MvccKey - if i == 0 { - regionStartKey = start - } else { - regionStartKey = NewMvccKey(regionPairs[i][0].Key) - } - if i == len(regionPairs)-1 { - regionEndKey = end - } else { - // Use the next region's first key as region end key. - regionEndKey = NewMvccKey(regionPairs[i+1][0].Key) - } - newRegion.updateKeyRange(regionStartKey, regionEndKey) - c.regions[newRegion.Meta.Id] = newRegion - } -} - -// evacuateOldRegionRanges evacuate the range [start, end]. -// Old regions has intersection with [start, end) will be updated or deleted. -func (c *Cluster) evacuateOldRegionRanges(start, end MvccKey) { - oldRegions := c.getRegionsCoverRange(start, end) - for _, oldRegion := range oldRegions { - startCmp := bytes.Compare(oldRegion.Meta.StartKey, start) - endCmp := bytes.Compare(oldRegion.Meta.EndKey, end) - if len(oldRegion.Meta.EndKey) == 0 { - endCmp = 1 - } - if startCmp >= 0 && endCmp <= 0 { - // The region is within table data, it will be replaced by new regions. - delete(c.regions, oldRegion.Meta.Id) - } else if startCmp < 0 && endCmp > 0 { - // A single Region covers table data, split into two regions that do not overlap table data. - oldEnd := oldRegion.Meta.EndKey - oldRegion.updateKeyRange(oldRegion.Meta.StartKey, start) - peerID := c.allocID() - newRegion := newRegion(c.allocID(), []uint64{c.firstStoreID()}, []uint64{peerID}, peerID) - newRegion.updateKeyRange(end, oldEnd) - c.regions[newRegion.Meta.Id] = newRegion - } else if startCmp < 0 { - oldRegion.updateKeyRange(oldRegion.Meta.StartKey, start) - } else { - oldRegion.updateKeyRange(end, oldRegion.Meta.EndKey) - } - } -} - -func (c *Cluster) firstStoreID() uint64 { - for id := range c.stores { - return id - } - return 0 -} - -// getRegionsCoverRange gets regions in the cluster that has intersection with [start, end). -func (c *Cluster) getRegionsCoverRange(start, end MvccKey) []*Region { - regions := make([]*Region, 0, len(c.regions)) - for _, region := range c.regions { - onRight := bytes.Compare(end, region.Meta.StartKey) <= 0 - onLeft := bytes.Compare(region.Meta.EndKey, start) <= 0 - if len(region.Meta.EndKey) == 0 { - onLeft = false - } - if onLeft || onRight { - continue - } - regions = append(regions, region) - } - return regions -} - -// Region is the Region meta data. -type Region struct { - Meta *metapb.Region - leader uint64 -} - -func newPeerMeta(peerID, storeID uint64) *metapb.Peer { - return &metapb.Peer{ - Id: peerID, - StoreId: storeID, - } -} - -func newRegion(regionID uint64, storeIDs, peerIDs []uint64, leaderPeerID uint64) *Region { - if len(storeIDs) != len(peerIDs) { - panic("len(storeIDs) != len(peerIds)") - } - peers := make([]*metapb.Peer, 0, len(storeIDs)) - for i := range storeIDs { - peers = append(peers, newPeerMeta(peerIDs[i], storeIDs[i])) - } - meta := &metapb.Region{ - Id: regionID, - Peers: peers, - } - return &Region{ - Meta: meta, - leader: leaderPeerID, - } -} - -func (r *Region) addPeer(peerID, storeID uint64) { - r.Meta.Peers = append(r.Meta.Peers, newPeerMeta(peerID, storeID)) - r.incConfVer() -} - -func (r *Region) removePeer(peerID uint64) { - for i, peer := range r.Meta.Peers { - if peer.GetId() == peerID { - r.Meta.Peers = append(r.Meta.Peers[:i], r.Meta.Peers[i+1:]...) - break - } - } - if r.leader == peerID { - r.leader = 0 - } - r.incConfVer() -} - -func (r *Region) changeLeader(leaderID uint64) { - r.leader = leaderID -} - -func (r *Region) leaderPeer() *metapb.Peer { - for _, p := range r.Meta.Peers { - if p.GetId() == r.leader { - return p - } - } - return nil -} - -func (r *Region) split(newRegionID uint64, key MvccKey, peerIDs []uint64, leaderPeerID uint64) *Region { - if len(r.Meta.Peers) != len(peerIDs) { - panic("len(r.meta.Peers) != len(peerIDs)") - } - storeIDs := make([]uint64, 0, len(r.Meta.Peers)) - for _, peer := range r.Meta.Peers { - storeIDs = append(storeIDs, peer.GetStoreId()) - } - region := newRegion(newRegionID, storeIDs, peerIDs, leaderPeerID) - region.updateKeyRange(key, r.Meta.EndKey) - r.updateKeyRange(r.Meta.StartKey, key) - return region -} - -func (r *Region) merge(endKey MvccKey) { - r.Meta.EndKey = endKey - r.incVersion() -} - -func (r *Region) updateKeyRange(start, end MvccKey) { - r.Meta.StartKey = start - r.Meta.EndKey = end - r.incVersion() -} - -func (r *Region) incConfVer() { - r.Meta.RegionEpoch = &metapb.RegionEpoch{ - ConfVer: r.Meta.GetRegionEpoch().GetConfVer() + 1, - Version: r.Meta.GetRegionEpoch().GetVersion(), - } -} - -func (r *Region) incVersion() { - r.Meta.RegionEpoch = &metapb.RegionEpoch{ - ConfVer: r.Meta.GetRegionEpoch().GetConfVer(), - Version: r.Meta.GetRegionEpoch().GetVersion() + 1, - } -} - -// Store is the Store's meta data. -type Store struct { - meta *metapb.Store - cancel bool // return context.Cancelled error when cancel is true. -} - -func newStore(storeID uint64, addr string, labels ...*metapb.StoreLabel) *Store { - return &Store{ - meta: &metapb.Store{ - Id: storeID, - Address: addr, - Labels: labels, - }, - } -} - -func (s *Store) mergeLabels(labels []*metapb.StoreLabel) { - if len(s.meta.Labels) < 1 { - s.meta.Labels = labels - return - } - kv := make(map[string]string, len(s.meta.Labels)) - for _, label := range s.meta.Labels { - kv[label.Key] = label.Value - } - for _, label := range labels { - kv[label.Key] = label.Value - } - mergedLabels := make([]*metapb.StoreLabel, 0, len(kv)) - for k, v := range kv { - mergedLabels = append(mergedLabels, &metapb.StoreLabel{ - Key: k, - Value: v, - }) - } - s.meta.Labels = mergedLabels -} diff --git a/store/tikv/mockstore/mocktikv/cluster_manipulate.go b/store/tikv/mockstore/mocktikv/cluster_manipulate.go deleted file mode 100644 index 11512521609f5..0000000000000 --- a/store/tikv/mockstore/mocktikv/cluster_manipulate.go +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "fmt" - - "github.com/pingcap/kvproto/pkg/metapb" -) - -// BootstrapWithSingleStore initializes a Cluster with 1 Region and 1 Store. -func BootstrapWithSingleStore(cluster *Cluster) (storeID, peerID, regionID uint64) { - ids := cluster.AllocIDs(3) - storeID, peerID, regionID = ids[0], ids[1], ids[2] - cluster.AddStore(storeID, fmt.Sprintf("store%d", storeID)) - cluster.Bootstrap(regionID, []uint64{storeID}, []uint64{peerID}, peerID) - return -} - -// BootstrapWithMultiStores initializes a Cluster with 1 Region and n Stores. -func BootstrapWithMultiStores(cluster *Cluster, n int) (storeIDs, peerIDs []uint64, regionID uint64, leaderPeer uint64) { - storeIDs = cluster.AllocIDs(n) - peerIDs = cluster.AllocIDs(n) - leaderPeer = peerIDs[0] - regionID = cluster.AllocID() - for _, storeID := range storeIDs { - labels := []*metapb.StoreLabel{ - { - Key: "id", - Value: fmt.Sprintf("%v", storeID), - }, - } - cluster.AddStore(storeID, fmt.Sprintf("store%d", storeID), labels...) - } - cluster.Bootstrap(regionID, storeIDs, peerIDs, leaderPeer) - return -} - -// BootstrapWithMultiRegions initializes a Cluster with multiple Regions and 1 -// Store. The number of Regions will be len(splitKeys) + 1. -func BootstrapWithMultiRegions(cluster *Cluster, splitKeys ...[]byte) (storeID uint64, regionIDs, peerIDs []uint64) { - var firstRegionID, firstPeerID uint64 - storeID, firstPeerID, firstRegionID = BootstrapWithSingleStore(cluster) - regionIDs = append([]uint64{firstRegionID}, cluster.AllocIDs(len(splitKeys))...) - peerIDs = append([]uint64{firstPeerID}, cluster.AllocIDs(len(splitKeys))...) - for i, k := range splitKeys { - cluster.Split(regionIDs[i], regionIDs[i+1], k, []uint64{peerIDs[i]}, peerIDs[i]) - } - return -} diff --git a/store/tikv/mockstore/mocktikv/errors.go b/store/tikv/mockstore/mocktikv/errors.go deleted file mode 100644 index 15ecbec852478..0000000000000 --- a/store/tikv/mockstore/mocktikv/errors.go +++ /dev/null @@ -1,122 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "encoding/hex" - "fmt" - - "github.com/pingcap/kvproto/pkg/kvrpcpb" -) - -// ErrLocked is returned when trying to Read/Write on a locked key. Client should -// backoff or cleanup the lock then retry. -type ErrLocked struct { - Key MvccKey - Primary []byte - StartTS uint64 - ForUpdateTS uint64 - TTL uint64 - TxnSize uint64 - LockType kvrpcpb.Op -} - -// Error formats the lock to a string. -func (e *ErrLocked) Error() string { - return fmt.Sprintf("key is locked, key: %q, primary: %q, txnStartTS: %v, forUpdateTs: %v, LockType: %v", - e.Key, e.Primary, e.StartTS, e.ForUpdateTS, e.LockType) -} - -// ErrKeyAlreadyExist is returned when key exists but this key has a constraint that -// it should not exist. Client should return duplicated entry error. -type ErrKeyAlreadyExist struct { - Key []byte -} - -func (e *ErrKeyAlreadyExist) Error() string { - return fmt.Sprintf("key already exist, key: %q", e.Key) -} - -// ErrRetryable suggests that client may restart the txn. -type ErrRetryable string - -func (e ErrRetryable) Error() string { - return fmt.Sprintf("retryable: %s", string(e)) -} - -// ErrAbort means something is wrong and client should abort the txn. -type ErrAbort string - -func (e ErrAbort) Error() string { - return fmt.Sprintf("abort: %s", string(e)) -} - -// ErrAlreadyCommitted is returned specially when client tries to rollback a -// committed lock. -type ErrAlreadyCommitted uint64 - -func (e ErrAlreadyCommitted) Error() string { - return "txn already committed" -} - -// ErrAlreadyRollbacked is returned when lock operation meets rollback write record -type ErrAlreadyRollbacked struct { - startTS uint64 - key []byte -} - -func (e *ErrAlreadyRollbacked) Error() string { - return fmt.Sprintf("txn=%v on key=%s is already rolled back", e.startTS, hex.EncodeToString(e.key)) -} - -// ErrConflict is returned when the commitTS of key in the DB is greater than startTS. -type ErrConflict struct { - StartTS uint64 - ConflictTS uint64 - ConflictCommitTS uint64 - Key []byte -} - -func (e *ErrConflict) Error() string { - return "write conflict" -} - -// ErrDeadlock is returned when deadlock error is detected. -type ErrDeadlock struct { - LockTS uint64 - LockKey []byte - DealockKeyHash uint64 -} - -func (e *ErrDeadlock) Error() string { - return "deadlock" -} - -// ErrCommitTSExpired is returned when commit.CommitTS < lock.MinCommitTS -type ErrCommitTSExpired struct { - kvrpcpb.CommitTsExpired -} - -func (e *ErrCommitTSExpired) Error() string { - return "commit ts expired" -} - -// ErrTxnNotFound is returned when the primary lock of the txn is not found. -type ErrTxnNotFound struct { - kvrpcpb.TxnNotFound -} - -func (e *ErrTxnNotFound) Error() string { - return "txn not found" -} diff --git a/store/tikv/mockstore/mocktikv/mock.go b/store/tikv/mockstore/mocktikv/mock.go deleted file mode 100644 index 59502ba42e00c..0000000000000 --- a/store/tikv/mockstore/mocktikv/mock.go +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "github.com/pingcap/errors" - pd "github.com/tikv/pd/client" -) - -// NewTiKVAndPDClient creates a TiKV client and PD client from options. -func NewTiKVAndPDClient(path string, coprHandler CoprRPCHandler) (*RPCClient, *Cluster, pd.Client, error) { - mvccStore, err := NewMVCCLevelDB(path) - if err != nil { - return nil, nil, nil, errors.Trace(err) - } - cluster := NewCluster(mvccStore) - - return NewRPCClient(cluster, mvccStore, coprHandler), cluster, NewPDClient(cluster), nil -} diff --git a/store/tikv/mockstore/mocktikv/mock_tikv_test.go b/store/tikv/mockstore/mocktikv/mock_tikv_test.go deleted file mode 100644 index ba8a0313772e0..0000000000000 --- a/store/tikv/mockstore/mocktikv/mock_tikv_test.go +++ /dev/null @@ -1,767 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "math" - "testing" - - . "github.com/pingcap/check" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" -) - -func TestT(t *testing.T) { - TestingT(t) -} - -// testMockTiKVSuite tests MVCCStore interface. -// SetUpTest should set specific MVCCStore implementation. -type testMockTiKVSuite struct { - store MVCCStore -} - -type testMarshal struct{} - -// testMVCCLevelDB is used to test MVCCLevelDB implementation. -type testMVCCLevelDB struct { - testMockTiKVSuite -} - -var ( - _ = Suite(&testMockTiKVSuite{}) - _ = Suite(&testMVCCLevelDB{}) - _ = Suite(testMarshal{}) -) - -func (s *testMockTiKVSuite) SetUpTest(c *C) { - var err error - s.store, err = NewMVCCLevelDB("") - c.Assert(err, IsNil) -} - -func lock(key, primary string, ts uint64) *kvrpcpb.LockInfo { - return &kvrpcpb.LockInfo{ - Key: []byte(key), - PrimaryLock: []byte(primary), - LockVersion: ts, - } -} - -func (s *testMockTiKVSuite) mustGetNone(c *C, key string, ts uint64) { - val, err := s.store.Get([]byte(key), ts, kvrpcpb.IsolationLevel_SI, nil) - c.Assert(err, IsNil) - c.Assert(val, IsNil) -} - -func (s *testMockTiKVSuite) mustGetErr(c *C, key string, ts uint64) { - val, err := s.store.Get([]byte(key), ts, kvrpcpb.IsolationLevel_SI, nil) - c.Assert(err, NotNil) - c.Assert(val, IsNil) -} - -func (s *testMockTiKVSuite) mustGetOK(c *C, key string, ts uint64, expect string) { - val, err := s.store.Get([]byte(key), ts, kvrpcpb.IsolationLevel_SI, nil) - c.Assert(err, IsNil) - c.Assert(string(val), Equals, expect) -} - -func (s *testMockTiKVSuite) mustGetRC(c *C, key string, ts uint64, expect string) { - val, err := s.store.Get([]byte(key), ts, kvrpcpb.IsolationLevel_RC, nil) - c.Assert(err, IsNil) - c.Assert(string(val), Equals, expect) -} - -func (s *testMockTiKVSuite) mustPutOK(c *C, key, value string, startTS, commitTS uint64) { - req := &kvrpcpb.PrewriteRequest{ - Mutations: putMutations(key, value), - PrimaryLock: []byte(key), - StartVersion: startTS, - } - errs := s.store.Prewrite(req) - for _, err := range errs { - c.Assert(err, IsNil) - } - err := s.store.Commit([][]byte{[]byte(key)}, startTS, commitTS) - c.Assert(err, IsNil) -} - -func (s *testMockTiKVSuite) mustDeleteOK(c *C, key string, startTS, commitTS uint64) { - mutations := []*kvrpcpb.Mutation{ - { - Op: kvrpcpb.Op_Del, - Key: []byte(key), - }, - } - req := &kvrpcpb.PrewriteRequest{ - Mutations: mutations, - PrimaryLock: []byte(key), - StartVersion: startTS, - } - errs := s.store.Prewrite(req) - for _, err := range errs { - c.Assert(err, IsNil) - } - err := s.store.Commit([][]byte{[]byte(key)}, startTS, commitTS) - c.Assert(err, IsNil) -} - -func (s *testMockTiKVSuite) mustScanOK(c *C, start string, limit int, ts uint64, expect ...string) { - s.mustRangeScanOK(c, start, "", limit, ts, expect...) -} - -func (s *testMockTiKVSuite) mustRangeScanOK(c *C, start, end string, limit int, ts uint64, expect ...string) { - pairs := s.store.Scan([]byte(start), []byte(end), limit, ts, kvrpcpb.IsolationLevel_SI, nil) - c.Assert(len(pairs)*2, Equals, len(expect)) - for i := 0; i < len(pairs); i++ { - c.Assert(pairs[i].Err, IsNil) - c.Assert(pairs[i].Key, BytesEquals, []byte(expect[i*2])) - c.Assert(string(pairs[i].Value), Equals, expect[i*2+1]) - } -} - -func (s *testMockTiKVSuite) mustReverseScanOK(c *C, end string, limit int, ts uint64, expect ...string) { - s.mustRangeReverseScanOK(c, "", end, limit, ts, expect...) -} - -func (s *testMockTiKVSuite) mustRangeReverseScanOK(c *C, start, end string, limit int, ts uint64, expect ...string) { - pairs := s.store.ReverseScan([]byte(start), []byte(end), limit, ts, kvrpcpb.IsolationLevel_SI, nil) - c.Assert(len(pairs)*2, Equals, len(expect)) - for i := 0; i < len(pairs); i++ { - c.Assert(pairs[i].Err, IsNil) - c.Assert(pairs[i].Key, BytesEquals, []byte(expect[i*2])) - c.Assert(string(pairs[i].Value), Equals, expect[i*2+1]) - } -} - -func (s *testMockTiKVSuite) mustPrewriteOK(c *C, mutations []*kvrpcpb.Mutation, primary string, startTS uint64) { - s.mustPrewriteWithTTLOK(c, mutations, primary, startTS, 0) -} - -func (s *testMockTiKVSuite) mustPrewriteWithTTLOK(c *C, mutations []*kvrpcpb.Mutation, primary string, startTS uint64, ttl uint64) { - c.Assert(mustPrewriteWithTTL(s.store, mutations, primary, startTS, ttl), IsTrue) -} - -func (s *testMockTiKVSuite) mustCommitOK(c *C, keys [][]byte, startTS, commitTS uint64) { - err := s.store.Commit(keys, startTS, commitTS) - c.Assert(err, IsNil) -} - -func (s *testMockTiKVSuite) mustCommitErr(c *C, keys [][]byte, startTS, commitTS uint64) { - err := s.store.Commit(keys, startTS, commitTS) - c.Assert(err, NotNil) -} - -func (s *testMockTiKVSuite) mustRollbackOK(c *C, keys [][]byte, startTS uint64) { - err := s.store.Rollback(keys, startTS) - c.Assert(err, IsNil) -} - -func (s *testMockTiKVSuite) mustRollbackErr(c *C, keys [][]byte, startTS uint64) { - err := s.store.Rollback(keys, startTS) - c.Assert(err, NotNil) -} - -func (s *testMockTiKVSuite) mustScanLock(c *C, maxTs uint64, expect []*kvrpcpb.LockInfo) { - locks, err := s.store.ScanLock(nil, nil, maxTs) - c.Assert(err, IsNil) - c.Assert(locks, DeepEquals, expect) -} - -func (s *testMockTiKVSuite) mustResolveLock(c *C, startTS, commitTS uint64) { - c.Assert(s.store.ResolveLock(nil, nil, startTS, commitTS), IsNil) -} - -func (s *testMockTiKVSuite) mustBatchResolveLock(c *C, txnInfos map[uint64]uint64) { - c.Assert(s.store.BatchResolveLock(nil, nil, txnInfos), IsNil) -} - -func (s *testMockTiKVSuite) mustGC(c *C, safePoint uint64) { - c.Assert(s.store.GC(nil, nil, safePoint), IsNil) -} - -func (s *testMockTiKVSuite) mustDeleteRange(c *C, startKey, endKey string) { - err := s.store.DeleteRange([]byte(startKey), []byte(endKey)) - c.Assert(err, IsNil) -} - -func (s *testMockTiKVSuite) TestGet(c *C) { - s.mustGetNone(c, "x", 10) - s.mustPutOK(c, "x", "x", 5, 10) - s.mustGetNone(c, "x", 9) - s.mustGetOK(c, "x", 10, "x") - s.mustGetOK(c, "x", 11, "x") -} - -func (s *testMockTiKVSuite) TestGetWithLock(c *C) { - key := "key" - value := "value" - s.mustPutOK(c, key, value, 5, 10) - mutations := []*kvrpcpb.Mutation{{ - Op: kvrpcpb.Op_Lock, - Key: []byte(key), - }, - } - // test with lock's type is lock - s.mustPrewriteOK(c, mutations, key, 20) - s.mustGetOK(c, key, 25, value) - s.mustCommitOK(c, [][]byte{[]byte(key)}, 20, 30) - - // test get with lock's max ts and primary key - s.mustPrewriteOK(c, putMutations(key, "value2", "key2", "v5"), key, 40) - s.mustGetErr(c, key, 41) - s.mustGetErr(c, "key2", math.MaxUint64) - s.mustGetOK(c, key, math.MaxUint64, "value") -} - -func (s *testMockTiKVSuite) TestDelete(c *C) { - s.mustPutOK(c, "x", "x5-10", 5, 10) - s.mustDeleteOK(c, "x", 15, 20) - s.mustGetNone(c, "x", 5) - s.mustGetNone(c, "x", 9) - s.mustGetOK(c, "x", 10, "x5-10") - s.mustGetOK(c, "x", 19, "x5-10") - s.mustGetNone(c, "x", 20) - s.mustGetNone(c, "x", 21) -} - -func (s *testMockTiKVSuite) TestCleanupRollback(c *C) { - s.mustPutOK(c, "secondary", "s-0", 1, 2) - s.mustPrewriteOK(c, putMutations("primary", "p-5", "secondary", "s-5"), "primary", 5) - s.mustGetErr(c, "secondary", 8) - s.mustGetErr(c, "secondary", 12) - s.mustCommitOK(c, [][]byte{[]byte("primary")}, 5, 10) - s.mustRollbackErr(c, [][]byte{[]byte("primary")}, 5) -} - -func (s *testMockTiKVSuite) TestReverseScan(c *C) { - // ver10: A(10) - B(_) - C(10) - D(_) - E(10) - s.mustPutOK(c, "A", "A10", 5, 10) - s.mustPutOK(c, "C", "C10", 5, 10) - s.mustPutOK(c, "E", "E10", 5, 10) - - checkV10 := func() { - s.mustReverseScanOK(c, "Z", 0, 10) - s.mustReverseScanOK(c, "Z", 1, 10, "E", "E10") - s.mustReverseScanOK(c, "Z", 2, 10, "E", "E10", "C", "C10") - s.mustReverseScanOK(c, "Z", 3, 10, "E", "E10", "C", "C10", "A", "A10") - s.mustReverseScanOK(c, "Z", 4, 10, "E", "E10", "C", "C10", "A", "A10") - s.mustReverseScanOK(c, "E\x00", 3, 10, "E", "E10", "C", "C10", "A", "A10") - s.mustReverseScanOK(c, "C\x00", 3, 10, "C", "C10", "A", "A10") - s.mustReverseScanOK(c, "C\x00", 4, 10, "C", "C10", "A", "A10") - s.mustReverseScanOK(c, "B", 1, 10, "A", "A10") - s.mustRangeReverseScanOK(c, "", "E", 5, 10, "C", "C10", "A", "A10") - s.mustRangeReverseScanOK(c, "", "C\x00", 5, 10, "C", "C10", "A", "A10") - s.mustRangeReverseScanOK(c, "A\x00", "C", 5, 10) - } - checkV10() - - // ver20: A(10) - B(20) - C(10) - D(20) - E(10) - s.mustPutOK(c, "B", "B20", 15, 20) - s.mustPutOK(c, "D", "D20", 15, 20) - - checkV20 := func() { - s.mustReverseScanOK(c, "Z", 5, 20, "E", "E10", "D", "D20", "C", "C10", "B", "B20", "A", "A10") - s.mustReverseScanOK(c, "C\x00", 5, 20, "C", "C10", "B", "B20", "A", "A10") - s.mustReverseScanOK(c, "A\x00", 1, 20, "A", "A10") - s.mustRangeReverseScanOK(c, "B", "D", 5, 20, "C", "C10", "B", "B20") - s.mustRangeReverseScanOK(c, "B", "D\x00", 5, 20, "D", "D20", "C", "C10", "B", "B20") - s.mustRangeReverseScanOK(c, "B\x00", "D\x00", 5, 20, "D", "D20", "C", "C10") - } - checkV10() - checkV20() - - // ver30: A(_) - B(20) - C(10) - D(_) - E(10) - s.mustDeleteOK(c, "A", 25, 30) - s.mustDeleteOK(c, "D", 25, 30) - - checkV30 := func() { - s.mustReverseScanOK(c, "Z", 5, 30, "E", "E10", "C", "C10", "B", "B20") - s.mustReverseScanOK(c, "C", 1, 30, "B", "B20") - s.mustReverseScanOK(c, "C\x00", 5, 30, "C", "C10", "B", "B20") - } - checkV10() - checkV20() - checkV30() - - // ver40: A(_) - B(_) - C(40) - D(40) - E(10) - s.mustDeleteOK(c, "B", 35, 40) - s.mustPutOK(c, "C", "C40", 35, 40) - s.mustPutOK(c, "D", "D40", 35, 40) - - checkV40 := func() { - s.mustReverseScanOK(c, "Z", 5, 40, "E", "E10", "D", "D40", "C", "C40") - s.mustReverseScanOK(c, "Z", 5, 100, "E", "E10", "D", "D40", "C", "C40") - } - checkV10() - checkV20() - checkV30() - checkV40() -} - -func (s *testMockTiKVSuite) TestScan(c *C) { - // ver10: A(10) - B(_) - C(10) - D(_) - E(10) - s.mustPutOK(c, "A", "A10", 5, 10) - s.mustPutOK(c, "C", "C10", 5, 10) - s.mustPutOK(c, "E", "E10", 5, 10) - - checkV10 := func() { - s.mustScanOK(c, "", 0, 10) - s.mustScanOK(c, "", 1, 10, "A", "A10") - s.mustScanOK(c, "", 2, 10, "A", "A10", "C", "C10") - s.mustScanOK(c, "", 3, 10, "A", "A10", "C", "C10", "E", "E10") - s.mustScanOK(c, "", 4, 10, "A", "A10", "C", "C10", "E", "E10") - s.mustScanOK(c, "A", 3, 10, "A", "A10", "C", "C10", "E", "E10") - s.mustScanOK(c, "A\x00", 3, 10, "C", "C10", "E", "E10") - s.mustScanOK(c, "C", 4, 10, "C", "C10", "E", "E10") - s.mustScanOK(c, "F", 1, 10) - s.mustRangeScanOK(c, "", "E", 5, 10, "A", "A10", "C", "C10") - s.mustRangeScanOK(c, "", "C\x00", 5, 10, "A", "A10", "C", "C10") - s.mustRangeScanOK(c, "A\x00", "C", 5, 10) - } - checkV10() - - // ver20: A(10) - B(20) - C(10) - D(20) - E(10) - s.mustPutOK(c, "B", "B20", 15, 20) - s.mustPutOK(c, "D", "D20", 15, 20) - - checkV20 := func() { - s.mustScanOK(c, "", 5, 20, "A", "A10", "B", "B20", "C", "C10", "D", "D20", "E", "E10") - s.mustScanOK(c, "C", 5, 20, "C", "C10", "D", "D20", "E", "E10") - s.mustScanOK(c, "D\x00", 1, 20, "E", "E10") - s.mustRangeScanOK(c, "B", "D", 5, 20, "B", "B20", "C", "C10") - s.mustRangeScanOK(c, "B", "D\x00", 5, 20, "B", "B20", "C", "C10", "D", "D20") - s.mustRangeScanOK(c, "B\x00", "D\x00", 5, 20, "C", "C10", "D", "D20") - } - checkV10() - checkV20() - - // ver30: A(_) - B(20) - C(10) - D(_) - E(10) - s.mustDeleteOK(c, "A", 25, 30) - s.mustDeleteOK(c, "D", 25, 30) - - checkV30 := func() { - s.mustScanOK(c, "", 5, 30, "B", "B20", "C", "C10", "E", "E10") - s.mustScanOK(c, "A", 1, 30, "B", "B20") - s.mustScanOK(c, "C\x00", 5, 30, "E", "E10") - } - checkV10() - checkV20() - checkV30() - - // ver40: A(_) - B(_) - C(40) - D(40) - E(10) - s.mustDeleteOK(c, "B", 35, 40) - s.mustPutOK(c, "C", "C40", 35, 40) - s.mustPutOK(c, "D", "D40", 35, 40) - - checkV40 := func() { - s.mustScanOK(c, "", 5, 40, "C", "C40", "D", "D40", "E", "E10") - s.mustScanOK(c, "", 5, 100, "C", "C40", "D", "D40", "E", "E10") - } - checkV10() - checkV20() - checkV30() - checkV40() -} - -func (s *testMockTiKVSuite) TestBatchGet(c *C) { - s.mustPutOK(c, "k1", "v1", 1, 2) - s.mustPutOK(c, "k2", "v2", 1, 2) - s.mustPutOK(c, "k2", "v2", 3, 4) - s.mustPutOK(c, "k3", "v3", 1, 2) - batchKeys := [][]byte{[]byte("k1"), []byte("k2"), []byte("k3")} - pairs := s.store.BatchGet(batchKeys, 5, kvrpcpb.IsolationLevel_SI, nil) - for _, pair := range pairs { - c.Assert(pair.Err, IsNil) - } - c.Assert(string(pairs[0].Value), Equals, "v1") - c.Assert(string(pairs[1].Value), Equals, "v2") - c.Assert(string(pairs[2].Value), Equals, "v3") -} - -func (s *testMockTiKVSuite) TestScanLock(c *C) { - s.mustPutOK(c, "k1", "v1", 1, 2) - s.mustPrewriteOK(c, putMutations("p1", "v5", "s1", "v5"), "p1", 5) - s.mustPrewriteOK(c, putMutations("p2", "v10", "s2", "v10"), "p2", 10) - s.mustPrewriteOK(c, putMutations("p3", "v20", "s3", "v20"), "p3", 20) - - locks, err := s.store.ScanLock([]byte("a"), []byte("r"), 12) - c.Assert(err, IsNil) - c.Assert(locks, DeepEquals, []*kvrpcpb.LockInfo{ - lock("p1", "p1", 5), - lock("p2", "p2", 10), - }) - - s.mustScanLock(c, 10, []*kvrpcpb.LockInfo{ - lock("p1", "p1", 5), - lock("p2", "p2", 10), - lock("s1", "p1", 5), - lock("s2", "p2", 10), - }) -} - -func (s *testMockTiKVSuite) TestScanWithResolvedLock(c *C) { - s.mustPrewriteOK(c, putMutations("p1", "v5", "s1", "v5"), "p1", 5) - s.mustPrewriteOK(c, putMutations("p2", "v10", "s2", "v10"), "p1", 5) - - pairs := s.store.Scan([]byte("p1"), nil, 3, 10, kvrpcpb.IsolationLevel_SI, nil) - lock, ok := errors.Cause(pairs[0].Err).(*ErrLocked) - c.Assert(ok, IsTrue) - _, ok = errors.Cause(pairs[1].Err).(*ErrLocked) - c.Assert(ok, IsTrue) - - // Mock the request after resolving lock. - pairs = s.store.Scan([]byte("p1"), nil, 3, 10, kvrpcpb.IsolationLevel_SI, []uint64{lock.StartTS}) - for _, pair := range pairs { - c.Assert(pair.Err, IsNil) - } -} - -func (s *testMockTiKVSuite) TestCommitConflict(c *C) { - // txn A want set x to A - // txn B want set x to B - // A prewrite. - s.mustPrewriteOK(c, putMutations("x", "A"), "x", 5) - // B prewrite and find A's lock. - req := &kvrpcpb.PrewriteRequest{ - Mutations: putMutations("x", "B"), - PrimaryLock: []byte("x"), - StartVersion: 10, - } - errs := s.store.Prewrite(req) - c.Assert(errs[0], NotNil) - // B find rollback A because A exist too long. - s.mustRollbackOK(c, [][]byte{[]byte("x")}, 5) - // if A commit here, it would find its lock removed, report error txn not found. - s.mustCommitErr(c, [][]byte{[]byte("x")}, 5, 10) - // B prewrite itself after it rollback A. - s.mustPrewriteOK(c, putMutations("x", "B"), "x", 10) - // if A commit here, it would find its lock replaced by others and commit fail. - s.mustCommitErr(c, [][]byte{[]byte("x")}, 5, 20) - // B commit success. - s.mustCommitOK(c, [][]byte{[]byte("x")}, 10, 20) - // if B commit again, it will success because the key already committed. - s.mustCommitOK(c, [][]byte{[]byte("x")}, 10, 20) -} - -func (s *testMockTiKVSuite) TestResolveLock(c *C) { - s.mustPrewriteOK(c, putMutations("p1", "v5", "s1", "v5"), "p1", 5) - s.mustPrewriteOK(c, putMutations("p2", "v10", "s2", "v10"), "p2", 10) - s.mustResolveLock(c, 5, 0) - s.mustResolveLock(c, 10, 20) - s.mustGetNone(c, "p1", 20) - s.mustGetNone(c, "s1", 30) - s.mustGetOK(c, "p2", 20, "v10") - s.mustGetOK(c, "s2", 30, "v10") - s.mustScanLock(c, 30, nil) -} - -func (s *testMockTiKVSuite) TestBatchResolveLock(c *C) { - s.mustPrewriteOK(c, putMutations("p1", "v11", "s1", "v11"), "p1", 11) - s.mustPrewriteOK(c, putMutations("p2", "v12", "s2", "v12"), "p2", 12) - s.mustPrewriteOK(c, putMutations("p3", "v13"), "p3", 13) - s.mustPrewriteOK(c, putMutations("p4", "v14", "s3", "v14", "s4", "v14"), "p4", 14) - s.mustPrewriteOK(c, putMutations("p5", "v15", "s5", "v15"), "p5", 15) - txnInfos := map[uint64]uint64{ - 11: 0, - 12: 22, - 13: 0, - 14: 24, - } - s.mustBatchResolveLock(c, txnInfos) - s.mustGetNone(c, "p1", 20) - s.mustGetNone(c, "p3", 30) - s.mustGetOK(c, "p2", 30, "v12") - s.mustGetOK(c, "s4", 30, "v14") - s.mustScanLock(c, 30, []*kvrpcpb.LockInfo{ - lock("p5", "p5", 15), - lock("s5", "p5", 15), - }) - txnInfos = map[uint64]uint64{ - 15: 0, - } - s.mustBatchResolveLock(c, txnInfos) - s.mustScanLock(c, 30, nil) -} - -func (s *testMockTiKVSuite) TestGC(c *C) { - var safePoint uint64 = 100 - - // Prepare data - s.mustPutOK(c, "k1", "v1", 1, 2) - s.mustPutOK(c, "k1", "v2", 11, 12) - - s.mustPutOK(c, "k2", "v1", 1, 2) - s.mustPutOK(c, "k2", "v2", 11, 12) - s.mustPutOK(c, "k2", "v3", 101, 102) - - s.mustPutOK(c, "k3", "v1", 1, 2) - s.mustPutOK(c, "k3", "v2", 11, 12) - s.mustDeleteOK(c, "k3", 101, 102) - - s.mustPutOK(c, "k4", "v1", 1, 2) - s.mustDeleteOK(c, "k4", 11, 12) - - // Check prepared data - s.mustGetOK(c, "k1", 5, "v1") - s.mustGetOK(c, "k1", 15, "v2") - s.mustGetOK(c, "k2", 5, "v1") - s.mustGetOK(c, "k2", 15, "v2") - s.mustGetOK(c, "k2", 105, "v3") - s.mustGetOK(c, "k3", 5, "v1") - s.mustGetOK(c, "k3", 15, "v2") - s.mustGetNone(c, "k3", 105) - s.mustGetOK(c, "k4", 5, "v1") - s.mustGetNone(c, "k4", 105) - - s.mustGC(c, safePoint) - - s.mustGetNone(c, "k1", 5) - s.mustGetOK(c, "k1", 15, "v2") - s.mustGetNone(c, "k2", 5) - s.mustGetOK(c, "k2", 15, "v2") - s.mustGetOK(c, "k2", 105, "v3") - s.mustGetNone(c, "k3", 5) - s.mustGetOK(c, "k3", 15, "v2") - s.mustGetNone(c, "k3", 105) - s.mustGetNone(c, "k4", 5) - s.mustGetNone(c, "k4", 105) -} - -func (s *testMockTiKVSuite) TestRollbackAndWriteConflict(c *C) { - s.mustPutOK(c, "test", "test", 1, 3) - req := &kvrpcpb.PrewriteRequest{ - Mutations: putMutations("lock", "lock", "test", "test1"), - PrimaryLock: []byte("test"), - StartVersion: 2, - LockTtl: 2, - } - errs := s.store.Prewrite(req) - s.mustWriteWriteConflict(c, errs, 1) - - s.mustPutOK(c, "test", "test2", 5, 8) - - // simulate `getTxnStatus` for txn 2. - err := s.store.Cleanup([]byte("test"), 2, math.MaxUint64) - c.Assert(err, IsNil) - req = &kvrpcpb.PrewriteRequest{ - Mutations: putMutations("test", "test3"), - PrimaryLock: []byte("test"), - StartVersion: 6, - LockTtl: 1, - } - errs = s.store.Prewrite(req) - s.mustWriteWriteConflict(c, errs, 0) -} - -func (s *testMockTiKVSuite) TestDeleteRange(c *C) { - for i := 1; i <= 5; i++ { - key := string(byte(i) + byte('0')) - value := "v" + key - s.mustPutOK(c, key, value, uint64(1+2*i), uint64(2+2*i)) - } - - s.mustScanOK(c, "0", 10, 20, "1", "v1", "2", "v2", "3", "v3", "4", "v4", "5", "v5") - - s.mustDeleteRange(c, "2", "4") - s.mustScanOK(c, "0", 10, 30, "1", "v1", "4", "v4", "5", "v5") - - s.mustDeleteRange(c, "5", "5") - s.mustScanOK(c, "0", 10, 40, "1", "v1", "4", "v4", "5", "v5") - - s.mustDeleteRange(c, "41", "42") - s.mustScanOK(c, "0", 10, 50, "1", "v1", "4", "v4", "5", "v5") - - s.mustDeleteRange(c, "4\x00", "5\x00") - s.mustScanOK(c, "0", 10, 60, "1", "v1", "4", "v4") - - s.mustDeleteRange(c, "0", "9") - s.mustScanOK(c, "0", 10, 70) -} - -func (s *testMockTiKVSuite) mustWriteWriteConflict(c *C, errs []error, i int) { - c.Assert(errs[i], NotNil) - _, ok := errs[i].(*ErrConflict) - c.Assert(ok, IsTrue) -} - -func (s *testMockTiKVSuite) TestRC(c *C) { - s.mustPutOK(c, "key", "v1", 5, 10) - s.mustPrewriteOK(c, putMutations("key", "v2"), "key", 15) - s.mustGetErr(c, "key", 20) - s.mustGetRC(c, "key", 12, "v1") - s.mustGetRC(c, "key", 20, "v1") -} - -func (s testMarshal) TestMarshalmvccLock(c *C) { - l := mvccLock{ - startTS: 47, - primary: []byte{'a', 'b', 'c'}, - value: []byte{'d', 'e'}, - op: kvrpcpb.Op_Put, - ttl: 444, - minCommitTS: 666, - } - bin, err := l.MarshalBinary() - c.Assert(err, IsNil) - - var l1 mvccLock - err = l1.UnmarshalBinary(bin) - c.Assert(err, IsNil) - - c.Assert(l.startTS, Equals, l1.startTS) - c.Assert(l.op, Equals, l1.op) - c.Assert(l.ttl, Equals, l1.ttl) - c.Assert(string(l.primary), Equals, string(l1.primary)) - c.Assert(string(l.value), Equals, string(l1.value)) - c.Assert(l.minCommitTS, Equals, l1.minCommitTS) -} - -func (s testMarshal) TestMarshalmvccValue(c *C) { - v := mvccValue{ - valueType: typePut, - startTS: 42, - commitTS: 55, - value: []byte{'d', 'e'}, - } - bin, err := v.MarshalBinary() - c.Assert(err, IsNil) - - var v1 mvccValue - err = v1.UnmarshalBinary(bin) - c.Assert(err, IsNil) - - c.Assert(v.valueType, Equals, v1.valueType) - c.Assert(v.startTS, Equals, v1.startTS) - c.Assert(v.commitTS, Equals, v1.commitTS) - c.Assert(string(v.value), Equals, string(v.value)) -} - -func (s *testMVCCLevelDB) TestErrors(c *C) { - c.Assert((&ErrKeyAlreadyExist{}).Error(), Equals, `key already exist, key: ""`) - c.Assert(ErrAbort("txn").Error(), Equals, "abort: txn") - c.Assert(ErrAlreadyCommitted(0).Error(), Equals, "txn already committed") - c.Assert((&ErrConflict{}).Error(), Equals, "write conflict") -} - -func (s *testMVCCLevelDB) TestCheckTxnStatus(c *C) { - startTS := uint64(5 << 18) - s.mustPrewriteWithTTLOK(c, putMutations("pk", "val"), "pk", startTS, 666) - - ttl, commitTS, action, err := s.store.CheckTxnStatus([]byte("pk"), startTS, startTS+100, 666, false, false) - c.Assert(err, IsNil) - c.Assert(ttl, Equals, uint64(666)) - c.Assert(commitTS, Equals, uint64(0)) - c.Assert(action, Equals, kvrpcpb.Action_MinCommitTSPushed) - - // MaxUint64 as callerStartTS shouldn't update minCommitTS but return Action_MinCommitTSPushed. - ttl, commitTS, action, err = s.store.CheckTxnStatus([]byte("pk"), startTS, math.MaxUint64, 666, false, false) - c.Assert(err, IsNil) - c.Assert(ttl, Equals, uint64(666)) - c.Assert(commitTS, Equals, uint64(0)) - c.Assert(action, Equals, kvrpcpb.Action_MinCommitTSPushed) - s.mustCommitOK(c, [][]byte{[]byte("pk")}, startTS, startTS+101) - - ttl, commitTS, _, err = s.store.CheckTxnStatus([]byte("pk"), startTS, 0, 666, false, false) - c.Assert(err, IsNil) - c.Assert(ttl, Equals, uint64(0)) - c.Assert(commitTS, Equals, startTS+101) - - s.mustPrewriteWithTTLOK(c, putMutations("pk1", "val"), "pk1", startTS, 666) - s.mustRollbackOK(c, [][]byte{[]byte("pk1")}, startTS) - - ttl, commitTS, action, err = s.store.CheckTxnStatus([]byte("pk1"), startTS, 0, 666, false, false) - c.Assert(err, IsNil) - c.Assert(ttl, Equals, uint64(0)) - c.Assert(commitTS, Equals, uint64(0)) - c.Assert(action, Equals, kvrpcpb.Action_NoAction) - - s.mustPrewriteWithTTLOK(c, putMutations("pk2", "val"), "pk2", startTS, 666) - currentTS := uint64(777 << 18) - ttl, commitTS, action, err = s.store.CheckTxnStatus([]byte("pk2"), startTS, 0, currentTS, false, false) - c.Assert(err, IsNil) - c.Assert(ttl, Equals, uint64(0)) - c.Assert(commitTS, Equals, uint64(0)) - c.Assert(action, Equals, kvrpcpb.Action_TTLExpireRollback) - - // Cover the TxnNotFound case. - _, _, _, err = s.store.CheckTxnStatus([]byte("txnNotFound"), 5, 0, 666, false, false) - c.Assert(err, NotNil) - notFound, ok := errors.Cause(err).(*ErrTxnNotFound) - c.Assert(ok, IsTrue) - c.Assert(notFound.StartTs, Equals, uint64(5)) - c.Assert(string(notFound.PrimaryKey), Equals, "txnNotFound") - - ttl, commitTS, action, err = s.store.CheckTxnStatus([]byte("txnNotFound"), 5, 0, 666, true, false) - c.Assert(err, IsNil) - c.Assert(ttl, Equals, uint64(0)) - c.Assert(commitTS, Equals, uint64(0)) - c.Assert(action, Equals, kvrpcpb.Action_LockNotExistRollback) - - // Check the rollback tombstone blocks this prewrite which comes with a smaller startTS. - req := &kvrpcpb.PrewriteRequest{ - Mutations: putMutations("txnNotFound", "val"), - PrimaryLock: []byte("txnNotFound"), - StartVersion: 4, - MinCommitTs: 6, - } - errs := s.store.Prewrite(req) - c.Assert(errs, NotNil) -} - -func (s *testMVCCLevelDB) TestRejectCommitTS(c *C) { - s.mustPrewriteOK(c, putMutations("x", "A"), "x", 5) - // Push the minCommitTS - _, _, _, err := s.store.CheckTxnStatus([]byte("x"), 5, 100, 100, false, false) - c.Assert(err, IsNil) - err = s.store.Commit([][]byte{[]byte("x")}, 5, 10) - e, ok := errors.Cause(err).(*ErrCommitTSExpired) - c.Assert(ok, IsTrue) - c.Assert(e.MinCommitTs, Equals, uint64(101)) -} - -func (s *testMVCCLevelDB) TestMvccGetByKey(c *C) { - s.mustPrewriteOK(c, putMutations("q1", "v5"), "p1", 5) - debugger, ok := s.store.(MVCCDebugger) - c.Assert(ok, IsTrue) - mvccInfo := debugger.MvccGetByKey([]byte("q1")) - except := &kvrpcpb.MvccInfo{ - Lock: &kvrpcpb.MvccLock{ - Type: kvrpcpb.Op_Put, - StartTs: 5, - Primary: []byte("p1"), - ShortValue: []byte("v5"), - }, - } - c.Assert(mvccInfo, DeepEquals, except) -} - -func (s *testMVCCLevelDB) TestTxnHeartBeat(c *C) { - s.mustPrewriteWithTTLOK(c, putMutations("pk", "val"), "pk", 5, 666) - - // Update the ttl - ttl, err := s.store.TxnHeartBeat([]byte("pk"), 5, 888) - c.Assert(err, IsNil) - c.Assert(ttl, Greater, uint64(666)) - - // Advise ttl is small - ttl, err = s.store.TxnHeartBeat([]byte("pk"), 5, 300) - c.Assert(err, IsNil) - c.Assert(ttl, Greater, uint64(300)) - - // The lock has already been clean up - c.Assert(s.store.Cleanup([]byte("pk"), 5, math.MaxUint64), IsNil) - _, err = s.store.TxnHeartBeat([]byte("pk"), 5, 1000) - c.Assert(err, NotNil) -} diff --git a/store/tikv/mockstore/mocktikv/mvcc.go b/store/tikv/mockstore/mocktikv/mvcc.go deleted file mode 100644 index 8d0883dd043a5..0000000000000 --- a/store/tikv/mockstore/mocktikv/mvcc.go +++ /dev/null @@ -1,320 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "bytes" - "encoding/binary" - "io" - "math" - - "github.com/google/btree" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/util/codec" -) - -type mvccValueType int - -const ( - typePut mvccValueType = iota - typeDelete - typeRollback - typeLock -) - -type mvccValue struct { - valueType mvccValueType - startTS uint64 - commitTS uint64 - value []byte -} - -type mvccLock struct { - startTS uint64 - primary []byte - value []byte - op kvrpcpb.Op - ttl uint64 - forUpdateTS uint64 - txnSize uint64 - minCommitTS uint64 -} - -type mvccEntry struct { - key MvccKey - values []mvccValue - lock *mvccLock -} - -// MarshalBinary implements encoding.BinaryMarshaler interface. -func (l *mvccLock) MarshalBinary() ([]byte, error) { - var ( - mh marshalHelper - buf bytes.Buffer - ) - mh.WriteNumber(&buf, l.startTS) - mh.WriteSlice(&buf, l.primary) - mh.WriteSlice(&buf, l.value) - mh.WriteNumber(&buf, l.op) - mh.WriteNumber(&buf, l.ttl) - mh.WriteNumber(&buf, l.forUpdateTS) - mh.WriteNumber(&buf, l.txnSize) - mh.WriteNumber(&buf, l.minCommitTS) - return buf.Bytes(), errors.Trace(mh.err) -} - -// UnmarshalBinary implements encoding.BinaryUnmarshaler interface. -func (l *mvccLock) UnmarshalBinary(data []byte) error { - var mh marshalHelper - buf := bytes.NewBuffer(data) - mh.ReadNumber(buf, &l.startTS) - mh.ReadSlice(buf, &l.primary) - mh.ReadSlice(buf, &l.value) - mh.ReadNumber(buf, &l.op) - mh.ReadNumber(buf, &l.ttl) - mh.ReadNumber(buf, &l.forUpdateTS) - mh.ReadNumber(buf, &l.txnSize) - mh.ReadNumber(buf, &l.minCommitTS) - return errors.Trace(mh.err) -} - -// MarshalBinary implements encoding.BinaryMarshaler interface. -func (v mvccValue) MarshalBinary() ([]byte, error) { - var ( - mh marshalHelper - buf bytes.Buffer - ) - mh.WriteNumber(&buf, int64(v.valueType)) - mh.WriteNumber(&buf, v.startTS) - mh.WriteNumber(&buf, v.commitTS) - mh.WriteSlice(&buf, v.value) - return buf.Bytes(), errors.Trace(mh.err) -} - -// UnmarshalBinary implements encoding.BinaryUnmarshaler interface. -func (v *mvccValue) UnmarshalBinary(data []byte) error { - var mh marshalHelper - buf := bytes.NewBuffer(data) - var vt int64 - mh.ReadNumber(buf, &vt) - v.valueType = mvccValueType(vt) - mh.ReadNumber(buf, &v.startTS) - mh.ReadNumber(buf, &v.commitTS) - mh.ReadSlice(buf, &v.value) - return errors.Trace(mh.err) -} - -type marshalHelper struct { - err error -} - -func (mh *marshalHelper) WriteSlice(buf io.Writer, slice []byte) { - if mh.err != nil { - return - } - var tmp [binary.MaxVarintLen64]byte - off := binary.PutUvarint(tmp[:], uint64(len(slice))) - if err := writeFull(buf, tmp[:off]); err != nil { - mh.err = errors.Trace(err) - return - } - if err := writeFull(buf, slice); err != nil { - mh.err = errors.Trace(err) - } -} - -func (mh *marshalHelper) WriteNumber(buf io.Writer, n interface{}) { - if mh.err != nil { - return - } - err := binary.Write(buf, binary.LittleEndian, n) - if err != nil { - mh.err = errors.Trace(err) - } -} - -func writeFull(w io.Writer, slice []byte) error { - written := 0 - for written < len(slice) { - n, err := w.Write(slice[written:]) - if err != nil { - return errors.Trace(err) - } - written += n - } - return nil -} - -func (mh *marshalHelper) ReadNumber(r io.Reader, n interface{}) { - if mh.err != nil { - return - } - err := binary.Read(r, binary.LittleEndian, n) - if err != nil { - mh.err = errors.Trace(err) - } -} - -func (mh *marshalHelper) ReadSlice(r *bytes.Buffer, slice *[]byte) { - if mh.err != nil { - return - } - sz, err := binary.ReadUvarint(r) - if err != nil { - mh.err = errors.Trace(err) - return - } - const c10M = 10 * 1024 * 1024 - if sz > c10M { - mh.err = errors.New("too large slice, maybe something wrong") - return - } - data := make([]byte, sz) - if _, err := io.ReadFull(r, data); err != nil { - mh.err = errors.Trace(err) - return - } - *slice = data -} - -// lockErr returns ErrLocked. -// Note that parameter key is raw key, while key in ErrLocked is mvcc key. -func (l *mvccLock) lockErr(key []byte) error { - return &ErrLocked{ - Key: mvccEncode(key, lockVer), - Primary: l.primary, - StartTS: l.startTS, - ForUpdateTS: l.forUpdateTS, - TTL: l.ttl, - TxnSize: l.txnSize, - LockType: l.op, - } -} - -func (l *mvccLock) check(ts uint64, key []byte, resolvedLocks []uint64) (uint64, error) { - // ignore when ts is older than lock or lock's type is Lock. - // Pessimistic lock doesn't block read. - if l.startTS > ts || l.op == kvrpcpb.Op_Lock || l.op == kvrpcpb.Op_PessimisticLock { - return ts, nil - } - // for point get latest version. - if ts == math.MaxUint64 && bytes.Equal(l.primary, key) { - return l.startTS - 1, nil - } - // Skip lock if the lock is resolved. - for _, resolved := range resolvedLocks { - if l.startTS == resolved { - return ts, nil - } - } - return 0, l.lockErr(key) -} - -func (e *mvccEntry) Less(than btree.Item) bool { - return bytes.Compare(e.key, than.(*mvccEntry).key) < 0 -} - -func (e *mvccEntry) Get(ts uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) ([]byte, error) { - if isoLevel == kvrpcpb.IsolationLevel_SI && e.lock != nil { - var err error - ts, err = e.lock.check(ts, e.key.Raw(), resolvedLocks) - if err != nil { - return nil, err - } - } - for _, v := range e.values { - if v.commitTS <= ts && v.valueType != typeRollback && v.valueType != typeLock { - return v.value, nil - } - } - return nil, nil -} - -// MVCCStore is a mvcc key-value storage. -type MVCCStore interface { - Get(key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) ([]byte, error) - Scan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) []Pair - ReverseScan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) []Pair - BatchGet(ks [][]byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) []Pair - PessimisticLock(req *kvrpcpb.PessimisticLockRequest) *kvrpcpb.PessimisticLockResponse - PessimisticRollback(keys [][]byte, startTS, forUpdateTS uint64) []error - Prewrite(req *kvrpcpb.PrewriteRequest) []error - Commit(keys [][]byte, startTS, commitTS uint64) error - Rollback(keys [][]byte, startTS uint64) error - Cleanup(key []byte, startTS, currentTS uint64) error - ScanLock(startKey, endKey []byte, maxTS uint64) ([]*kvrpcpb.LockInfo, error) - TxnHeartBeat(primaryKey []byte, startTS uint64, adviseTTL uint64) (uint64, error) - ResolveLock(startKey, endKey []byte, startTS, commitTS uint64) error - BatchResolveLock(startKey, endKey []byte, txnInfos map[uint64]uint64) error - GC(startKey, endKey []byte, safePoint uint64) error - DeleteRange(startKey, endKey []byte) error - CheckTxnStatus(primaryKey []byte, lockTS uint64, startTS, currentTS uint64, rollbackIfNotFound bool, resolvingPessimisticLock bool) (uint64, uint64, kvrpcpb.Action, error) - Close() error -} - -// RawKV is a key-value storage. MVCCStore can be implemented upon it with timestamp encoded into key. -type RawKV interface { - RawGet(key []byte) []byte - RawBatchGet(keys [][]byte) [][]byte - RawScan(startKey, endKey []byte, limit int) []Pair // Scan the range of [startKey, endKey) - RawReverseScan(startKey, endKey []byte, limit int) []Pair // Scan the range of [endKey, startKey) - RawPut(key, value []byte) - RawBatchPut(keys, values [][]byte) - RawDelete(key []byte) - RawBatchDelete(keys [][]byte) - RawDeleteRange(startKey, endKey []byte) -} - -// MVCCDebugger is for debugging. -type MVCCDebugger interface { - MvccGetByStartTS(starTS uint64) (*kvrpcpb.MvccInfo, []byte) - MvccGetByKey(key []byte) *kvrpcpb.MvccInfo -} - -// Pair is a KV pair read from MvccStore or an error if any occurs. -type Pair struct { - Key []byte - Value []byte - Err error -} - -func regionContains(startKey []byte, endKey []byte, key []byte) bool { - return bytes.Compare(startKey, key) <= 0 && - (bytes.Compare(key, endKey) < 0 || len(endKey) == 0) -} - -// MvccKey is the encoded key type. -// On TiKV, keys are encoded before they are saved into storage engine. -type MvccKey []byte - -// NewMvccKey encodes a key into MvccKey. -func NewMvccKey(key []byte) MvccKey { - if len(key) == 0 { - return nil - } - return codec.EncodeBytes(nil, key) -} - -// Raw decodes a MvccKey to original key. -func (key MvccKey) Raw() []byte { - if len(key) == 0 { - return nil - } - _, k, err := codec.DecodeBytes(key, nil) - if err != nil { - panic(err) - } - return k -} diff --git a/store/tikv/mockstore/mocktikv/mvcc_leveldb.go b/store/tikv/mockstore/mocktikv/mvcc_leveldb.go deleted file mode 100644 index ab9b7dc5987a7..0000000000000 --- a/store/tikv/mockstore/mocktikv/mvcc_leveldb.go +++ /dev/null @@ -1,1764 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "bytes" - "math" - "sync" - - "github.com/dgryski/go-farm" - "github.com/pingcap/errors" - "github.com/pingcap/goleveldb/leveldb" - "github.com/pingcap/goleveldb/leveldb/iterator" - "github.com/pingcap/goleveldb/leveldb/opt" - "github.com/pingcap/goleveldb/leveldb/storage" - "github.com/pingcap/goleveldb/leveldb/util" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/mockstore/deadlock" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/util/codec" - "go.uber.org/zap" -) - -// Used for pessimistic lock wait time -// these two constants are special for lock protocol with tikv -// 0 means always wait, -1 means nowait, others meaning lock wait in milliseconds -var ( - LockAlwaysWait = int64(0) - LockNoWait = int64(-1) -) - -// MVCCLevelDB implements the MVCCStore interface. -type MVCCLevelDB struct { - // Key layout: - // ... - // Key_lock -- (0) - // Key_verMax -- (1) - // ... - // Key_ver+1 -- (2) - // Key_ver -- (3) - // Key_ver-1 -- (4) - // ... - // Key_0 -- (5) - // NextKey_lock -- (6) - // NextKey_verMax -- (7) - // ... - // NextKey_ver+1 -- (8) - // NextKey_ver -- (9) - // NextKey_ver-1 -- (10) - // ... - // NextKey_0 -- (11) - // ... - // EOF - - // db represents leveldb - db *leveldb.DB - // mu used for lock - // leveldb can not guarantee multiple operations to be atomic, for example, read - // then write, another write may happen during it, so this lock is necessory. - mu sync.RWMutex - deadlockDetector *deadlock.Detector -} - -const lockVer uint64 = math.MaxUint64 - -// ErrInvalidEncodedKey describes parsing an invalid format of EncodedKey. -var ErrInvalidEncodedKey = errors.New("invalid encoded key") - -// mvccEncode returns the encoded key. -func mvccEncode(key []byte, ver uint64) []byte { - b := codec.EncodeBytes(nil, key) - ret := codec.EncodeUintDesc(b, ver) - return ret -} - -// mvccDecode parses the origin key and version of an encoded key, if the encoded key is a meta key, -// just returns the origin key. -func mvccDecode(encodedKey []byte) ([]byte, uint64, error) { - // Skip DataPrefix - remainBytes, key, err := codec.DecodeBytes(encodedKey, nil) - if err != nil { - // should never happen - return nil, 0, errors.Trace(err) - } - // if it's meta key - if len(remainBytes) == 0 { - return key, 0, nil - } - var ver uint64 - remainBytes, ver, err = codec.DecodeUintDesc(remainBytes) - if err != nil { - // should never happen - return nil, 0, errors.Trace(err) - } - if len(remainBytes) != 0 { - return nil, 0, ErrInvalidEncodedKey - } - return key, ver, nil -} - -// MustNewMVCCStore is used for testing, use NewMVCCLevelDB instead. -func MustNewMVCCStore() MVCCStore { - mvccStore, err := NewMVCCLevelDB("") - if err != nil { - panic(err) - } - return mvccStore -} - -// NewMVCCLevelDB returns a new MVCCLevelDB object. -func NewMVCCLevelDB(path string) (*MVCCLevelDB, error) { - var ( - d *leveldb.DB - err error - ) - if path == "" { - d, err = leveldb.Open(storage.NewMemStorage(), nil) - } else { - d, err = leveldb.OpenFile(path, &opt.Options{BlockCacheCapacity: 600 * 1024 * 1024}) - } - - return &MVCCLevelDB{db: d, deadlockDetector: deadlock.NewDetector()}, errors.Trace(err) -} - -// Iterator wraps iterator.Iterator to provide Valid() method. -type Iterator struct { - iterator.Iterator - valid bool -} - -// Next moves the iterator to the next key/value pair. -func (iter *Iterator) Next() { - iter.valid = iter.Iterator.Next() -} - -// Valid returns whether the iterator is exhausted. -func (iter *Iterator) Valid() bool { - return iter.valid -} - -func newIterator(db *leveldb.DB, slice *util.Range) *Iterator { - iter := &Iterator{db.NewIterator(slice, nil), true} - iter.Next() - return iter -} - -func newScanIterator(db *leveldb.DB, startKey, endKey []byte) (*Iterator, []byte, error) { - var start, end []byte - if len(startKey) > 0 { - start = mvccEncode(startKey, lockVer) - } - if len(endKey) > 0 { - end = mvccEncode(endKey, lockVer) - } - iter := newIterator(db, &util.Range{ - Start: start, - Limit: end, - }) - // newScanIterator must handle startKey is nil, in this case, the real startKey - // should be change the frist key of the store. - if len(startKey) == 0 && iter.Valid() { - key, _, err := mvccDecode(iter.Key()) - if err != nil { - return nil, nil, errors.Trace(err) - } - startKey = key - } - return iter, startKey, nil -} - -type lockDecoder struct { - lock mvccLock - expectKey []byte -} - -// Decode decodes the lock value if current iterator is at expectKey::lock. -func (dec *lockDecoder) Decode(iter *Iterator) (bool, error) { - if iter.Error() != nil || !iter.Valid() { - return false, iter.Error() - } - - iterKey := iter.Key() - key, ver, err := mvccDecode(iterKey) - if err != nil { - return false, errors.Trace(err) - } - if !bytes.Equal(key, dec.expectKey) { - return false, nil - } - if ver != lockVer { - return false, nil - } - - var lock mvccLock - err = lock.UnmarshalBinary(iter.Value()) - if err != nil { - return false, errors.Trace(err) - } - dec.lock = lock - iter.Next() - return true, nil -} - -type valueDecoder struct { - value mvccValue - expectKey []byte -} - -// Decode decodes a mvcc value if iter key is expectKey. -func (dec *valueDecoder) Decode(iter *Iterator) (bool, error) { - if iter.Error() != nil || !iter.Valid() { - return false, iter.Error() - } - - key, ver, err := mvccDecode(iter.Key()) - if err != nil { - return false, errors.Trace(err) - } - if !bytes.Equal(key, dec.expectKey) { - return false, nil - } - if ver == lockVer { - return false, nil - } - - var value mvccValue - err = value.UnmarshalBinary(iter.Value()) - if err != nil { - return false, errors.Trace(err) - } - dec.value = value - iter.Next() - return true, nil -} - -type skipDecoder struct { - currKey []byte -} - -// Decode skips the iterator as long as its key is currKey, the new key would be stored. -func (dec *skipDecoder) Decode(iter *Iterator) (bool, error) { - if iter.Error() != nil { - return false, iter.Error() - } - for iter.Valid() { - key, _, err := mvccDecode(iter.Key()) - if err != nil { - return false, errors.Trace(err) - } - if !bytes.Equal(key, dec.currKey) { - dec.currKey = key - return true, nil - } - iter.Next() - } - return false, nil -} - -// Get implements the MVCCStore interface. -// key cannot be nil or []byte{} -func (mvcc *MVCCLevelDB) Get(key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) ([]byte, error) { - mvcc.mu.RLock() - defer mvcc.mu.RUnlock() - - return mvcc.getValue(key, startTS, isoLevel, resolvedLocks) -} - -func (mvcc *MVCCLevelDB) getValue(key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) ([]byte, error) { - startKey := mvccEncode(key, lockVer) - iter := newIterator(mvcc.db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - return getValue(iter, key, startTS, isoLevel, resolvedLocks) -} - -func getValue(iter *Iterator, key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) ([]byte, error) { - dec1 := lockDecoder{expectKey: key} - ok, err := dec1.Decode(iter) - if ok && isoLevel == kvrpcpb.IsolationLevel_SI { - startTS, err = dec1.lock.check(startTS, key, resolvedLocks) - } - if err != nil { - return nil, errors.Trace(err) - } - dec2 := valueDecoder{expectKey: key} - for iter.Valid() { - ok, err := dec2.Decode(iter) - if err != nil { - return nil, errors.Trace(err) - } - if !ok { - break - } - - value := &dec2.value - if value.valueType == typeRollback || value.valueType == typeLock { - continue - } - // Read the first committed value that can be seen at startTS. - if value.commitTS <= startTS { - if value.valueType == typeDelete { - return nil, nil - } - return value.value, nil - } - } - return nil, nil -} - -// BatchGet implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) BatchGet(ks [][]byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) []Pair { - mvcc.mu.RLock() - defer mvcc.mu.RUnlock() - - pairs := make([]Pair, 0, len(ks)) - for _, k := range ks { - v, err := mvcc.getValue(k, startTS, isoLevel, resolvedLocks) - if v == nil && err == nil { - continue - } - pairs = append(pairs, Pair{ - Key: k, - Value: v, - Err: errors.Trace(err), - }) - } - return pairs -} - -// Scan implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) Scan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLock []uint64) []Pair { - mvcc.mu.RLock() - defer mvcc.mu.RUnlock() - - iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey) - defer iter.Release() - if err != nil { - logutil.BgLogger().Error("scan new iterator fail", zap.Error(err)) - return nil - } - - ok := true - var pairs []Pair - for len(pairs) < limit && ok { - value, err := getValue(iter, currKey, startTS, isoLevel, resolvedLock) - if err != nil { - pairs = append(pairs, Pair{ - Key: currKey, - Err: errors.Trace(err), - }) - } - if value != nil { - pairs = append(pairs, Pair{ - Key: currKey, - Value: value, - }) - } - - skip := skipDecoder{currKey} - ok, err = skip.Decode(iter) - if err != nil { - logutil.BgLogger().Error("seek to next key error", zap.Error(err)) - break - } - currKey = skip.currKey - } - return pairs -} - -// ReverseScan implements the MVCCStore interface. The search range is [startKey, endKey). -func (mvcc *MVCCLevelDB) ReverseScan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel, resolvedLocks []uint64) []Pair { - mvcc.mu.RLock() - defer mvcc.mu.RUnlock() - - var mvccEnd []byte - if len(endKey) != 0 { - mvccEnd = mvccEncode(endKey, lockVer) - } - iter := mvcc.db.NewIterator(&util.Range{ - Limit: mvccEnd, - }, nil) - defer iter.Release() - - succ := iter.Last() - currKey, _, err := mvccDecode(iter.Key()) - // TODO: return error. - terror.Log(errors.Trace(err)) - helper := reverseScanHelper{ - startTS: startTS, - isoLevel: isoLevel, - currKey: currKey, - resolvedLocks: resolvedLocks, - } - - for succ && len(helper.pairs) < limit { - key, ver, err := mvccDecode(iter.Key()) - if err != nil { - break - } - if bytes.Compare(key, startKey) < 0 { - break - } - - if !bytes.Equal(key, helper.currKey) { - helper.finishEntry() - helper.currKey = key - } - if ver == lockVer { - var lock mvccLock - err = lock.UnmarshalBinary(iter.Value()) - helper.entry.lock = &lock - } else { - var value mvccValue - err = value.UnmarshalBinary(iter.Value()) - helper.entry.values = append(helper.entry.values, value) - } - if err != nil { - logutil.BgLogger().Error("unmarshal fail", zap.Error(err)) - break - } - succ = iter.Prev() - } - if len(helper.pairs) < limit { - helper.finishEntry() - } - return helper.pairs -} - -type reverseScanHelper struct { - startTS uint64 - isoLevel kvrpcpb.IsolationLevel - resolvedLocks []uint64 - currKey []byte - entry mvccEntry - pairs []Pair -} - -func (helper *reverseScanHelper) finishEntry() { - reverse(helper.entry.values) - helper.entry.key = NewMvccKey(helper.currKey) - val, err := helper.entry.Get(helper.startTS, helper.isoLevel, helper.resolvedLocks) - if len(val) != 0 || err != nil { - helper.pairs = append(helper.pairs, Pair{ - Key: helper.currKey, - Value: val, - Err: err, - }) - } - helper.entry = mvccEntry{} -} - -func reverse(values []mvccValue) { - i, j := 0, len(values)-1 - for i < j { - values[i], values[j] = values[j], values[i] - i++ - j-- - } -} - -type lockCtx struct { - startTS uint64 - forUpdateTS uint64 - primary []byte - ttl uint64 - minCommitTs uint64 - - returnValues bool - values [][]byte -} - -// PessimisticLock writes the pessimistic lock. -func (mvcc *MVCCLevelDB) PessimisticLock(req *kvrpcpb.PessimisticLockRequest) *kvrpcpb.PessimisticLockResponse { - resp := &kvrpcpb.PessimisticLockResponse{} - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - mutations := req.Mutations - lCtx := &lockCtx{ - startTS: req.StartVersion, - forUpdateTS: req.ForUpdateTs, - primary: req.PrimaryLock, - ttl: req.LockTtl, - minCommitTs: req.MinCommitTs, - returnValues: req.ReturnValues, - } - lockWaitTime := req.WaitTimeout - - anyError := false - batch := &leveldb.Batch{} - errs := make([]error, 0, len(mutations)) - for _, m := range mutations { - err := mvcc.pessimisticLockMutation(batch, m, lCtx) - errs = append(errs, err) - if err != nil { - anyError = true - } - if lockWaitTime == LockNoWait { - if _, ok := err.(*ErrLocked); ok { - break - } - } - } - if anyError { - if lockWaitTime != LockNoWait { - // TODO: remove this when implement sever side wait. - simulateServerSideWaitLock(errs) - } - resp.Errors = convertToKeyErrors(errs) - return resp - } - if err := mvcc.db.Write(batch, nil); err != nil { - resp.Errors = convertToKeyErrors([]error{err}) - return resp - } - if req.ReturnValues { - resp.Values = lCtx.values - } - return resp -} - -func (mvcc *MVCCLevelDB) pessimisticLockMutation(batch *leveldb.Batch, mutation *kvrpcpb.Mutation, lctx *lockCtx) error { - startTS := lctx.startTS - forUpdateTS := lctx.forUpdateTS - startKey := mvccEncode(mutation.Key, lockVer) - iter := newIterator(mvcc.db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - dec := lockDecoder{ - expectKey: mutation.Key, - } - ok, err := dec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - if ok { - if dec.lock.startTS != startTS { - errDeadlock := mvcc.deadlockDetector.Detect(startTS, dec.lock.startTS, farm.Fingerprint64(mutation.Key)) - if errDeadlock != nil { - return &ErrDeadlock{ - LockKey: mutation.Key, - LockTS: dec.lock.startTS, - DealockKeyHash: errDeadlock.KeyHash, - } - } - return dec.lock.lockErr(mutation.Key) - } - return nil - } - - // For pessimisticLockMutation, check the correspond rollback record, there may be rollbackLock - // operation between startTS and forUpdateTS - val, err := checkConflictValue(iter, mutation, forUpdateTS, startTS, true) - if err != nil { - return err - } - if lctx.returnValues { - lctx.values = append(lctx.values, val) - } - - lock := mvccLock{ - startTS: startTS, - primary: lctx.primary, - op: kvrpcpb.Op_PessimisticLock, - ttl: lctx.ttl, - forUpdateTS: forUpdateTS, - minCommitTS: lctx.minCommitTs, - } - writeKey := mvccEncode(mutation.Key, lockVer) - writeValue, err := lock.MarshalBinary() - if err != nil { - return errors.Trace(err) - } - - batch.Put(writeKey, writeValue) - return nil -} - -// PessimisticRollback implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) PessimisticRollback(keys [][]byte, startTS, forUpdateTS uint64) []error { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - anyError := false - batch := &leveldb.Batch{} - errs := make([]error, 0, len(keys)) - for _, key := range keys { - err := pessimisticRollbackKey(mvcc.db, batch, key, startTS, forUpdateTS) - errs = append(errs, err) - if err != nil { - anyError = true - } - } - if anyError { - return errs - } - if err := mvcc.db.Write(batch, nil); err != nil { - return []error{err} - } - return errs -} - -func pessimisticRollbackKey(db *leveldb.DB, batch *leveldb.Batch, key []byte, startTS, forUpdateTS uint64) error { - startKey := mvccEncode(key, lockVer) - iter := newIterator(db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - dec := lockDecoder{ - expectKey: key, - } - ok, err := dec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - if ok { - lock := dec.lock - if lock.op == kvrpcpb.Op_PessimisticLock && lock.startTS == startTS && lock.forUpdateTS <= forUpdateTS { - batch.Delete(startKey) - } - } - return nil -} - -// Prewrite implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) Prewrite(req *kvrpcpb.PrewriteRequest) []error { - mutations := req.Mutations - primary := req.PrimaryLock - startTS := req.StartVersion - forUpdateTS := req.GetForUpdateTs() - ttl := req.LockTtl - minCommitTS := req.MinCommitTs - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - anyError := false - batch := &leveldb.Batch{} - errs := make([]error, 0, len(mutations)) - txnSize := req.TxnSize - for i, m := range mutations { - // If the operation is Insert, check if key is exists at first. - var err error - // no need to check insert values for pessimistic transaction. - op := m.GetOp() - if (op == kvrpcpb.Op_Insert || op == kvrpcpb.Op_CheckNotExists) && forUpdateTS == 0 { - v, err := mvcc.getValue(m.Key, startTS, kvrpcpb.IsolationLevel_SI, req.Context.ResolvedLocks) - if err != nil { - errs = append(errs, err) - anyError = true - continue - } - if v != nil { - err = &ErrKeyAlreadyExist{ - Key: m.Key, - } - errs = append(errs, err) - anyError = true - continue - } - } - if op == kvrpcpb.Op_CheckNotExists { - continue - } - isPessimisticLock := len(req.IsPessimisticLock) > 0 && req.IsPessimisticLock[i] - err = prewriteMutation(mvcc.db, batch, m, startTS, primary, ttl, txnSize, isPessimisticLock, minCommitTS) - errs = append(errs, err) - if err != nil { - anyError = true - } - } - if anyError { - return errs - } - if err := mvcc.db.Write(batch, nil); err != nil { - return []error{err} - } - - return errs -} - -func checkConflictValue(iter *Iterator, m *kvrpcpb.Mutation, forUpdateTS uint64, startTS uint64, getVal bool) ([]byte, error) { - dec := &valueDecoder{ - expectKey: m.Key, - } - ok, err := dec.Decode(iter) - if err != nil { - return nil, errors.Trace(err) - } - if !ok { - return nil, nil - } - - // Note that it's a write conflict here, even if the value is a rollback one, or a op_lock record - if dec.value.commitTS > forUpdateTS { - return nil, &ErrConflict{ - StartTS: forUpdateTS, - ConflictTS: dec.value.startTS, - ConflictCommitTS: dec.value.commitTS, - Key: m.Key, - } - } - - needGetVal := getVal - needCheckAssertion := m.Assertion == kvrpcpb.Assertion_NotExist - needCheckRollback := true - var retVal []byte - // do the check or get operations within one iteration to make CI faster - for ok { - if needCheckRollback { - if dec.value.valueType == typeRollback { - if dec.value.commitTS == startTS { - logutil.BgLogger().Warn("rollback value found", - zap.Uint64("txnID", startTS), - zap.Int32("rollbacked.valueType", int32(dec.value.valueType)), - zap.Uint64("rollbacked.startTS", dec.value.startTS), - zap.Uint64("rollbacked.commitTS", dec.value.commitTS)) - return nil, &ErrAlreadyRollbacked{ - startTS: startTS, - key: m.Key, - } - } - } - if dec.value.commitTS < startTS { - needCheckRollback = false - } - } - if needCheckAssertion { - if dec.value.valueType == typePut || dec.value.valueType == typeLock { - if m.Op == kvrpcpb.Op_PessimisticLock { - return nil, &ErrKeyAlreadyExist{ - Key: m.Key, - } - } - } else if dec.value.valueType == typeDelete { - needCheckAssertion = false - } - } - if needGetVal { - if dec.value.valueType == typeDelete || dec.value.valueType == typePut { - retVal = dec.value.value - needGetVal = false - } - } - if !needCheckAssertion && !needGetVal && !needCheckRollback { - break - } - ok, err = dec.Decode(iter) - if err != nil { - return nil, errors.Trace(err) - } - } - if getVal { - return retVal, nil - } - return nil, nil -} - -func prewriteMutation(db *leveldb.DB, batch *leveldb.Batch, - mutation *kvrpcpb.Mutation, startTS uint64, - primary []byte, ttl uint64, txnSize uint64, - isPessimisticLock bool, minCommitTS uint64) error { - startKey := mvccEncode(mutation.Key, lockVer) - iter := newIterator(db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - dec := lockDecoder{ - expectKey: mutation.Key, - } - ok, err := dec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - if ok { - if dec.lock.startTS != startTS { - if isPessimisticLock { - // NOTE: A special handling. - // When pessimistic txn prewrite meets lock, set the TTL = 0 means - // telling TiDB to rollback the transaction **unconditionly**. - dec.lock.ttl = 0 - } - return dec.lock.lockErr(mutation.Key) - } - if dec.lock.op != kvrpcpb.Op_PessimisticLock { - return nil - } - // Overwrite the pessimistic lock. - if ttl < dec.lock.ttl { - // Maybe ttlManager has already set the lock TTL, don't decrease it. - ttl = dec.lock.ttl - } - if minCommitTS < dec.lock.minCommitTS { - // The minCommitTS has been pushed forward. - minCommitTS = dec.lock.minCommitTS - } - } else { - if isPessimisticLock { - return ErrAbort("pessimistic lock not found") - } - _, err = checkConflictValue(iter, mutation, startTS, startTS, false) - if err != nil { - return err - } - } - - op := mutation.GetOp() - if op == kvrpcpb.Op_Insert { - op = kvrpcpb.Op_Put - } - lock := mvccLock{ - startTS: startTS, - primary: primary, - value: mutation.Value, - op: op, - ttl: ttl, - txnSize: txnSize, - } - // Write minCommitTS on the primary lock. - if bytes.Equal(primary, mutation.GetKey()) { - lock.minCommitTS = minCommitTS - } - - writeKey := mvccEncode(mutation.Key, lockVer) - writeValue, err := lock.MarshalBinary() - if err != nil { - return errors.Trace(err) - } - - batch.Put(writeKey, writeValue) - return nil -} - -// Commit implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) Commit(keys [][]byte, startTS, commitTS uint64) error { - mvcc.mu.Lock() - defer func() { - mvcc.mu.Unlock() - mvcc.deadlockDetector.CleanUp(startTS) - }() - - batch := &leveldb.Batch{} - for _, k := range keys { - err := commitKey(mvcc.db, batch, k, startTS, commitTS) - if err != nil { - return errors.Trace(err) - } - } - return mvcc.db.Write(batch, nil) -} - -func commitKey(db *leveldb.DB, batch *leveldb.Batch, key []byte, startTS, commitTS uint64) error { - startKey := mvccEncode(key, lockVer) - iter := newIterator(db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - dec := lockDecoder{ - expectKey: key, - } - ok, err := dec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - if !ok || dec.lock.startTS != startTS { - // If the lock of this transaction is not found, or the lock is replaced by - // another transaction, check commit information of this transaction. - c, ok, err1 := getTxnCommitInfo(iter, key, startTS) - if err1 != nil { - return errors.Trace(err1) - } - if ok && c.valueType != typeRollback { - // c.valueType != typeRollback means the transaction is already committed, do nothing. - return nil - } - return ErrRetryable("txn not found") - } - // Reject the commit request whose commitTS is less than minCommiTS. - if dec.lock.minCommitTS > commitTS { - return &ErrCommitTSExpired{ - kvrpcpb.CommitTsExpired{ - StartTs: startTS, - AttemptedCommitTs: commitTS, - Key: key, - MinCommitTs: dec.lock.minCommitTS, - }} - } - - if err = commitLock(batch, dec.lock, key, startTS, commitTS); err != nil { - return errors.Trace(err) - } - return nil -} - -func commitLock(batch *leveldb.Batch, lock mvccLock, key []byte, startTS, commitTS uint64) error { - var valueType mvccValueType - if lock.op == kvrpcpb.Op_Put { - valueType = typePut - } else if lock.op == kvrpcpb.Op_Lock { - valueType = typeLock - } else { - valueType = typeDelete - } - value := mvccValue{ - valueType: valueType, - startTS: startTS, - commitTS: commitTS, - value: lock.value, - } - writeKey := mvccEncode(key, commitTS) - writeValue, err := value.MarshalBinary() - if err != nil { - return errors.Trace(err) - } - batch.Put(writeKey, writeValue) - batch.Delete(mvccEncode(key, lockVer)) - return nil -} - -// Rollback implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) Rollback(keys [][]byte, startTS uint64) error { - mvcc.mu.Lock() - defer func() { - mvcc.mu.Unlock() - mvcc.deadlockDetector.CleanUp(startTS) - }() - - batch := &leveldb.Batch{} - for _, k := range keys { - err := rollbackKey(mvcc.db, batch, k, startTS) - if err != nil { - return errors.Trace(err) - } - } - return mvcc.db.Write(batch, nil) -} - -func rollbackKey(db *leveldb.DB, batch *leveldb.Batch, key []byte, startTS uint64) error { - startKey := mvccEncode(key, lockVer) - iter := newIterator(db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - if iter.Valid() { - dec := lockDecoder{ - expectKey: key, - } - ok, err := dec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - // If current transaction's lock exist. - if ok && dec.lock.startTS == startTS { - if err = rollbackLock(batch, key, startTS); err != nil { - return errors.Trace(err) - } - return nil - } - - // If current transaction's lock not exist. - // If commit info of current transaction exist. - c, ok, err := getTxnCommitInfo(iter, key, startTS) - if err != nil { - return errors.Trace(err) - } - if ok { - // If current transaction is already committed. - if c.valueType != typeRollback { - return ErrAlreadyCommitted(c.commitTS) - } - // If current transaction is already rollback. - return nil - } - } - - // If current transaction is not prewritted before. - value := mvccValue{ - valueType: typeRollback, - startTS: startTS, - commitTS: startTS, - } - writeKey := mvccEncode(key, startTS) - writeValue, err := value.MarshalBinary() - if err != nil { - return errors.Trace(err) - } - batch.Put(writeKey, writeValue) - return nil -} - -func writeRollback(batch *leveldb.Batch, key []byte, startTS uint64) error { - tomb := mvccValue{ - valueType: typeRollback, - startTS: startTS, - commitTS: startTS, - } - writeKey := mvccEncode(key, startTS) - writeValue, err := tomb.MarshalBinary() - if err != nil { - return errors.Trace(err) - } - batch.Put(writeKey, writeValue) - return nil -} - -func rollbackLock(batch *leveldb.Batch, key []byte, startTS uint64) error { - err := writeRollback(batch, key, startTS) - if err != nil { - return err - } - batch.Delete(mvccEncode(key, lockVer)) - return nil -} - -func getTxnCommitInfo(iter *Iterator, expectKey []byte, startTS uint64) (mvccValue, bool, error) { - for iter.Valid() { - dec := valueDecoder{ - expectKey: expectKey, - } - ok, err := dec.Decode(iter) - if err != nil || !ok { - return mvccValue{}, ok, errors.Trace(err) - } - - if dec.value.startTS == startTS { - return dec.value, true, nil - } - } - return mvccValue{}, false, nil -} - -// Cleanup implements the MVCCStore interface. -// Cleanup API is deprecated, use CheckTxnStatus instead. -func (mvcc *MVCCLevelDB) Cleanup(key []byte, startTS, currentTS uint64) error { - mvcc.mu.Lock() - defer func() { - mvcc.mu.Unlock() - mvcc.deadlockDetector.CleanUp(startTS) - }() - - batch := &leveldb.Batch{} - startKey := mvccEncode(key, lockVer) - iter := newIterator(mvcc.db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - if iter.Valid() { - dec := lockDecoder{ - expectKey: key, - } - ok, err := dec.Decode(iter) - if err != nil { - return err - } - // If current transaction's lock exists. - if ok && dec.lock.startTS == startTS { - // If the lock has already outdated, clean up it. - if currentTS == 0 || uint64(oracle.ExtractPhysical(dec.lock.startTS))+dec.lock.ttl < uint64(oracle.ExtractPhysical(currentTS)) { - if err = rollbackLock(batch, key, startTS); err != nil { - return err - } - return mvcc.db.Write(batch, nil) - } - - // Otherwise, return a locked error with the TTL information. - return dec.lock.lockErr(key) - } - - // If current transaction's lock does not exist. - // If the commit information of the current transaction exist. - c, ok, err := getTxnCommitInfo(iter, key, startTS) - if err != nil { - return errors.Trace(err) - } - if ok { - // If the current transaction has already committed. - if c.valueType != typeRollback { - return ErrAlreadyCommitted(c.commitTS) - } - // If the current transaction has already rollbacked. - return nil - } - } - - // If current transaction is not prewritted before. - value := mvccValue{ - valueType: typeRollback, - startTS: startTS, - commitTS: startTS, - } - writeKey := mvccEncode(key, startTS) - writeValue, err := value.MarshalBinary() - if err != nil { - return errors.Trace(err) - } - batch.Put(writeKey, writeValue) - return nil -} - -// CheckTxnStatus checks the primary lock of a transaction to decide its status. -// The return values are (ttl, commitTS, err): -// If the transaction is active, this function returns the ttl of the lock; -// If the transaction is committed, this function returns the commitTS; -// If the transaction is rollbacked, this function returns (0, 0, nil) -// Note that CheckTxnStatus may also push forward the `minCommitTS` of the -// transaction, so it's not simply a read-only operation. -// -// primaryKey + lockTS together could locate the primary lock. -// callerStartTS is the start ts of reader transaction. -// currentTS is the current ts, but it may be inaccurate. Just use it to check TTL. -func (mvcc *MVCCLevelDB) CheckTxnStatus(primaryKey []byte, lockTS, callerStartTS, currentTS uint64, - rollbackIfNotExist bool, resolvingPessimisticLock bool) (ttl uint64, commitTS uint64, action kvrpcpb.Action, err error) { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - action = kvrpcpb.Action_NoAction - - startKey := mvccEncode(primaryKey, lockVer) - iter := newIterator(mvcc.db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - if iter.Valid() { - dec := lockDecoder{ - expectKey: primaryKey, - } - var ok bool - ok, err = dec.Decode(iter) - if err != nil { - err = errors.Trace(err) - return - } - // If current transaction's lock exists. - if ok && dec.lock.startTS == lockTS { - lock := dec.lock - batch := &leveldb.Batch{} - - // If the lock has already outdated, clean up it. - if uint64(oracle.ExtractPhysical(lock.startTS))+lock.ttl < uint64(oracle.ExtractPhysical(currentTS)) { - if resolvingPessimisticLock && lock.op == kvrpcpb.Op_PessimisticLock { - action = kvrpcpb.Action_TTLExpirePessimisticRollback - if err = pessimisticRollbackKey(mvcc.db, batch, primaryKey, lock.startTS, lock.forUpdateTS); err != nil { - err = errors.Trace(err) - return - } - } else { - action = kvrpcpb.Action_TTLExpireRollback - if err = rollbackLock(batch, primaryKey, lockTS); err != nil { - err = errors.Trace(err) - return - } - } - if err = mvcc.db.Write(batch, nil); err != nil { - err = errors.Trace(err) - return - } - return 0, 0, action, nil - } - - // If the caller_start_ts is MaxUint64, it's a point get in the autocommit transaction. - // Even though the MinCommitTs is not pushed, the point get can ingore the lock - // next time because it's not committed. So we pretend it has been pushed. - if callerStartTS == math.MaxUint64 { - action = kvrpcpb.Action_MinCommitTSPushed - - // If this is a large transaction and the lock is active, push forward the minCommitTS. - // lock.minCommitTS == 0 may be a secondary lock, or not a large transaction (old version TiDB). - } else if lock.minCommitTS > 0 { - action = kvrpcpb.Action_MinCommitTSPushed - // We *must* guarantee the invariance lock.minCommitTS >= callerStartTS + 1 - if lock.minCommitTS < callerStartTS+1 { - lock.minCommitTS = callerStartTS + 1 - - // Remove this condition should not affect correctness. - // We do it because pushing forward minCommitTS as far as possible could avoid - // the lock been pushed again several times, and thus reduce write operations. - if lock.minCommitTS < currentTS { - lock.minCommitTS = currentTS - } - - writeKey := mvccEncode(primaryKey, lockVer) - writeValue, err1 := lock.MarshalBinary() - if err1 != nil { - err = errors.Trace(err1) - return - } - batch.Put(writeKey, writeValue) - if err1 = mvcc.db.Write(batch, nil); err1 != nil { - err = errors.Trace(err1) - return - } - } - } - - return lock.ttl, 0, action, nil - } - - // If current transaction's lock does not exist. - // If the commit info of the current transaction exists. - c, ok, err1 := getTxnCommitInfo(iter, primaryKey, lockTS) - if err1 != nil { - err = errors.Trace(err1) - return - } - if ok { - // If current transaction is already committed. - if c.valueType != typeRollback { - return 0, c.commitTS, action, nil - } - // If current transaction is already rollback. - return 0, 0, kvrpcpb.Action_NoAction, nil - } - } - - // If current transaction is not prewritted before, it may be pessimistic lock. - // When pessimistic txn rollback statement, it may not leave a 'rollbacked' tombstone. - - // Or maybe caused by concurrent prewrite operation. - // Especially in the non-block reading case, the secondary lock is likely to be - // written before the primary lock. - - if rollbackIfNotExist { - if resolvingPessimisticLock { - return 0, 0, kvrpcpb.Action_LockNotExistDoNothing, nil - } - // Write rollback record, but not delete the lock on the primary key. There may exist lock which has - // different lock.startTS with input lockTS, for example the primary key could be already - // locked by the caller transaction, deleting this key will mistakenly delete the lock on - // primary key, see case TestSingleStatementRollback in session_test suite for example - batch := &leveldb.Batch{} - if err1 := writeRollback(batch, primaryKey, lockTS); err1 != nil { - err = errors.Trace(err1) - return - } - if err1 := mvcc.db.Write(batch, nil); err1 != nil { - err = errors.Trace(err1) - return - } - return 0, 0, kvrpcpb.Action_LockNotExistRollback, nil - } - - return 0, 0, action, &ErrTxnNotFound{kvrpcpb.TxnNotFound{ - StartTs: lockTS, - PrimaryKey: primaryKey, - }} -} - -// TxnHeartBeat implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) TxnHeartBeat(key []byte, startTS uint64, adviseTTL uint64) (uint64, error) { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - startKey := mvccEncode(key, lockVer) - iter := newIterator(mvcc.db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - if iter.Valid() { - dec := lockDecoder{ - expectKey: key, - } - ok, err := dec.Decode(iter) - if err != nil { - return 0, errors.Trace(err) - } - if ok && dec.lock.startTS == startTS { - if !bytes.Equal(dec.lock.primary, key) { - return 0, errors.New("txnHeartBeat on non-primary key, the code should not run here") - } - - lock := dec.lock - batch := &leveldb.Batch{} - // Increase the ttl of this transaction. - if adviseTTL > lock.ttl { - lock.ttl = adviseTTL - writeKey := mvccEncode(key, lockVer) - writeValue, err := lock.MarshalBinary() - if err != nil { - return 0, errors.Trace(err) - } - batch.Put(writeKey, writeValue) - if err = mvcc.db.Write(batch, nil); err != nil { - return 0, errors.Trace(err) - } - } - return lock.ttl, nil - } - } - return 0, errors.New("lock doesn't exist") -} - -// ScanLock implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) ScanLock(startKey, endKey []byte, maxTS uint64) ([]*kvrpcpb.LockInfo, error) { - mvcc.mu.RLock() - defer mvcc.mu.RUnlock() - - iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey) - defer iter.Release() - if err != nil { - return nil, errors.Trace(err) - } - - var locks []*kvrpcpb.LockInfo - for iter.Valid() { - dec := lockDecoder{expectKey: currKey} - ok, err := dec.Decode(iter) - if err != nil { - return nil, errors.Trace(err) - } - if ok && dec.lock.startTS <= maxTS { - locks = append(locks, &kvrpcpb.LockInfo{ - PrimaryLock: dec.lock.primary, - LockVersion: dec.lock.startTS, - Key: currKey, - }) - } - - skip := skipDecoder{currKey: currKey} - _, err = skip.Decode(iter) - if err != nil { - return nil, errors.Trace(err) - } - currKey = skip.currKey - } - return locks, nil -} - -// ResolveLock implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) ResolveLock(startKey, endKey []byte, startTS, commitTS uint64) error { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey) - defer iter.Release() - if err != nil { - return errors.Trace(err) - } - - batch := &leveldb.Batch{} - for iter.Valid() { - dec := lockDecoder{expectKey: currKey} - ok, err := dec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - if ok && dec.lock.startTS == startTS { - if commitTS > 0 { - err = commitLock(batch, dec.lock, currKey, startTS, commitTS) - } else { - err = rollbackLock(batch, currKey, startTS) - } - if err != nil { - return errors.Trace(err) - } - } - - skip := skipDecoder{currKey: currKey} - _, err = skip.Decode(iter) - if err != nil { - return errors.Trace(err) - } - currKey = skip.currKey - } - return mvcc.db.Write(batch, nil) -} - -// BatchResolveLock implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) BatchResolveLock(startKey, endKey []byte, txnInfos map[uint64]uint64) error { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey) - defer iter.Release() - if err != nil { - return errors.Trace(err) - } - - batch := &leveldb.Batch{} - for iter.Valid() { - dec := lockDecoder{expectKey: currKey} - ok, err := dec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - if ok { - if commitTS, ok := txnInfos[dec.lock.startTS]; ok { - if commitTS > 0 { - err = commitLock(batch, dec.lock, currKey, dec.lock.startTS, commitTS) - } else { - err = rollbackLock(batch, currKey, dec.lock.startTS) - } - if err != nil { - return errors.Trace(err) - } - } - } - - skip := skipDecoder{currKey: currKey} - _, err = skip.Decode(iter) - if err != nil { - return errors.Trace(err) - } - currKey = skip.currKey - } - return mvcc.db.Write(batch, nil) -} - -// GC implements the MVCCStore interface -func (mvcc *MVCCLevelDB) GC(startKey, endKey []byte, safePoint uint64) error { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - iter, currKey, err := newScanIterator(mvcc.db, startKey, endKey) - defer iter.Release() - if err != nil { - return errors.Trace(err) - } - - // Mock TiKV usually doesn't need to process large amount of data. So write it in a single batch. - batch := &leveldb.Batch{} - - for iter.Valid() { - lockDec := lockDecoder{expectKey: currKey} - ok, err := lockDec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - if ok && lockDec.lock.startTS <= safePoint { - return errors.Errorf( - "key %+q has lock with startTs %v which is under safePoint %v", - currKey, - lockDec.lock.startTS, - safePoint) - } - - keepNext := true - dec := valueDecoder{expectKey: currKey} - - for iter.Valid() { - ok, err := dec.Decode(iter) - if err != nil { - return errors.Trace(err) - } - - if !ok { - // Go to the next key - currKey, _, err = mvccDecode(iter.Key()) - if err != nil { - return errors.Trace(err) - } - break - } - - if dec.value.commitTS > safePoint { - continue - } - - if dec.value.valueType == typePut || dec.value.valueType == typeDelete { - // Keep the latest version if it's `typePut` - if !keepNext || dec.value.valueType == typeDelete { - batch.Delete(mvccEncode(currKey, dec.value.commitTS)) - } - keepNext = false - } else { - // Delete all other types - batch.Delete(mvccEncode(currKey, dec.value.commitTS)) - } - } - } - - return mvcc.db.Write(batch, nil) -} - -// DeleteRange implements the MVCCStore interface. -func (mvcc *MVCCLevelDB) DeleteRange(startKey, endKey []byte) error { - return mvcc.doRawDeleteRange(codec.EncodeBytes(nil, startKey), codec.EncodeBytes(nil, endKey)) -} - -// Close calls leveldb's Close to free resources. -func (mvcc *MVCCLevelDB) Close() error { - return mvcc.db.Close() -} - -// RawPut implements the RawKV interface. -func (mvcc *MVCCLevelDB) RawPut(key, value []byte) { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - if value == nil { - value = []byte{} - } - terror.Log(mvcc.db.Put(key, value, nil)) -} - -// RawBatchPut implements the RawKV interface -func (mvcc *MVCCLevelDB) RawBatchPut(keys, values [][]byte) { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - batch := &leveldb.Batch{} - for i, key := range keys { - value := values[i] - if value == nil { - value = []byte{} - } - batch.Put(key, value) - } - terror.Log(mvcc.db.Write(batch, nil)) -} - -// RawGet implements the RawKV interface. -func (mvcc *MVCCLevelDB) RawGet(key []byte) []byte { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - ret, err := mvcc.db.Get(key, nil) - terror.Log(err) - return ret -} - -// RawBatchGet implements the RawKV interface. -func (mvcc *MVCCLevelDB) RawBatchGet(keys [][]byte) [][]byte { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - values := make([][]byte, 0, len(keys)) - for _, key := range keys { - value, err := mvcc.db.Get(key, nil) - terror.Log(err) - values = append(values, value) - } - return values -} - -// RawDelete implements the RawKV interface. -func (mvcc *MVCCLevelDB) RawDelete(key []byte) { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - terror.Log(mvcc.db.Delete(key, nil)) -} - -// RawBatchDelete implements the RawKV interface. -func (mvcc *MVCCLevelDB) RawBatchDelete(keys [][]byte) { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - batch := &leveldb.Batch{} - for _, key := range keys { - batch.Delete(key) - } - terror.Log(mvcc.db.Write(batch, nil)) -} - -// RawScan implements the RawKV interface. -func (mvcc *MVCCLevelDB) RawScan(startKey, endKey []byte, limit int) []Pair { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - iter := mvcc.db.NewIterator(&util.Range{ - Start: startKey, - }, nil) - - var pairs []Pair - for iter.Next() && len(pairs) < limit { - key := iter.Key() - value := iter.Value() - err := iter.Error() - if len(endKey) > 0 && bytes.Compare(key, endKey) >= 0 { - break - } - pairs = append(pairs, Pair{ - Key: append([]byte{}, key...), - Value: append([]byte{}, value...), - Err: err, - }) - } - return pairs -} - -// RawReverseScan implements the RawKV interface. -// Scan the range of [endKey, startKey) -// It doesn't support Scanning from "", because locating the last Region is not yet implemented. -func (mvcc *MVCCLevelDB) RawReverseScan(startKey, endKey []byte, limit int) []Pair { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - iter := mvcc.db.NewIterator(&util.Range{ - Limit: startKey, - }, nil) - - success := iter.Last() - - var pairs []Pair - for success && len(pairs) < limit { - key := iter.Key() - value := iter.Value() - err := iter.Error() - if bytes.Compare(key, endKey) < 0 { - break - } - pairs = append(pairs, Pair{ - Key: append([]byte{}, key...), - Value: append([]byte{}, value...), - Err: err, - }) - success = iter.Prev() - } - return pairs -} - -// RawDeleteRange implements the RawKV interface. -func (mvcc *MVCCLevelDB) RawDeleteRange(startKey, endKey []byte) { - terror.Log(mvcc.doRawDeleteRange(startKey, endKey)) -} - -// doRawDeleteRange deletes all keys in a range and return the error if any. -func (mvcc *MVCCLevelDB) doRawDeleteRange(startKey, endKey []byte) error { - mvcc.mu.Lock() - defer mvcc.mu.Unlock() - - batch := &leveldb.Batch{} - - iter := mvcc.db.NewIterator(&util.Range{ - Start: startKey, - Limit: endKey, - }, nil) - for iter.Next() { - batch.Delete(iter.Key()) - } - - return mvcc.db.Write(batch, nil) -} - -// MvccGetByStartTS implements the MVCCDebugger interface. -func (mvcc *MVCCLevelDB) MvccGetByStartTS(starTS uint64) (*kvrpcpb.MvccInfo, []byte) { - mvcc.mu.RLock() - defer mvcc.mu.RUnlock() - - var key []byte - iter := newIterator(mvcc.db, nil) - defer iter.Release() - - // find the first committed key for which `start_ts` equals to `ts` - for iter.Valid() { - var value mvccValue - err := value.UnmarshalBinary(iter.Value()) - if err == nil && value.startTS == starTS { - if _, key, err = codec.DecodeBytes(iter.Key(), nil); err != nil { - return nil, nil - } - break - } - iter.Next() - } - - return mvcc.mvccGetByKeyNoLock(key), key -} - -var valueTypeOpMap = [...]kvrpcpb.Op{ - typePut: kvrpcpb.Op_Put, - typeDelete: kvrpcpb.Op_Del, - typeRollback: kvrpcpb.Op_Rollback, - typeLock: kvrpcpb.Op_Lock, -} - -// MvccGetByKey implements the MVCCDebugger interface. -func (mvcc *MVCCLevelDB) MvccGetByKey(key []byte) *kvrpcpb.MvccInfo { - mvcc.mu.RLock() - defer mvcc.mu.RUnlock() - - return mvcc.mvccGetByKeyNoLock(key) -} - -// mvcc.mu.RLock must be held before calling mvccGetByKeyNoLock. -func (mvcc *MVCCLevelDB) mvccGetByKeyNoLock(key []byte) *kvrpcpb.MvccInfo { - info := &kvrpcpb.MvccInfo{} - - startKey := mvccEncode(key, lockVer) - iter := newIterator(mvcc.db, &util.Range{ - Start: startKey, - }) - defer iter.Release() - - dec1 := lockDecoder{expectKey: key} - ok, err := dec1.Decode(iter) - if err != nil { - return nil - } - if ok { - var shortValue []byte - if isShortValue(dec1.lock.value) { - shortValue = dec1.lock.value - } - info.Lock = &kvrpcpb.MvccLock{ - Type: dec1.lock.op, - StartTs: dec1.lock.startTS, - Primary: dec1.lock.primary, - ShortValue: shortValue, - } - } - - dec2 := valueDecoder{expectKey: key} - var writes []*kvrpcpb.MvccWrite - var values []*kvrpcpb.MvccValue - for iter.Valid() { - ok, err := dec2.Decode(iter) - if err != nil { - return nil - } - if !ok { - iter.Next() - break - } - var shortValue []byte - if isShortValue(dec2.value.value) { - shortValue = dec2.value.value - } - write := &kvrpcpb.MvccWrite{ - Type: valueTypeOpMap[dec2.value.valueType], - StartTs: dec2.value.startTS, - CommitTs: dec2.value.commitTS, - ShortValue: shortValue, - } - writes = append(writes, write) - value := &kvrpcpb.MvccValue{ - StartTs: dec2.value.startTS, - Value: dec2.value.value, - } - values = append(values, value) - } - info.Writes = writes - info.Values = values - - return info -} - -const shortValueMaxLen = 64 - -func isShortValue(value []byte) bool { - return len(value) <= shortValueMaxLen -} diff --git a/store/tikv/mockstore/mocktikv/mvcc_test.go b/store/tikv/mockstore/mocktikv/mvcc_test.go deleted file mode 100644 index 0890e2efafff4..0000000000000 --- a/store/tikv/mockstore/mocktikv/mvcc_test.go +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2018-present, 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import . "github.com/pingcap/check" - -var _ = Suite(testMvccSuite{}) - -type testMvccSuite struct { -} - -func (s testMvccSuite) TestRegionContains(c *C) { - c.Check(regionContains([]byte{}, []byte{}, []byte{}), IsTrue) - c.Check(regionContains([]byte{}, []byte{}, []byte{1}), IsTrue) - c.Check(regionContains([]byte{1, 1, 1}, []byte{}, []byte{1, 1, 0}), IsFalse) - c.Check(regionContains([]byte{1, 1, 1}, []byte{}, []byte{1, 1, 1}), IsTrue) - c.Check(regionContains([]byte{}, []byte{2, 2, 2}, []byte{2, 2, 1}), IsTrue) - c.Check(regionContains([]byte{}, []byte{2, 2, 2}, []byte{2, 2, 2}), IsFalse) - c.Check(regionContains([]byte{1, 1, 1}, []byte{2, 2, 2}, []byte{1, 1, 0}), IsFalse) - c.Check(regionContains([]byte{1, 1, 1}, []byte{2, 2, 2}, []byte{1, 1, 1}), IsTrue) - c.Check(regionContains([]byte{1, 1, 1}, []byte{2, 2, 2}, []byte{2, 2, 1}), IsTrue) - c.Check(regionContains([]byte{1, 1, 1}, []byte{2, 2, 2}, []byte{2, 2, 2}), IsFalse) -} diff --git a/store/tikv/mockstore/mocktikv/pd.go b/store/tikv/mockstore/mocktikv/pd.go deleted file mode 100644 index 6531a4ca71cc3..0000000000000 --- a/store/tikv/mockstore/mocktikv/pd.go +++ /dev/null @@ -1,206 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "context" - "fmt" - "math" - "sync" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" - pd "github.com/tikv/pd/client" -) - -// Use global variables to prevent pdClients from creating duplicate timestamps. -var tsMu = struct { - sync.Mutex - physicalTS int64 - logicalTS int64 -}{} - -type pdClient struct { - cluster *Cluster - // SafePoint set by `UpdateGCSafePoint`. Not to be confused with SafePointKV. - gcSafePoint uint64 - // Represents the current safePoint of all services including TiDB, representing how much data they want to retain - // in GC. - serviceSafePoints map[string]uint64 - gcSafePointMu sync.Mutex -} - -// NewPDClient creates a mock pd.Client that uses local timestamp and meta data -// from a Cluster. -func NewPDClient(cluster *Cluster) pd.Client { - return &pdClient{ - cluster: cluster, - serviceSafePoints: make(map[string]uint64), - } -} - -func (c *pdClient) GetClusterID(ctx context.Context) uint64 { - return 1 -} - -func (c *pdClient) GetTS(context.Context) (int64, int64, error) { - tsMu.Lock() - defer tsMu.Unlock() - - ts := time.Now().UnixNano() / int64(time.Millisecond) - if tsMu.physicalTS >= ts { - tsMu.logicalTS++ - } else { - tsMu.physicalTS = ts - tsMu.logicalTS = 0 - } - return tsMu.physicalTS, tsMu.logicalTS, nil -} - -func (c *pdClient) GetLocalTS(ctx context.Context, dcLocation string) (int64, int64, error) { - return c.GetTS(ctx) -} - -func (c *pdClient) GetTSAsync(ctx context.Context) pd.TSFuture { - return &mockTSFuture{c, ctx, false} -} - -func (c *pdClient) GetLocalTSAsync(ctx context.Context, dcLocation string) pd.TSFuture { - return c.GetTSAsync(ctx) -} - -type mockTSFuture struct { - pdc *pdClient - ctx context.Context - used bool -} - -func (m *mockTSFuture) Wait() (int64, int64, error) { - if m.used { - return 0, 0, errors.New("cannot wait tso twice") - } - m.used = true - return m.pdc.GetTS(m.ctx) -} - -func (c *pdClient) GetRegion(ctx context.Context, key []byte) (*pd.Region, error) { - region, peer := c.cluster.GetRegionByKey(key) - return &pd.Region{Meta: region, Leader: peer}, nil -} - -func (c *pdClient) GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string) (*pd.Region, error) { - return &pd.Region{}, nil -} - -func (c *pdClient) GetPrevRegion(ctx context.Context, key []byte) (*pd.Region, error) { - region, peer := c.cluster.GetPrevRegionByKey(key) - return &pd.Region{Meta: region, Leader: peer}, nil -} - -func (c *pdClient) GetRegionByID(ctx context.Context, regionID uint64) (*pd.Region, error) { - region, peer := c.cluster.GetRegionByID(regionID) - return &pd.Region{Meta: region, Leader: peer}, nil -} - -func (c *pdClient) ScanRegions(ctx context.Context, startKey []byte, endKey []byte, limit int) ([]*pd.Region, error) { - regions := c.cluster.ScanRegions(startKey, endKey, limit) - return regions, nil -} - -func (c *pdClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) { - select { - case <-ctx.Done(): - return nil, ctx.Err() - default: - } - store := c.cluster.GetStore(storeID) - // It's same as PD's implementation. - if store == nil { - return nil, fmt.Errorf("invalid store ID %d, not found", storeID) - } - if store.GetState() == metapb.StoreState_Tombstone { - return nil, nil - } - return store, nil -} - -func (c *pdClient) GetAllStores(ctx context.Context, opts ...pd.GetStoreOption) ([]*metapb.Store, error) { - return c.cluster.GetAllStores(), nil -} - -func (c *pdClient) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) { - c.gcSafePointMu.Lock() - defer c.gcSafePointMu.Unlock() - - if safePoint > c.gcSafePoint { - c.gcSafePoint = safePoint - } - return c.gcSafePoint, nil -} - -func (c *pdClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { - c.gcSafePointMu.Lock() - defer c.gcSafePointMu.Unlock() - - if ttl == 0 { - delete(c.serviceSafePoints, serviceID) - } else { - var minSafePoint uint64 = math.MaxUint64 - for _, ssp := range c.serviceSafePoints { - if ssp < minSafePoint { - minSafePoint = ssp - } - } - - if len(c.serviceSafePoints) == 0 || minSafePoint <= safePoint { - c.serviceSafePoints[serviceID] = safePoint - } - } - - // The minSafePoint may have changed. Reload it. - var minSafePoint uint64 = math.MaxUint64 - for _, ssp := range c.serviceSafePoints { - if ssp < minSafePoint { - minSafePoint = ssp - } - } - return minSafePoint, nil -} - -func (c *pdClient) Close() { -} - -func (c *pdClient) ScatterRegion(ctx context.Context, regionID uint64) error { - return nil -} - -func (c *pdClient) ScatterRegions(ctx context.Context, regionsID []uint64, opts ...pd.RegionsOption) (*pdpb.ScatterRegionResponse, error) { - return nil, nil -} - -func (c *pdClient) SplitRegions(ctx context.Context, splitKeys [][]byte, opts ...pd.RegionsOption) (*pdpb.SplitRegionsResponse, error) { - return nil, nil -} - -func (c *pdClient) GetOperator(ctx context.Context, regionID uint64) (*pdpb.GetOperatorResponse, error) { - return &pdpb.GetOperatorResponse{Status: pdpb.OperatorStatus_SUCCESS}, nil -} - -func (c *pdClient) GetAllMembers(ctx context.Context) ([]*pdpb.Member, error) { - return nil, nil -} - -func (c *pdClient) GetLeaderAddr() string { return "mockpd" } diff --git a/store/tikv/mockstore/mocktikv/rpc.go b/store/tikv/mockstore/mocktikv/rpc.go deleted file mode 100644 index 7ab8ef64e8e2f..0000000000000 --- a/store/tikv/mockstore/mocktikv/rpc.go +++ /dev/null @@ -1,979 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "bytes" - "context" - "math" - "strconv" - "sync" - "time" - - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/debugpb" - "github.com/pingcap/kvproto/pkg/errorpb" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" -) - -// For gofail injection. -var undeterminedErr = terror.ErrResultUndetermined - -const requestMaxSize = 8 * 1024 * 1024 - -func checkGoContext(ctx context.Context) error { - select { - case <-ctx.Done(): - return ctx.Err() - default: - return nil - } -} - -func convertToKeyError(err error) *kvrpcpb.KeyError { - if locked, ok := errors.Cause(err).(*ErrLocked); ok { - return &kvrpcpb.KeyError{ - Locked: &kvrpcpb.LockInfo{ - Key: locked.Key.Raw(), - PrimaryLock: locked.Primary, - LockVersion: locked.StartTS, - LockTtl: locked.TTL, - TxnSize: locked.TxnSize, - LockType: locked.LockType, - LockForUpdateTs: locked.ForUpdateTS, - }, - } - } - if alreadyExist, ok := errors.Cause(err).(*ErrKeyAlreadyExist); ok { - return &kvrpcpb.KeyError{ - AlreadyExist: &kvrpcpb.AlreadyExist{ - Key: alreadyExist.Key, - }, - } - } - if writeConflict, ok := errors.Cause(err).(*ErrConflict); ok { - return &kvrpcpb.KeyError{ - Conflict: &kvrpcpb.WriteConflict{ - Key: writeConflict.Key, - ConflictTs: writeConflict.ConflictTS, - ConflictCommitTs: writeConflict.ConflictCommitTS, - StartTs: writeConflict.StartTS, - }, - } - } - if dead, ok := errors.Cause(err).(*ErrDeadlock); ok { - return &kvrpcpb.KeyError{ - Deadlock: &kvrpcpb.Deadlock{ - LockTs: dead.LockTS, - LockKey: dead.LockKey, - DeadlockKeyHash: dead.DealockKeyHash, - }, - } - } - if retryable, ok := errors.Cause(err).(ErrRetryable); ok { - return &kvrpcpb.KeyError{ - Retryable: retryable.Error(), - } - } - if expired, ok := errors.Cause(err).(*ErrCommitTSExpired); ok { - return &kvrpcpb.KeyError{ - CommitTsExpired: &expired.CommitTsExpired, - } - } - if tmp, ok := errors.Cause(err).(*ErrTxnNotFound); ok { - return &kvrpcpb.KeyError{ - TxnNotFound: &tmp.TxnNotFound, - } - } - return &kvrpcpb.KeyError{ - Abort: err.Error(), - } -} - -func convertToKeyErrors(errs []error) []*kvrpcpb.KeyError { - var keyErrors = make([]*kvrpcpb.KeyError, 0) - for _, err := range errs { - if err != nil { - keyErrors = append(keyErrors, convertToKeyError(err)) - } - } - return keyErrors -} - -func convertToPbPairs(pairs []Pair) []*kvrpcpb.KvPair { - kvPairs := make([]*kvrpcpb.KvPair, 0, len(pairs)) - for _, p := range pairs { - var kvPair *kvrpcpb.KvPair - if p.Err == nil { - kvPair = &kvrpcpb.KvPair{ - Key: p.Key, - Value: p.Value, - } - } else { - kvPair = &kvrpcpb.KvPair{ - Error: convertToKeyError(p.Err), - } - } - kvPairs = append(kvPairs, kvPair) - } - return kvPairs -} - -// kvHandler mocks tikv's side handler behavior. In general, you may assume -// TiKV just translate the logic from Go to Rust. -type kvHandler struct { - *Session -} - -func (h kvHandler) handleKvGet(req *kvrpcpb.GetRequest) *kvrpcpb.GetResponse { - if !h.checkKeyInRegion(req.Key) { - panic("KvGet: key not in region") - } - - val, err := h.mvccStore.Get(req.Key, req.GetVersion(), h.isolationLevel, req.Context.GetResolvedLocks()) - if err != nil { - return &kvrpcpb.GetResponse{ - Error: convertToKeyError(err), - } - } - return &kvrpcpb.GetResponse{ - Value: val, - } -} - -func (h kvHandler) handleKvScan(req *kvrpcpb.ScanRequest) *kvrpcpb.ScanResponse { - endKey := MvccKey(h.endKey).Raw() - var pairs []Pair - if !req.Reverse { - if !h.checkKeyInRegion(req.GetStartKey()) { - panic("KvScan: startKey not in region") - } - if len(req.EndKey) > 0 && (len(endKey) == 0 || bytes.Compare(NewMvccKey(req.EndKey), h.endKey) < 0) { - endKey = req.EndKey - } - pairs = h.mvccStore.Scan(req.GetStartKey(), endKey, int(req.GetLimit()), req.GetVersion(), h.isolationLevel, req.Context.ResolvedLocks) - } else { - // TiKV use range [end_key, start_key) for reverse scan. - // Should use the req.EndKey to check in region. - if !h.checkKeyInRegion(req.GetEndKey()) { - panic("KvScan: startKey not in region") - } - - // TiKV use range [end_key, start_key) for reverse scan. - // So the req.StartKey actually is the end_key. - if len(req.StartKey) > 0 && (len(endKey) == 0 || bytes.Compare(NewMvccKey(req.StartKey), h.endKey) < 0) { - endKey = req.StartKey - } - - pairs = h.mvccStore.ReverseScan(req.EndKey, endKey, int(req.GetLimit()), req.GetVersion(), h.isolationLevel, req.Context.ResolvedLocks) - } - - return &kvrpcpb.ScanResponse{ - Pairs: convertToPbPairs(pairs), - } -} - -func (h kvHandler) handleKvPrewrite(req *kvrpcpb.PrewriteRequest) *kvrpcpb.PrewriteResponse { - regionID := req.Context.RegionId - h.cluster.handleDelay(req.StartVersion, regionID) - - for _, m := range req.Mutations { - if !h.checkKeyInRegion(m.Key) { - panic("KvPrewrite: key not in region") - } - } - errs := h.mvccStore.Prewrite(req) - return &kvrpcpb.PrewriteResponse{ - Errors: convertToKeyErrors(errs), - } -} - -func (h kvHandler) handleKvPessimisticLock(req *kvrpcpb.PessimisticLockRequest) *kvrpcpb.PessimisticLockResponse { - for _, m := range req.Mutations { - if !h.checkKeyInRegion(m.Key) { - panic("KvPessimisticLock: key not in region") - } - } - startTS := req.StartVersion - regionID := req.Context.RegionId - h.cluster.handleDelay(startTS, regionID) - return h.mvccStore.PessimisticLock(req) -} - -func simulateServerSideWaitLock(errs []error) { - for _, err := range errs { - if _, ok := err.(*ErrLocked); ok { - time.Sleep(time.Millisecond * 5) - break - } - } -} - -func (h kvHandler) handleKvPessimisticRollback(req *kvrpcpb.PessimisticRollbackRequest) *kvrpcpb.PessimisticRollbackResponse { - for _, key := range req.Keys { - if !h.checkKeyInRegion(key) { - panic("KvPessimisticRollback: key not in region") - } - } - errs := h.mvccStore.PessimisticRollback(req.Keys, req.StartVersion, req.ForUpdateTs) - return &kvrpcpb.PessimisticRollbackResponse{ - Errors: convertToKeyErrors(errs), - } -} - -func (h kvHandler) handleKvCommit(req *kvrpcpb.CommitRequest) *kvrpcpb.CommitResponse { - for _, k := range req.Keys { - if !h.checkKeyInRegion(k) { - panic("KvCommit: key not in region") - } - } - var resp kvrpcpb.CommitResponse - err := h.mvccStore.Commit(req.Keys, req.GetStartVersion(), req.GetCommitVersion()) - if err != nil { - resp.Error = convertToKeyError(err) - } - return &resp -} - -func (h kvHandler) handleKvCleanup(req *kvrpcpb.CleanupRequest) *kvrpcpb.CleanupResponse { - if !h.checkKeyInRegion(req.Key) { - panic("KvCleanup: key not in region") - } - var resp kvrpcpb.CleanupResponse - err := h.mvccStore.Cleanup(req.Key, req.GetStartVersion(), req.GetCurrentTs()) - if err != nil { - if commitTS, ok := errors.Cause(err).(ErrAlreadyCommitted); ok { - resp.CommitVersion = uint64(commitTS) - } else { - resp.Error = convertToKeyError(err) - } - } - return &resp -} - -func (h kvHandler) handleKvCheckTxnStatus(req *kvrpcpb.CheckTxnStatusRequest) *kvrpcpb.CheckTxnStatusResponse { - if !h.checkKeyInRegion(req.PrimaryKey) { - panic("KvCheckTxnStatus: key not in region") - } - var resp kvrpcpb.CheckTxnStatusResponse - ttl, commitTS, action, err := h.mvccStore.CheckTxnStatus(req.GetPrimaryKey(), req.GetLockTs(), req.GetCallerStartTs(), req.GetCurrentTs(), req.GetRollbackIfNotExist(), req.ResolvingPessimisticLock) - if err != nil { - resp.Error = convertToKeyError(err) - } else { - resp.LockTtl, resp.CommitVersion, resp.Action = ttl, commitTS, action - } - return &resp -} - -func (h kvHandler) handleTxnHeartBeat(req *kvrpcpb.TxnHeartBeatRequest) *kvrpcpb.TxnHeartBeatResponse { - if !h.checkKeyInRegion(req.PrimaryLock) { - panic("KvTxnHeartBeat: key not in region") - } - var resp kvrpcpb.TxnHeartBeatResponse - ttl, err := h.mvccStore.TxnHeartBeat(req.PrimaryLock, req.StartVersion, req.AdviseLockTtl) - if err != nil { - resp.Error = convertToKeyError(err) - } - resp.LockTtl = ttl - return &resp -} - -func (h kvHandler) handleKvBatchGet(req *kvrpcpb.BatchGetRequest) *kvrpcpb.BatchGetResponse { - for _, k := range req.Keys { - if !h.checkKeyInRegion(k) { - panic("KvBatchGet: key not in region") - } - } - pairs := h.mvccStore.BatchGet(req.Keys, req.GetVersion(), h.isolationLevel, req.Context.GetResolvedLocks()) - return &kvrpcpb.BatchGetResponse{ - Pairs: convertToPbPairs(pairs), - } -} - -func (h kvHandler) handleMvccGetByKey(req *kvrpcpb.MvccGetByKeyRequest) *kvrpcpb.MvccGetByKeyResponse { - debugger, ok := h.mvccStore.(MVCCDebugger) - if !ok { - return &kvrpcpb.MvccGetByKeyResponse{ - Error: "not implement", - } - } - - if !h.checkKeyInRegion(req.Key) { - panic("MvccGetByKey: key not in region") - } - var resp kvrpcpb.MvccGetByKeyResponse - resp.Info = debugger.MvccGetByKey(req.Key) - return &resp -} - -func (h kvHandler) handleMvccGetByStartTS(req *kvrpcpb.MvccGetByStartTsRequest) *kvrpcpb.MvccGetByStartTsResponse { - debugger, ok := h.mvccStore.(MVCCDebugger) - if !ok { - return &kvrpcpb.MvccGetByStartTsResponse{ - Error: "not implement", - } - } - var resp kvrpcpb.MvccGetByStartTsResponse - resp.Info, resp.Key = debugger.MvccGetByStartTS(req.StartTs) - return &resp -} - -func (h kvHandler) handleKvBatchRollback(req *kvrpcpb.BatchRollbackRequest) *kvrpcpb.BatchRollbackResponse { - err := h.mvccStore.Rollback(req.Keys, req.StartVersion) - if err != nil { - return &kvrpcpb.BatchRollbackResponse{ - Error: convertToKeyError(err), - } - } - return &kvrpcpb.BatchRollbackResponse{} -} - -func (h kvHandler) handleKvScanLock(req *kvrpcpb.ScanLockRequest) *kvrpcpb.ScanLockResponse { - startKey := MvccKey(h.startKey).Raw() - endKey := MvccKey(h.endKey).Raw() - locks, err := h.mvccStore.ScanLock(startKey, endKey, req.GetMaxVersion()) - if err != nil { - return &kvrpcpb.ScanLockResponse{ - Error: convertToKeyError(err), - } - } - return &kvrpcpb.ScanLockResponse{ - Locks: locks, - } -} - -func (h kvHandler) handleKvResolveLock(req *kvrpcpb.ResolveLockRequest) *kvrpcpb.ResolveLockResponse { - startKey := MvccKey(h.startKey).Raw() - endKey := MvccKey(h.endKey).Raw() - err := h.mvccStore.ResolveLock(startKey, endKey, req.GetStartVersion(), req.GetCommitVersion()) - if err != nil { - return &kvrpcpb.ResolveLockResponse{ - Error: convertToKeyError(err), - } - } - return &kvrpcpb.ResolveLockResponse{} -} - -func (h kvHandler) handleKvGC(req *kvrpcpb.GCRequest) *kvrpcpb.GCResponse { - startKey := MvccKey(h.startKey).Raw() - endKey := MvccKey(h.endKey).Raw() - err := h.mvccStore.GC(startKey, endKey, req.GetSafePoint()) - if err != nil { - return &kvrpcpb.GCResponse{ - Error: convertToKeyError(err), - } - } - return &kvrpcpb.GCResponse{} -} - -func (h kvHandler) handleKvDeleteRange(req *kvrpcpb.DeleteRangeRequest) *kvrpcpb.DeleteRangeResponse { - if !h.checkKeyInRegion(req.StartKey) { - panic("KvDeleteRange: key not in region") - } - var resp kvrpcpb.DeleteRangeResponse - err := h.mvccStore.DeleteRange(req.StartKey, req.EndKey) - if err != nil { - resp.Error = err.Error() - } - return &resp -} - -func (h kvHandler) handleKvRawGet(req *kvrpcpb.RawGetRequest) *kvrpcpb.RawGetResponse { - rawKV, ok := h.mvccStore.(RawKV) - if !ok { - return &kvrpcpb.RawGetResponse{ - Error: "not implemented", - } - } - return &kvrpcpb.RawGetResponse{ - Value: rawKV.RawGet(req.GetKey()), - } -} - -func (h kvHandler) handleKvRawBatchGet(req *kvrpcpb.RawBatchGetRequest) *kvrpcpb.RawBatchGetResponse { - rawKV, ok := h.mvccStore.(RawKV) - if !ok { - // TODO should we add error ? - return &kvrpcpb.RawBatchGetResponse{ - RegionError: &errorpb.Error{ - Message: "not implemented", - }, - } - } - values := rawKV.RawBatchGet(req.Keys) - kvPairs := make([]*kvrpcpb.KvPair, len(values)) - for i, key := range req.Keys { - kvPairs[i] = &kvrpcpb.KvPair{ - Key: key, - Value: values[i], - } - } - return &kvrpcpb.RawBatchGetResponse{ - Pairs: kvPairs, - } -} - -func (h kvHandler) handleKvRawPut(req *kvrpcpb.RawPutRequest) *kvrpcpb.RawPutResponse { - rawKV, ok := h.mvccStore.(RawKV) - if !ok { - return &kvrpcpb.RawPutResponse{ - Error: "not implemented", - } - } - rawKV.RawPut(req.GetKey(), req.GetValue()) - return &kvrpcpb.RawPutResponse{} -} - -func (h kvHandler) handleKvRawBatchPut(req *kvrpcpb.RawBatchPutRequest) *kvrpcpb.RawBatchPutResponse { - rawKV, ok := h.mvccStore.(RawKV) - if !ok { - return &kvrpcpb.RawBatchPutResponse{ - Error: "not implemented", - } - } - keys := make([][]byte, 0, len(req.Pairs)) - values := make([][]byte, 0, len(req.Pairs)) - for _, pair := range req.Pairs { - keys = append(keys, pair.Key) - values = append(values, pair.Value) - } - rawKV.RawBatchPut(keys, values) - return &kvrpcpb.RawBatchPutResponse{} -} - -func (h kvHandler) handleKvRawDelete(req *kvrpcpb.RawDeleteRequest) *kvrpcpb.RawDeleteResponse { - rawKV, ok := h.mvccStore.(RawKV) - if !ok { - return &kvrpcpb.RawDeleteResponse{ - Error: "not implemented", - } - } - rawKV.RawDelete(req.GetKey()) - return &kvrpcpb.RawDeleteResponse{} -} - -func (h kvHandler) handleKvRawBatchDelete(req *kvrpcpb.RawBatchDeleteRequest) *kvrpcpb.RawBatchDeleteResponse { - rawKV, ok := h.mvccStore.(RawKV) - if !ok { - return &kvrpcpb.RawBatchDeleteResponse{ - Error: "not implemented", - } - } - rawKV.RawBatchDelete(req.Keys) - return &kvrpcpb.RawBatchDeleteResponse{} -} - -func (h kvHandler) handleKvRawDeleteRange(req *kvrpcpb.RawDeleteRangeRequest) *kvrpcpb.RawDeleteRangeResponse { - rawKV, ok := h.mvccStore.(RawKV) - if !ok { - return &kvrpcpb.RawDeleteRangeResponse{ - Error: "not implemented", - } - } - rawKV.RawDeleteRange(req.GetStartKey(), req.GetEndKey()) - return &kvrpcpb.RawDeleteRangeResponse{} -} - -func (h kvHandler) handleKvRawScan(req *kvrpcpb.RawScanRequest) *kvrpcpb.RawScanResponse { - rawKV, ok := h.mvccStore.(RawKV) - if !ok { - errStr := "not implemented" - return &kvrpcpb.RawScanResponse{ - RegionError: &errorpb.Error{ - Message: errStr, - }, - } - } - - var pairs []Pair - if req.Reverse { - lowerBound := h.startKey - if bytes.Compare(req.EndKey, lowerBound) > 0 { - lowerBound = req.EndKey - } - pairs = rawKV.RawReverseScan( - req.StartKey, - lowerBound, - int(req.GetLimit()), - ) - } else { - upperBound := h.endKey - if len(req.EndKey) > 0 && (len(upperBound) == 0 || bytes.Compare(req.EndKey, upperBound) < 0) { - upperBound = req.EndKey - } - pairs = rawKV.RawScan( - req.StartKey, - upperBound, - int(req.GetLimit()), - ) - } - - return &kvrpcpb.RawScanResponse{ - Kvs: convertToPbPairs(pairs), - } -} - -func (h kvHandler) handleSplitRegion(req *kvrpcpb.SplitRegionRequest) *kvrpcpb.SplitRegionResponse { - keys := req.GetSplitKeys() - resp := &kvrpcpb.SplitRegionResponse{Regions: make([]*metapb.Region, 0, len(keys)+1)} - for i, key := range keys { - k := NewMvccKey(key) - region, _ := h.cluster.GetRegionByKey(k) - if bytes.Equal(region.GetStartKey(), key) { - continue - } - if i == 0 { - // Set the leftmost region. - resp.Regions = append(resp.Regions, region) - } - newRegionID, newPeerIDs := h.cluster.AllocID(), h.cluster.AllocIDs(len(region.Peers)) - newRegion := h.cluster.SplitRaw(region.GetId(), newRegionID, k, newPeerIDs, newPeerIDs[0]) - resp.Regions = append(resp.Regions, newRegion) - } - return resp -} - -// Client is a client that sends RPC. -// This is same with tikv.Client, define again for avoid circle import. -type Client interface { - // Close should release all data. - Close() error - // SendRequest sends Request. - SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) -} - -// CoprRPCHandler is the interface to handle coprocessor RPC commands. -type CoprRPCHandler interface { - HandleCmdCop(reqCtx *kvrpcpb.Context, session *Session, r *coprocessor.Request) *coprocessor.Response - HandleBatchCop(ctx context.Context, reqCtx *kvrpcpb.Context, session *Session, r *coprocessor.BatchRequest, timeout time.Duration) (*tikvrpc.BatchCopStreamResponse, error) - HandleCopStream(ctx context.Context, reqCtx *kvrpcpb.Context, session *Session, r *coprocessor.Request, timeout time.Duration) (*tikvrpc.CopStreamResponse, error) - Close() -} - -// RPCClient sends kv RPC calls to mock cluster. RPCClient mocks the behavior of -// a rpc client at tikv's side. -type RPCClient struct { - Cluster *Cluster - MvccStore MVCCStore - coprHandler CoprRPCHandler - // rpcCli uses to redirects RPC request to TiDB rpc server, It is only use for test. - // Mock TiDB rpc service will have circle import problem, so just use a real RPC client to send this RPC server. - // sync.Once uses to avoid concurrency initialize rpcCli. - sync.Once - rpcCli Client -} - -// NewRPCClient creates an RPCClient. -// Note that close the RPCClient may close the underlying MvccStore. -func NewRPCClient(cluster *Cluster, mvccStore MVCCStore, coprHandler CoprRPCHandler) *RPCClient { - return &RPCClient{ - Cluster: cluster, - MvccStore: mvccStore, - coprHandler: coprHandler, - } -} - -func (c *RPCClient) getAndCheckStoreByAddr(addr string) (*metapb.Store, error) { - stores, err := c.Cluster.GetAndCheckStoreByAddr(addr) - if err != nil { - return nil, err - } - if len(stores) == 0 { - return nil, errors.New("connect fail") - } - for _, store := range stores { - if store.GetState() != metapb.StoreState_Offline && - store.GetState() != metapb.StoreState_Tombstone { - return store, nil - } - } - return nil, errors.New("connection refused") -} - -func (c *RPCClient) checkArgs(ctx context.Context, addr string) (*Session, error) { - if err := checkGoContext(ctx); err != nil { - return nil, err - } - - store, err := c.getAndCheckStoreByAddr(addr) - if err != nil { - return nil, err - } - session := &Session{ - cluster: c.Cluster, - mvccStore: c.MvccStore, - // set store id for current request - storeID: store.GetId(), - } - return session, nil -} - -// GRPCClientFactory is the GRPC client factory. -// Use global variable to avoid circle import. -// TODO: remove this global variable. -var GRPCClientFactory func() Client - -// redirectRequestToRPCServer redirects RPC request to TiDB rpc server, It is only use for test. -// Mock TiDB rpc service will have circle import problem, so just use a real RPC client to send this RPC server. -func (c *RPCClient) redirectRequestToRPCServer(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - c.Once.Do(func() { - if GRPCClientFactory != nil { - c.rpcCli = GRPCClientFactory() - } - }) - if c.rpcCli == nil { - return nil, errors.Errorf("GRPCClientFactory is nil") - } - return c.rpcCli.SendRequest(ctx, addr, req, timeout) -} - -// SendRequest sends a request to mock cluster. -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 { - span1 := span.Tracer().StartSpan("RPCClient.SendRequest", opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - - if val, err := util.EvalFailpoint("rpcServerBusy"); err == nil { - if val.(bool) { - return tikvrpc.GenRegionErrorResp(req, &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}}) - } - } - - // increase coverage for mock tikv - _ = req.Type.String() - _ = req.ToBatchCommandsRequest() - - reqCtx := &req.Context - resp := &tikvrpc.Response{} - // When the store type is TiDB, the request should handle over to TiDB rpc server to handle. - if req.StoreTp == tikvrpc.TiDB { - return c.redirectRequestToRPCServer(ctx, addr, req, timeout) - } - - session, err := c.checkArgs(ctx, addr) - if err != nil { - return nil, err - } - switch req.Type { - case tikvrpc.CmdGet: - r := req.Get() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.GetResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvGet(r) - case tikvrpc.CmdScan: - r := req.Scan() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.ScanResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvScan(r) - - case tikvrpc.CmdPrewrite: - if val, err := util.EvalFailpoint("rpcPrewriteResult"); err == nil { - switch val.(string) { - case "notLeader": - return &tikvrpc.Response{ - Resp: &kvrpcpb.PrewriteResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}, - }, nil - } - } - - r := req.Prewrite() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.PrewriteResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvPrewrite(r) - case tikvrpc.CmdPessimisticLock: - r := req.PessimisticLock() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.PessimisticLockResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvPessimisticLock(r) - case tikvrpc.CmdPessimisticRollback: - r := req.PessimisticRollback() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.PessimisticRollbackResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvPessimisticRollback(r) - case tikvrpc.CmdCommit: - if val, err := util.EvalFailpoint("rpcCommitResult"); err == nil { - switch val.(string) { - case "timeout": - return nil, errors.New("timeout") - case "notLeader": - return &tikvrpc.Response{ - Resp: &kvrpcpb.CommitResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}, - }, nil - case "keyError": - return &tikvrpc.Response{ - Resp: &kvrpcpb.CommitResponse{Error: &kvrpcpb.KeyError{}}, - }, nil - } - } - - r := req.Commit() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.CommitResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvCommit(r) - if val, err := util.EvalFailpoint("rpcCommitTimeout"); err == nil { - if val.(bool) { - return nil, undeterminedErr - } - } - case tikvrpc.CmdCleanup: - r := req.Cleanup() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.CleanupResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvCleanup(r) - case tikvrpc.CmdCheckTxnStatus: - r := req.CheckTxnStatus() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.CheckTxnStatusResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvCheckTxnStatus(r) - case tikvrpc.CmdTxnHeartBeat: - r := req.TxnHeartBeat() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.TxnHeartBeatResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleTxnHeartBeat(r) - case tikvrpc.CmdBatchGet: - r := req.BatchGet() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.BatchGetResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvBatchGet(r) - case tikvrpc.CmdBatchRollback: - r := req.BatchRollback() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.BatchRollbackResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvBatchRollback(r) - case tikvrpc.CmdScanLock: - r := req.ScanLock() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.ScanLockResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvScanLock(r) - case tikvrpc.CmdResolveLock: - r := req.ResolveLock() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.ResolveLockResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvResolveLock(r) - case tikvrpc.CmdGC: - r := req.GC() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.GCResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvGC(r) - case tikvrpc.CmdDeleteRange: - r := req.DeleteRange() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.DeleteRangeResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvDeleteRange(r) - case tikvrpc.CmdRawGet: - r := req.RawGet() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.RawGetResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvRawGet(r) - case tikvrpc.CmdRawBatchGet: - r := req.RawBatchGet() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.RawBatchGetResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvRawBatchGet(r) - case tikvrpc.CmdRawPut: - r := req.RawPut() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.RawPutResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvRawPut(r) - case tikvrpc.CmdRawBatchPut: - r := req.RawBatchPut() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.RawBatchPutResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvRawBatchPut(r) - case tikvrpc.CmdRawDelete: - r := req.RawDelete() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.RawDeleteResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvRawDelete(r) - case tikvrpc.CmdRawBatchDelete: - r := req.RawBatchDelete() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.RawBatchDeleteResponse{RegionError: err} - } - resp.Resp = kvHandler{session}.handleKvRawBatchDelete(r) - case tikvrpc.CmdRawDeleteRange: - r := req.RawDeleteRange() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.RawDeleteRangeResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvRawDeleteRange(r) - case tikvrpc.CmdRawScan: - r := req.RawScan() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.RawScanResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleKvRawScan(r) - case tikvrpc.CmdUnsafeDestroyRange: - panic("unimplemented") - case tikvrpc.CmdRegisterLockObserver: - return nil, errors.New("unimplemented") - case tikvrpc.CmdCheckLockObserver: - return nil, errors.New("unimplemented") - case tikvrpc.CmdRemoveLockObserver: - return nil, errors.New("unimplemented") - case tikvrpc.CmdPhysicalScanLock: - return nil, errors.New("unimplemented") - case tikvrpc.CmdCop: - if c.coprHandler == nil { - return nil, errors.New("unimplemented") - } - session.rawStartKey = MvccKey(session.startKey).Raw() - session.rawEndKey = MvccKey(session.endKey).Raw() - resp.Resp = c.coprHandler.HandleCmdCop(reqCtx, session, req.Cop()) - case tikvrpc.CmdBatchCop: - if value, err := util.EvalFailpoint("BatchCopCancelled"); err == nil { - if value.(bool) { - return nil, context.Canceled - } - } - - if value, err := util.EvalFailpoint("BatchCopRpcErr"); err != nil { - if value.(string) == addr { - return nil, errors.New("rpc error") - } - } - if c.coprHandler == nil { - return nil, errors.New("unimplemented") - } - batchResp, err := c.coprHandler.HandleBatchCop(ctx, reqCtx, session, req.BatchCop(), timeout) - if err != nil { - return nil, errors.Trace(err) - } - resp.Resp = batchResp - case tikvrpc.CmdCopStream: - if c.coprHandler == nil { - return nil, errors.New("unimplemented") - } - session.rawStartKey = MvccKey(session.startKey).Raw() - session.rawEndKey = MvccKey(session.endKey).Raw() - streamResp, err := c.coprHandler.HandleCopStream(ctx, reqCtx, session, req.Cop(), timeout) - if err != nil { - return nil, errors.Trace(err) - } - resp.Resp = streamResp - case tikvrpc.CmdMvccGetByKey: - r := req.MvccGetByKey() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.MvccGetByKeyResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleMvccGetByKey(r) - case tikvrpc.CmdMvccGetByStartTs: - r := req.MvccGetByStartTs() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.MvccGetByStartTsResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleMvccGetByStartTS(r) - case tikvrpc.CmdSplitRegion: - r := req.SplitRegion() - if err := session.checkRequest(reqCtx, r.Size()); err != nil { - resp.Resp = &kvrpcpb.SplitRegionResponse{RegionError: err} - return resp, nil - } - resp.Resp = kvHandler{session}.handleSplitRegion(r) - // DebugGetRegionProperties is for fast analyze in mock tikv. - case tikvrpc.CmdDebugGetRegionProperties: - r := req.DebugGetRegionProperties() - region, _ := c.Cluster.GetRegion(r.RegionId) - var reqCtx kvrpcpb.Context - scanResp := kvHandler{session}.handleKvScan(&kvrpcpb.ScanRequest{ - Context: &reqCtx, - StartKey: MvccKey(region.StartKey).Raw(), - EndKey: MvccKey(region.EndKey).Raw(), - Version: math.MaxUint64, - Limit: math.MaxUint32}) - resp.Resp = &debugpb.GetRegionPropertiesResponse{ - Props: []*debugpb.Property{{ - Name: "mvcc.num_rows", - Value: strconv.Itoa(len(scanResp.Pairs)), - }}} - default: - return nil, errors.Errorf("unsupported this request type %v", req.Type) - } - return resp, nil -} - -// Close closes the client. -func (c *RPCClient) Close() error { - if c.coprHandler != nil { - c.coprHandler.Close() - } - - var err error - if c.MvccStore != nil { - err = c.MvccStore.Close() - if err != nil { - return err - } - } - - if c.rpcCli != nil { - err = c.rpcCli.Close() - if err != nil { - return err - } - } - return nil -} diff --git a/store/tikv/mockstore/mocktikv/session.go b/store/tikv/mockstore/mocktikv/session.go deleted file mode 100644 index 7997c6c9f4ca7..0000000000000 --- a/store/tikv/mockstore/mocktikv/session.go +++ /dev/null @@ -1,171 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import ( - "github.com/gogo/protobuf/proto" - "github.com/pingcap/kvproto/pkg/errorpb" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" -) - -// Session stores session scope rpc data. -type Session struct { - cluster *Cluster - mvccStore MVCCStore - - // storeID stores id for current request - storeID uint64 - // startKey is used for handling normal request. - startKey []byte - endKey []byte - // rawStartKey is used for handling coprocessor request. - rawStartKey []byte - rawEndKey []byte - // isolationLevel is used for current request. - isolationLevel kvrpcpb.IsolationLevel - resolvedLocks []uint64 -} - -// GetIsolationLevel returns the session's isolation level. -func (s *Session) GetIsolationLevel() kvrpcpb.IsolationLevel { - return s.isolationLevel -} - -// GetMVCCStore returns the mock mvcc store. -func (s *Session) GetMVCCStore() MVCCStore { - return s.mvccStore -} - -// GetRawStartKey returns the raw start key of the request. -func (s *Session) GetRawStartKey() []byte { - return s.rawStartKey -} - -// GetRawEndKey returns the raw end key of the request. -func (s *Session) GetRawEndKey() []byte { - return s.rawEndKey -} - -// GetResolvedLocks returns the resolved locks of the request. -func (s *Session) GetResolvedLocks() []uint64 { - return s.resolvedLocks -} - -// CheckRequestContext checks if the context matches the request status. -func (s *Session) CheckRequestContext(ctx *kvrpcpb.Context) *errorpb.Error { - ctxPeer := ctx.GetPeer() - if ctxPeer != nil && ctxPeer.GetStoreId() != s.storeID { - return &errorpb.Error{ - Message: *proto.String("store not match"), - StoreNotMatch: &errorpb.StoreNotMatch{}, - } - } - region, leaderID := s.cluster.GetRegion(ctx.GetRegionId()) - // No region found. - if region == nil { - return &errorpb.Error{ - Message: *proto.String("region not found"), - RegionNotFound: &errorpb.RegionNotFound{ - RegionId: *proto.Uint64(ctx.GetRegionId()), - }, - } - } - var storePeer, leaderPeer *metapb.Peer - for _, p := range region.Peers { - if p.GetStoreId() == s.storeID { - storePeer = p - } - if p.GetId() == leaderID { - leaderPeer = p - } - } - // The Store does not contain a Peer of the Region. - if storePeer == nil { - return &errorpb.Error{ - Message: *proto.String("region not found"), - RegionNotFound: &errorpb.RegionNotFound{ - RegionId: *proto.Uint64(ctx.GetRegionId()), - }, - } - } - // No leader. - if leaderPeer == nil { - return &errorpb.Error{ - Message: *proto.String("no leader"), - NotLeader: &errorpb.NotLeader{ - RegionId: *proto.Uint64(ctx.GetRegionId()), - }, - } - } - // The Peer on the Store is not leader. If it's tiflash store , we pass this check. - if storePeer.GetId() != leaderPeer.GetId() && !isTiFlashStore(s.cluster.GetStore(storePeer.GetStoreId())) { - return &errorpb.Error{ - Message: *proto.String("not leader"), - NotLeader: &errorpb.NotLeader{ - RegionId: *proto.Uint64(ctx.GetRegionId()), - Leader: leaderPeer, - }, - } - } - // Region epoch does not match. - if !proto.Equal(region.GetRegionEpoch(), ctx.GetRegionEpoch()) { - nextRegion, _ := s.cluster.GetRegionByKey(region.GetEndKey()) - currentRegions := []*metapb.Region{region} - if nextRegion != nil { - currentRegions = append(currentRegions, nextRegion) - } - return &errorpb.Error{ - Message: *proto.String("epoch not match"), - EpochNotMatch: &errorpb.EpochNotMatch{ - CurrentRegions: currentRegions, - }, - } - } - s.startKey, s.endKey = region.StartKey, region.EndKey - s.isolationLevel = ctx.IsolationLevel - s.resolvedLocks = ctx.ResolvedLocks - return nil -} - -func (s *Session) checkRequestSize(size int) *errorpb.Error { - // TiKV has a limitation on raft log size. - // mocktikv has no raft inside, so we check the request's size instead. - if size >= requestMaxSize { - return &errorpb.Error{ - RaftEntryTooLarge: &errorpb.RaftEntryTooLarge{}, - } - } - return nil -} - -func (s *Session) checkRequest(ctx *kvrpcpb.Context, size int) *errorpb.Error { - if err := s.CheckRequestContext(ctx); err != nil { - return err - } - return s.checkRequestSize(size) -} - -func (s *Session) checkKeyInRegion(key []byte) bool { - return regionContains(s.startKey, s.endKey, NewMvccKey(key)) -} - -func isTiFlashStore(store *metapb.Store) bool { - for _, l := range store.GetLabels() { - if l.GetKey() == "engine" && l.GetValue() == "tiflash" { - return true - } - } - return false -} diff --git a/store/tikv/mockstore/mocktikv/utils.go b/store/tikv/mockstore/mocktikv/utils.go deleted file mode 100644 index fbd7c2b818b5a..0000000000000 --- a/store/tikv/mockstore/mocktikv/utils.go +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mocktikv - -import "github.com/pingcap/kvproto/pkg/kvrpcpb" - -// PutMutations is exported for testing. -var PutMutations = putMutations - -func putMutations(kvpairs ...string) []*kvrpcpb.Mutation { - var mutations []*kvrpcpb.Mutation - for i := 0; i < len(kvpairs); i += 2 { - mutations = append(mutations, &kvrpcpb.Mutation{ - Op: kvrpcpb.Op_Put, - Key: []byte(kvpairs[i]), - Value: []byte(kvpairs[i+1]), - }) - } - return mutations -} - -// MustPrewrite write mutations to mvcc store. -func MustPrewrite(store MVCCStore, mutations []*kvrpcpb.Mutation, primary string, startTS uint64, ttl uint64) bool { - return mustPrewriteWithTTL(store, mutations, primary, startTS, ttl) -} - -func mustPrewriteWithTTL(store MVCCStore, mutations []*kvrpcpb.Mutation, primary string, startTS uint64, ttl uint64) bool { - req := &kvrpcpb.PrewriteRequest{ - Mutations: mutations, - PrimaryLock: []byte(primary), - StartVersion: startTS, - LockTtl: ttl, - MinCommitTs: startTS + 1, - } - errs := store.Prewrite(req) - for _, err := range errs { - if err != nil { - return false - } - } - return true -} diff --git a/store/tikv/mockstore/test_suite.go b/store/tikv/mockstore/test_suite.go deleted file mode 100644 index 9ec7a0fa8aeef..0000000000000 --- a/store/tikv/mockstore/test_suite.go +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package mockstore - -import ( - "flag" - "sync" - - "github.com/pingcap/check" -) - -var ( - withTiKVGlobalLock sync.RWMutex - // WithTiKV is the flag which indicates whether it runs with tikv. - WithTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") -) - -// OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. -type OneByOneSuite struct{} - -// SetUpSuite implements the interface check.Suite. -func (s *OneByOneSuite) SetUpSuite(c *check.C) { - if *WithTiKV { - withTiKVGlobalLock.Lock() - } else { - withTiKVGlobalLock.RLock() - } -} - -// TearDownSuite implements the interface check.Suite. -func (s *OneByOneSuite) TearDownSuite(c *check.C) { - if *WithTiKV { - withTiKVGlobalLock.Unlock() - } else { - withTiKVGlobalLock.RUnlock() - } -} - -// LockGlobalTiKV locks withTiKVGlobalLock. -func (s *OneByOneSuite) LockGlobalTiKV() { - withTiKVGlobalLock.Lock() -} - -// UnLockGlobalTiKV unlocks withTiKVGlobalLock -func (s *OneByOneSuite) UnLockGlobalTiKV() { - withTiKVGlobalLock.Unlock() -} diff --git a/store/tikv/oracle/oracle.go b/store/tikv/oracle/oracle.go deleted file mode 100644 index fd95f1b357013..0000000000000 --- a/store/tikv/oracle/oracle.go +++ /dev/null @@ -1,86 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package oracle - -import ( - "context" - "time" -) - -// Option represents available options for the oracle.Oracle. -type Option struct { - TxnScope string -} - -// Oracle is the interface that provides strictly ascending timestamps. -type Oracle interface { - GetTimestamp(ctx context.Context, opt *Option) (uint64, error) - GetTimestampAsync(ctx context.Context, opt *Option) Future - GetLowResolutionTimestamp(ctx context.Context, opt *Option) (uint64, error) - GetLowResolutionTimestampAsync(ctx context.Context, opt *Option) Future - GetStaleTimestamp(ctx context.Context, txnScope string, prevSecond uint64) (uint64, error) - IsExpired(lockTimestamp, TTL uint64, opt *Option) bool - UntilExpired(lockTimeStamp, TTL uint64, opt *Option) int64 - Close() -} - -// Future is a future which promises to return a timestamp. -type Future interface { - Wait() (uint64, error) -} - -const ( - physicalShiftBits = 18 - logicalBits = (1 << physicalShiftBits) - 1 - // GlobalTxnScope is the default transaction scope for a Oracle service. - GlobalTxnScope = "global" -) - -// ComposeTS creates a ts from physical and logical parts. -func ComposeTS(physical, logical int64) uint64 { - return uint64((physical << physicalShiftBits) + logical) -} - -// ExtractPhysical returns a ts's physical part. -func ExtractPhysical(ts uint64) int64 { - return int64(ts >> physicalShiftBits) -} - -// ExtractLogical return a ts's logical part. -func ExtractLogical(ts uint64) int64 { - return int64(ts & logicalBits) -} - -// GetPhysical returns physical from an instant time with millisecond precision. -func GetPhysical(t time.Time) int64 { - return t.UnixNano() / int64(time.Millisecond) -} - -// GetTimeFromTS extracts time.Time from a timestamp. -func GetTimeFromTS(ts uint64) time.Time { - ms := ExtractPhysical(ts) - return time.Unix(ms/1e3, (ms%1e3)*1e6) -} - -// GoTimeToTS converts a Go time to uint64 timestamp. -func GoTimeToTS(t time.Time) uint64 { - ts := (t.UnixNano() / int64(time.Millisecond)) << physicalShiftBits - return uint64(ts) -} - -// GoTimeToLowerLimitStartTS returns the min start_ts of the uncommitted transaction. -// maxTxnTimeUse means the max time a Txn May use (in ms) from its begin to commit. -func GoTimeToLowerLimitStartTS(now time.Time, maxTxnTimeUse int64) uint64 { - return GoTimeToTS(now.Add(-time.Duration(maxTxnTimeUse) * time.Millisecond)) -} diff --git a/store/tikv/oracle/oracles/export_test.go b/store/tikv/oracle/oracles/export_test.go deleted file mode 100644 index a8911f920c585..0000000000000 --- a/store/tikv/oracle/oracles/export_test.go +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package oracles - -import ( - "sync/atomic" - "time" - - "github.com/pingcap/tidb/store/tikv/oracle" -) - -// SetOracleHookCurrentTime exports localOracle's time hook to test. -func SetOracleHookCurrentTime(oc oracle.Oracle, t time.Time) { - switch o := oc.(type) { - case *localOracle: - if o.hook == nil { - o.hook = &struct { - currentTime time.Time - }{} - } - o.hook.currentTime = t - } -} - -// NewEmptyPDOracle exports pdOracle struct to test -func NewEmptyPDOracle() oracle.Oracle { - return &pdOracle{} -} - -// SetEmptyPDOracleLastTs exports PD oracle's global last ts to test. -func SetEmptyPDOracleLastTs(oc oracle.Oracle, ts uint64) { - switch o := oc.(type) { - case *pdOracle: - lastTSInterface, _ := o.lastTSMap.LoadOrStore(oracle.GlobalTxnScope, new(uint64)) - lastTSPointer := lastTSInterface.(*uint64) - atomic.StoreUint64(lastTSPointer, ts) - lasTSArrivalInterface, _ := o.lastArrivalTSMap.LoadOrStore(oracle.GlobalTxnScope, new(uint64)) - lasTSArrivalPointer := lasTSArrivalInterface.(*uint64) - atomic.StoreUint64(lasTSArrivalPointer, uint64(time.Now().Unix()*1000)) - } - setEmptyPDOracleLastArrivalTs(oc, ts) -} - -// setEmptyPDOracleLastArrivalTs exports PD oracle's global last ts to test. -func setEmptyPDOracleLastArrivalTs(oc oracle.Oracle, ts uint64) { - switch o := oc.(type) { - case *pdOracle: - o.setLastArrivalTS(ts, oracle.GlobalTxnScope) - } -} diff --git a/store/tikv/oracle/oracles/local.go b/store/tikv/oracle/oracles/local.go deleted file mode 100644 index 8c7f8a30de645..0000000000000 --- a/store/tikv/oracle/oracles/local.go +++ /dev/null @@ -1,105 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package oracles - -import ( - "context" - "sync" - "time" - - "github.com/pingcap/tidb/store/tikv/oracle" -) - -var _ oracle.Oracle = &localOracle{} - -type localOracle struct { - sync.Mutex - lastTimeStampTS uint64 - n uint64 - hook *struct { - currentTime time.Time - } -} - -// NewLocalOracle creates an Oracle that uses local time as data source. -func NewLocalOracle() oracle.Oracle { - return &localOracle{} -} - -func (l *localOracle) IsExpired(lockTS, TTL uint64, _ *oracle.Option) bool { - now := time.Now() - if l.hook != nil { - now = l.hook.currentTime - } - expire := oracle.GetTimeFromTS(lockTS).Add(time.Duration(TTL) * time.Millisecond) - return !now.Before(expire) -} - -func (l *localOracle) GetTimestamp(ctx context.Context, _ *oracle.Option) (uint64, error) { - l.Lock() - defer l.Unlock() - now := time.Now() - if l.hook != nil { - now = l.hook.currentTime - } - ts := oracle.GoTimeToTS(now) - if l.lastTimeStampTS == ts { - l.n++ - return ts + l.n, nil - } - l.lastTimeStampTS = ts - l.n = 0 - return ts, nil -} - -func (l *localOracle) GetTimestampAsync(ctx context.Context, _ *oracle.Option) oracle.Future { - return &future{ - ctx: ctx, - l: l, - } -} - -func (l *localOracle) GetLowResolutionTimestamp(ctx context.Context, opt *oracle.Option) (uint64, error) { - return l.GetTimestamp(ctx, opt) -} - -func (l *localOracle) GetLowResolutionTimestampAsync(ctx context.Context, opt *oracle.Option) oracle.Future { - return l.GetTimestampAsync(ctx, opt) -} - -// GetStaleTimestamp return physical -func (l *localOracle) GetStaleTimestamp(ctx context.Context, txnScope string, prevSecond uint64) (ts uint64, err error) { - return oracle.GoTimeToTS(time.Now().Add(-time.Second * time.Duration(prevSecond))), nil -} - -type future struct { - ctx context.Context - l *localOracle -} - -func (f *future) Wait() (uint64, error) { - return f.l.GetTimestamp(f.ctx, &oracle.Option{}) -} - -// UntilExpired implement oracle.Oracle interface. -func (l *localOracle) UntilExpired(lockTimeStamp, TTL uint64, opt *oracle.Option) int64 { - now := time.Now() - if l.hook != nil { - now = l.hook.currentTime - } - return oracle.ExtractPhysical(lockTimeStamp) + int64(TTL) - oracle.GetPhysical(now) -} - -func (l *localOracle) Close() { -} diff --git a/store/tikv/oracle/oracles/local_test.go b/store/tikv/oracle/oracles/local_test.go deleted file mode 100644 index b478811002e84..0000000000000 --- a/store/tikv/oracle/oracles/local_test.go +++ /dev/null @@ -1,63 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package oracles_test - -import ( - "context" - "time" - - . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/oracle/oracles" -) - -var _ = Suite(&testOraclesSuite{}) - -type testOraclesSuite struct{} - -func (s *testOraclesSuite) TestLocalOracle(c *C) { - l := oracles.NewLocalOracle() - defer l.Close() - m := map[uint64]struct{}{} - for i := 0; i < 100000; i++ { - ts, err := l.GetTimestamp(context.Background(), &oracle.Option{}) - c.Assert(err, IsNil) - m[ts] = struct{}{} - } - - c.Assert(len(m), Equals, 100000, Commentf("should generate same ts")) -} - -func (s *testOraclesSuite) TestIsExpired(c *C) { - o := oracles.NewLocalOracle() - defer o.Close() - start := time.Now() - oracles.SetOracleHookCurrentTime(o, start) - ts, _ := o.GetTimestamp(context.Background(), &oracle.Option{}) - oracles.SetOracleHookCurrentTime(o, start.Add(10*time.Millisecond)) - expire := o.IsExpired(ts, 5, &oracle.Option{}) - c.Assert(expire, IsTrue, Commentf("should expire")) - expire = o.IsExpired(ts, 200, &oracle.Option{}) - c.Assert(expire, IsFalse, Commentf("should not expire")) -} - -func (s *testOraclesSuite) TestLocalOracle_UntilExpired(c *C) { - o := oracles.NewLocalOracle() - defer o.Close() - start := time.Now() - oracles.SetOracleHookCurrentTime(o, start) - ts, _ := o.GetTimestamp(context.Background(), &oracle.Option{}) - oracles.SetOracleHookCurrentTime(o, start.Add(10*time.Millisecond)) - c.Assert(o.UntilExpired(ts, 5, &oracle.Option{}) == -5 && o.UntilExpired(ts, 15, &oracle.Option{}) == 5, IsTrue, Commentf("before it is expired, it should be +-5")) -} diff --git a/store/tikv/oracle/oracles/mock.go b/store/tikv/oracle/oracles/mock.go deleted file mode 100644 index b1eabe57feb37..0000000000000 --- a/store/tikv/oracle/oracles/mock.go +++ /dev/null @@ -1,121 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package oracles - -import ( - "context" - "sync" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv/oracle" -) - -var errStopped = errors.New("stopped") - -// MockOracle is a mock oracle for test. -type MockOracle struct { - sync.RWMutex - stop bool - offset time.Duration - lastTS uint64 -} - -// Enable enables the Oracle -func (o *MockOracle) Enable() { - o.Lock() - defer o.Unlock() - o.stop = false -} - -// Disable disables the Oracle -func (o *MockOracle) Disable() { - o.Lock() - defer o.Unlock() - o.stop = true -} - -// AddOffset adds the offset of the oracle. -func (o *MockOracle) AddOffset(d time.Duration) { - o.Lock() - defer o.Unlock() - - o.offset += d -} - -// GetTimestamp implements oracle.Oracle interface. -func (o *MockOracle) GetTimestamp(ctx context.Context, _ *oracle.Option) (uint64, error) { - o.Lock() - defer o.Unlock() - - if o.stop { - return 0, errors.Trace(errStopped) - } - ts := oracle.GoTimeToTS(time.Now().Add(o.offset)) - if oracle.ExtractPhysical(o.lastTS) == oracle.ExtractPhysical(ts) { - ts = o.lastTS + 1 - } - o.lastTS = ts - return ts, nil -} - -// GetStaleTimestamp implements oracle.Oracle interface. -func (o *MockOracle) GetStaleTimestamp(ctx context.Context, txnScope string, prevSecond uint64) (ts uint64, err error) { - return oracle.GoTimeToTS(time.Now().Add(-time.Second * time.Duration(prevSecond))), nil -} - -type mockOracleFuture struct { - o *MockOracle - ctx context.Context -} - -func (m *mockOracleFuture) Wait() (uint64, error) { - return m.o.GetTimestamp(m.ctx, &oracle.Option{}) -} - -// GetTimestampAsync implements oracle.Oracle interface. -func (o *MockOracle) GetTimestampAsync(ctx context.Context, _ *oracle.Option) oracle.Future { - return &mockOracleFuture{o, ctx} -} - -// GetLowResolutionTimestamp implements oracle.Oracle interface. -func (o *MockOracle) GetLowResolutionTimestamp(ctx context.Context, opt *oracle.Option) (uint64, error) { - return o.GetTimestamp(ctx, opt) -} - -// GetLowResolutionTimestampAsync implements oracle.Oracle interface. -func (o *MockOracle) GetLowResolutionTimestampAsync(ctx context.Context, opt *oracle.Option) oracle.Future { - return o.GetTimestampAsync(ctx, opt) -} - -// IsExpired implements oracle.Oracle interface. -func (o *MockOracle) IsExpired(lockTimestamp, TTL uint64, _ *oracle.Option) bool { - o.RLock() - defer o.RUnlock() - expire := oracle.GetTimeFromTS(lockTimestamp).Add(time.Duration(TTL) * time.Millisecond) - return !time.Now().Add(o.offset).Before(expire) -} - -// UntilExpired implement oracle.Oracle interface. -func (o *MockOracle) UntilExpired(lockTimeStamp, TTL uint64, _ *oracle.Option) int64 { - o.RLock() - defer o.RUnlock() - expire := oracle.GetTimeFromTS(lockTimeStamp).Add(time.Duration(TTL) * time.Millisecond) - return expire.Sub(time.Now().Add(o.offset)).Milliseconds() -} - -// Close implements oracle.Oracle interface. -func (o *MockOracle) Close() { - -} diff --git a/store/tikv/oracle/oracles/pd.go b/store/tikv/oracle/oracles/pd.go deleted file mode 100644 index 907dc278d71cb..0000000000000 --- a/store/tikv/oracle/oracles/pd.go +++ /dev/null @@ -1,308 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package oracles - -import ( - "context" - "strings" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/oracle" - pd "github.com/tikv/pd/client" - "go.uber.org/zap" -) - -var _ oracle.Oracle = &pdOracle{} - -const slowDist = 30 * time.Millisecond - -// pdOracle is an Oracle that uses a placement driver client as source. -type pdOracle struct { - c pd.Client - // txn_scope (string) -> lastTSPointer (*uint64) - lastTSMap sync.Map - // txn_scope (string) -> lastArrivalTSPointer (*uint64) - lastArrivalTSMap sync.Map - quit chan struct{} -} - -// NewPdOracle create an Oracle that uses a pd client source. -// Refer https://github.com/tikv/pd/blob/master/client/client.go for more details. -// PdOracle mantains `lastTS` to store the last timestamp got from PD server. If -// `GetTimestamp()` is not called after `updateInterval`, it will be called by -// itself to keep up with the timestamp on PD server. -func NewPdOracle(pdClient pd.Client, updateInterval time.Duration) (oracle.Oracle, error) { - o := &pdOracle{ - c: pdClient, - quit: make(chan struct{}), - } - ctx := context.TODO() - go o.updateTS(ctx, updateInterval) - // Initialize the timestamp of the global txnScope by Get. - _, err := o.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) - if err != nil { - o.Close() - return nil, errors.Trace(err) - } - return o, nil -} - -// IsExpired returns whether lockTS+TTL is expired, both are ms. It uses `lastTS` -// to compare, may return false negative result temporarily. -func (o *pdOracle) IsExpired(lockTS, TTL uint64, opt *oracle.Option) bool { - lastTS, exist := o.getLastTS(opt.TxnScope) - if !exist { - return true - } - return oracle.ExtractPhysical(lastTS) >= oracle.ExtractPhysical(lockTS)+int64(TTL) -} - -// GetTimestamp gets a new increasing time. -func (o *pdOracle) GetTimestamp(ctx context.Context, opt *oracle.Option) (uint64, error) { - ts, err := o.getTimestamp(ctx, opt.TxnScope) - if err != nil { - return 0, errors.Trace(err) - } - o.setLastTS(ts, opt.TxnScope) - return ts, nil -} - -type tsFuture struct { - pd.TSFuture - o *pdOracle - txnScope string -} - -// Wait implements the oracle.Future interface. -func (f *tsFuture) Wait() (uint64, error) { - now := time.Now() - physical, logical, err := f.TSFuture.Wait() - metrics.TiKVTSFutureWaitDuration.Observe(time.Since(now).Seconds()) - if err != nil { - return 0, errors.Trace(err) - } - ts := oracle.ComposeTS(physical, logical) - f.o.setLastTS(ts, f.txnScope) - return ts, nil -} - -func (o *pdOracle) GetTimestampAsync(ctx context.Context, opt *oracle.Option) oracle.Future { - var ts pd.TSFuture - if opt.TxnScope == oracle.GlobalTxnScope || opt.TxnScope == "" { - ts = o.c.GetTSAsync(ctx) - } else { - ts = o.c.GetLocalTSAsync(ctx, opt.TxnScope) - } - return &tsFuture{ts, o, opt.TxnScope} -} - -func (o *pdOracle) getTimestamp(ctx context.Context, txnScope string) (uint64, error) { - now := time.Now() - var ( - physical, logical int64 - err error - ) - if txnScope == oracle.GlobalTxnScope || txnScope == "" { - physical, logical, err = o.c.GetTS(ctx) - } else { - physical, logical, err = o.c.GetLocalTS(ctx, txnScope) - } - if err != nil { - return 0, errors.Trace(err) - } - dist := time.Since(now) - if dist > slowDist { - logutil.Logger(ctx).Warn("get timestamp too slow", - zap.Duration("cost time", dist)) - } - return oracle.ComposeTS(physical, logical), nil -} - -func (o *pdOracle) getArrivalTimestamp() uint64 { - return oracle.GoTimeToTS(time.Now()) -} - -func (o *pdOracle) setLastTS(ts uint64, txnScope string) { - if txnScope == "" { - txnScope = oracle.GlobalTxnScope - } - lastTSInterface, ok := o.lastTSMap.Load(txnScope) - if !ok { - lastTSInterface, _ = o.lastTSMap.LoadOrStore(txnScope, new(uint64)) - } - lastTSPointer := lastTSInterface.(*uint64) - for { - lastTS := atomic.LoadUint64(lastTSPointer) - if ts <= lastTS { - return - } - if atomic.CompareAndSwapUint64(lastTSPointer, lastTS, ts) { - break - } - } - o.setLastArrivalTS(o.getArrivalTimestamp(), txnScope) -} - -func (o *pdOracle) setLastArrivalTS(ts uint64, txnScope string) { - if txnScope == "" { - txnScope = oracle.GlobalTxnScope - } - lastTSInterface, ok := o.lastArrivalTSMap.Load(txnScope) - if !ok { - lastTSInterface, _ = o.lastArrivalTSMap.LoadOrStore(txnScope, new(uint64)) - } - lastTSPointer := lastTSInterface.(*uint64) - for { - lastTS := atomic.LoadUint64(lastTSPointer) - if ts <= lastTS { - return - } - if atomic.CompareAndSwapUint64(lastTSPointer, lastTS, ts) { - return - } - } -} - -func (o *pdOracle) getLastTS(txnScope string) (uint64, bool) { - if txnScope == "" { - txnScope = oracle.GlobalTxnScope - } - lastTSInterface, ok := o.lastTSMap.Load(txnScope) - if !ok { - return 0, false - } - return atomic.LoadUint64(lastTSInterface.(*uint64)), true -} - -func (o *pdOracle) getLastArrivalTS(txnScope string) (uint64, bool) { - if txnScope == "" { - txnScope = oracle.GlobalTxnScope - } - lastArrivalTSInterface, ok := o.lastArrivalTSMap.Load(txnScope) - if !ok { - return 0, false - } - return atomic.LoadUint64(lastArrivalTSInterface.(*uint64)), true -} - -func (o *pdOracle) updateTS(ctx context.Context, interval time.Duration) { - ticker := time.NewTicker(interval) - defer ticker.Stop() - for { - select { - case <-ticker.C: - // Update the timestamp for each txnScope - o.lastTSMap.Range(func(key, _ interface{}) bool { - txnScope := key.(string) - ts, err := o.getTimestamp(ctx, txnScope) - if err != nil { - logutil.Logger(ctx).Error("updateTS error", zap.String("txnScope", txnScope), zap.Error(err)) - return true - } - o.setLastTS(ts, txnScope) - return true - }) - case <-o.quit: - return - } - } -} - -// UntilExpired implement oracle.Oracle interface. -func (o *pdOracle) UntilExpired(lockTS uint64, TTL uint64, opt *oracle.Option) int64 { - lastTS, ok := o.getLastTS(opt.TxnScope) - if !ok { - return 0 - } - return oracle.ExtractPhysical(lockTS) + int64(TTL) - oracle.ExtractPhysical(lastTS) -} - -func (o *pdOracle) Close() { - close(o.quit) -} - -// A future that resolves immediately to a low resolution timestamp. -type lowResolutionTsFuture struct { - ts uint64 - err error -} - -// Wait implements the oracle.Future interface. -func (f lowResolutionTsFuture) Wait() (uint64, error) { - return f.ts, f.err -} - -// GetLowResolutionTimestamp gets a new increasing time. -func (o *pdOracle) GetLowResolutionTimestamp(ctx context.Context, opt *oracle.Option) (uint64, error) { - lastTS, ok := o.getLastTS(opt.TxnScope) - if !ok { - return 0, errors.Errorf("get low resolution timestamp fail, invalid txnScope = %s", opt.TxnScope) - } - return lastTS, nil -} - -func (o *pdOracle) GetLowResolutionTimestampAsync(ctx context.Context, opt *oracle.Option) oracle.Future { - lastTS, ok := o.getLastTS(opt.TxnScope) - if !ok { - return lowResolutionTsFuture{ - ts: 0, - err: errors.Errorf("get low resolution timestamp async fail, invalid txnScope = %s", opt.TxnScope), - } - } - return lowResolutionTsFuture{ - ts: lastTS, - err: nil, - } -} - -func (o *pdOracle) getStaleTimestamp(txnScope string, prevSecond uint64) (uint64, error) { - ts, ok := o.getLastTS(txnScope) - if !ok { - return 0, errors.Errorf("get stale timestamp fail, txnScope: %s", txnScope) - } - arrivalTS, ok := o.getLastArrivalTS(txnScope) - if !ok { - return 0, errors.Errorf("get stale arrival timestamp fail, txnScope: %s", txnScope) - } - arrivalTime := oracle.GetTimeFromTS(arrivalTS) - physicalTime := oracle.GetTimeFromTS(ts) - if uint64(physicalTime.Unix()) <= prevSecond { - return 0, errors.Errorf("invalid prevSecond %v", prevSecond) - } - - staleTime := physicalTime.Add(-arrivalTime.Sub(time.Now().Add(-time.Duration(prevSecond) * time.Second))) - - return oracle.GoTimeToTS(staleTime), nil -} - -// GetStaleTimestamp generate a TSO which represents for the TSO prevSecond secs ago. -func (o *pdOracle) GetStaleTimestamp(ctx context.Context, txnScope string, prevSecond uint64) (ts uint64, err error) { - ts, err = o.getStaleTimestamp(txnScope, prevSecond) - if err != nil { - if !strings.HasPrefix(err.Error(), "invalid prevSecond") { - // If any error happened, we will try to fetch tso and set it as last ts. - _, tErr := o.GetTimestamp(ctx, &oracle.Option{TxnScope: txnScope}) - if tErr != nil { - return 0, errors.Trace(tErr) - } - } - return 0, errors.Trace(err) - } - return ts, nil -} diff --git a/store/tikv/oracle/oracles/pd_test.go b/store/tikv/oracle/oracles/pd_test.go deleted file mode 100644 index 2894a782e505f..0000000000000 --- a/store/tikv/oracle/oracles/pd_test.go +++ /dev/null @@ -1,88 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package oracles_test - -import ( - "context" - "math" - "testing" - "time" - - . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/oracle/oracles" -) - -func TestT(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&testPDSuite{}) - -type testPDSuite struct{} - -func (s *testPDSuite) TestPDOracle_UntilExpired(c *C) { - lockAfter, lockExp := 10, 15 - o := oracles.NewEmptyPDOracle() - start := time.Now() - oracles.SetEmptyPDOracleLastTs(o, oracle.GoTimeToTS(start)) - lockTs := oracle.GoTimeToTS((start.Add(time.Duration(lockAfter) * time.Millisecond))) + 1 - waitTs := o.UntilExpired(lockTs, uint64(lockExp), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) - c.Assert(waitTs, Equals, int64(lockAfter+lockExp), Commentf("waitTs shoulb be %d but got %d", int64(lockAfter+lockExp), waitTs)) -} - -func (s *testPDSuite) TestPdOracle_GetStaleTimestamp(c *C) { - o := oracles.NewEmptyPDOracle() - start := time.Now() - oracles.SetEmptyPDOracleLastTs(o, oracle.GoTimeToTS(start)) - ts, err := o.GetStaleTimestamp(context.Background(), oracle.GlobalTxnScope, 10) - c.Assert(err, IsNil) - - duration := start.Sub(oracle.GetTimeFromTS(ts)) - c.Assert(duration <= 12*time.Second && duration >= 8*time.Second, IsTrue, Commentf("stable TS have accuracy err, expect: %d +-2, obtain: %d", 10, duration)) - - _, err = o.GetStaleTimestamp(context.Background(), oracle.GlobalTxnScope, 1e12) - c.Assert(err, NotNil, Commentf("expect exceed err but get nil")) - - testcases := []struct { - name string - preSec uint64 - expectErr string - }{ - { - name: "normal case", - preSec: 6, - expectErr: "", - }, - { - name: "preSec too large", - preSec: math.MaxUint64, - expectErr: ".*invalid prevSecond.*", - }, - } - - for _, testcase := range testcases { - comment := Commentf("%s", testcase.name) - start = time.Now() - oracles.SetEmptyPDOracleLastTs(o, oracle.GoTimeToTS(start)) - ts, err = o.GetStaleTimestamp(context.Background(), oracle.GlobalTxnScope, testcase.preSec) - if testcase.expectErr == "" { - c.Assert(err, IsNil, comment) - duration = start.Sub(oracle.GetTimeFromTS(ts)) - c.Assert(duration <= time.Duration(testcase.preSec+2)*time.Second && duration >= time.Duration(testcase.preSec-2)*time.Second, IsTrue, Commentf("%s: stable TS have accuracy err, expect: %d +-2, obtain: %d", comment.CheckCommentString(), testcase.preSec, duration)) - } else { - c.Assert(err, ErrorMatches, testcase.expectErr, comment) - } - } -} diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go deleted file mode 100644 index 5ebac3651d912..0000000000000 --- a/store/tikv/pessimistic.go +++ /dev/null @@ -1,282 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "encoding/hex" - "math/rand" - "strings" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/client" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -// Used for pessimistic lock wait time -// these two constants are special for lock protocol with tikv -// 0 means always wait, -1 means nowait, others meaning lock wait in milliseconds -var ( - LockAlwaysWait = int64(0) - LockNoWait = int64(-1) -) - -type actionPessimisticLock struct { - *kv.LockCtx -} -type actionPessimisticRollback struct{} - -var ( - _ twoPhaseCommitAction = actionPessimisticLock{} - _ twoPhaseCommitAction = actionPessimisticRollback{} -) - -func (actionPessimisticLock) String() string { - return "pessimistic_lock" -} - -func (actionPessimisticLock) tiKVTxnRegionsNumHistogram() prometheus.Observer { - return metrics.TxnRegionsNumHistogramPessimisticLock -} - -func (actionPessimisticRollback) String() string { - return "pessimistic_rollback" -} - -func (actionPessimisticRollback) tiKVTxnRegionsNumHistogram() prometheus.Observer { - return metrics.TxnRegionsNumHistogramPessimisticRollback -} - -func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { - m := batch.mutations - mutations := make([]*kvrpcpb.Mutation, m.Len()) - for i := 0; i < m.Len(); i++ { - mut := &kvrpcpb.Mutation{ - Op: kvrpcpb.Op_PessimisticLock, - Key: m.GetKey(i), - } - if c.txn.us.HasPresumeKeyNotExists(m.GetKey(i)) || (c.doingAmend && m.GetOp(i) == kvrpcpb.Op_Insert) { - mut.Assertion = kvrpcpb.Assertion_NotExist - } - mutations[i] = mut - } - elapsed := uint64(time.Since(c.txn.startTime) / time.Millisecond) - ttl := elapsed + atomic.LoadUint64(&ManagedLockTTL) - if _, err := util.EvalFailpoint("shortPessimisticLockTTL"); err == nil { - ttl = 1 - keys := make([]string, 0, len(mutations)) - for _, m := range mutations { - keys = append(keys, hex.EncodeToString(m.Key)) - } - logutil.BgLogger().Info("[failpoint] injected lock ttl = 1 on pessimistic lock", - zap.Uint64("txnStartTS", c.startTS), zap.Strings("keys", keys)) - } - req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticLock, &kvrpcpb.PessimisticLockRequest{ - Mutations: mutations, - PrimaryLock: c.primary(), - StartVersion: c.startTS, - ForUpdateTs: c.forUpdateTS, - LockTtl: ttl, - IsFirstLock: c.isFirstLock, - WaitTimeout: action.LockWaitTime, - ReturnValues: action.ReturnValues, - MinCommitTs: c.forUpdateTS + 1, - }, kvrpcpb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: action.LockCtx.ResourceGroupTag}) - lockWaitStartTime := action.WaitStartTime - for { - // if lockWaitTime set, refine the request `WaitTimeout` field based on timeout limit - if action.LockWaitTime > 0 { - timeLeft := action.LockWaitTime - (time.Since(lockWaitStartTime)).Milliseconds() - if timeLeft <= 0 { - req.PessimisticLock().WaitTimeout = LockNoWait - } else { - req.PessimisticLock().WaitTimeout = timeLeft - } - } - if _, err := util.EvalFailpoint("PessimisticLockErrWriteConflict"); err == nil { - time.Sleep(300 * time.Millisecond) - return &tikverr.ErrWriteConflict{WriteConflict: nil} - } - startTime := time.Now() - resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort) - if action.LockCtx.Stats != nil { - atomic.AddInt64(&action.LockCtx.Stats.LockRPCTime, int64(time.Since(startTime))) - atomic.AddInt64(&action.LockCtx.Stats.LockRPCCount, 1) - } - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - // For other region error and the fake region error, backoff because - // there's something wrong. - // For the real EpochNotMatch error, don't backoff. - if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - } - same, err := batch.relocate(bo, c.store.regionCache) - if err != nil { - return errors.Trace(err) - } - if same { - continue - } - err = c.pessimisticLockMutations(bo, action.LockCtx, batch.mutations) - return errors.Trace(err) - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - lockResp := resp.Resp.(*kvrpcpb.PessimisticLockResponse) - keyErrs := lockResp.GetErrors() - if len(keyErrs) == 0 { - if action.ReturnValues { - action.ValuesLock.Lock() - for i, mutation := range mutations { - action.Values[string(mutation.Key)] = kv.ReturnedValue{Value: lockResp.Values[i]} - } - action.ValuesLock.Unlock() - } - return nil - } - var locks []*Lock - for _, keyErr := range keyErrs { - // Check already exists error - if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil { - e := &tikverr.ErrKeyExist{AlreadyExist: alreadyExist} - return c.extractKeyExistsErr(e) - } - if deadlock := keyErr.Deadlock; deadlock != nil { - return &tikverr.ErrDeadlock{Deadlock: deadlock} - } - - // Extract lock from key error - lock, err1 := extractLockFromKeyErr(keyErr) - if err1 != nil { - return errors.Trace(err1) - } - locks = append(locks, lock) - } - // Because we already waited on tikv, no need to Backoff here. - // tikv default will wait 3s(also the maximum wait value) when lock error occurs - startTime = time.Now() - msBeforeTxnExpired, _, err := c.store.lockResolver.ResolveLocks(bo, 0, locks) - if err != nil { - return errors.Trace(err) - } - if action.LockCtx.Stats != nil { - atomic.AddInt64(&action.LockCtx.Stats.ResolveLockTime, int64(time.Since(startTime))) - } - - // If msBeforeTxnExpired is not zero, it means there are still locks blocking us acquiring - // the pessimistic lock. We should return acquire fail with nowait set or timeout error if necessary. - if msBeforeTxnExpired > 0 { - if action.LockWaitTime == LockNoWait { - return tikverr.ErrLockAcquireFailAndNoWaitSet - } else if action.LockWaitTime == LockAlwaysWait { - // do nothing but keep wait - } else { - // the lockWaitTime is set, we should return wait timeout if we are still blocked by a lock - if time.Since(lockWaitStartTime).Milliseconds() >= action.LockWaitTime { - return errors.Trace(tikverr.ErrLockWaitTimeout) - } - } - if action.LockCtx.PessimisticLockWaited != nil { - atomic.StoreInt32(action.LockCtx.PessimisticLockWaited, 1) - } - } - - // Handle the killed flag when waiting for the pessimistic lock. - // When a txn runs into LockKeys() and backoff here, it has no chance to call - // executor.Next() and check the killed flag. - if action.Killed != nil { - // Do not reset the killed flag here! - // actionPessimisticLock runs on each region parallelly, we have to consider that - // the error may be dropped. - if atomic.LoadUint32(action.Killed) == 1 { - return errors.Trace(tikverr.ErrQueryInterrupted) - } - } - } -} - -func (actionPessimisticRollback) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { - req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticRollback, &kvrpcpb.PessimisticRollbackRequest{ - StartVersion: c.startTS, - ForUpdateTs: c.forUpdateTS, - Keys: batch.mutations.GetKeys(), - }) - resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort) - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - err = c.pessimisticRollbackMutations(bo, batch.mutations) - return errors.Trace(err) - } - return nil -} - -func (c *twoPhaseCommitter) pessimisticLockMutations(bo *Backoffer, lockCtx *kv.LockCtx, mutations CommitterMutations) error { - if c.sessionID > 0 { - if val, err := util.EvalFailpoint("beforePessimisticLock"); err == nil { - // Pass multiple instructions in one string, delimited by commas, to trigger multiple behaviors, like - // `return("delay,fail")`. Then they will be executed sequentially at once. - if v, ok := val.(string); ok { - for _, action := range strings.Split(v, ",") { - if action == "delay" { - duration := time.Duration(rand.Int63n(int64(time.Second) * 5)) - logutil.Logger(bo.GetCtx()).Info("[failpoint] injected delay at pessimistic lock", - zap.Uint64("txnStartTS", c.startTS), zap.Duration("duration", duration)) - time.Sleep(duration) - } else if action == "fail" { - logutil.Logger(bo.GetCtx()).Info("[failpoint] injected failure at pessimistic lock", - zap.Uint64("txnStartTS", c.startTS)) - return errors.New("injected failure at pessimistic lock") - } - } - } - } - } - return c.doActionOnMutations(bo, actionPessimisticLock{lockCtx}, mutations) -} - -func (c *twoPhaseCommitter) pessimisticRollbackMutations(bo *Backoffer, mutations CommitterMutations) error { - return c.doActionOnMutations(bo, actionPessimisticRollback{}, mutations) -} diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go deleted file mode 100644 index 145257c6966d8..0000000000000 --- a/store/tikv/prewrite.go +++ /dev/null @@ -1,325 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "encoding/hex" - "math" - "sync/atomic" - "time" - - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/config" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -type actionPrewrite struct{ retry bool } - -var _ twoPhaseCommitAction = actionPrewrite{} - -func (actionPrewrite) String() string { - return "prewrite" -} - -func (actionPrewrite) tiKVTxnRegionsNumHistogram() prometheus.Observer { - return metrics.TxnRegionsNumHistogramPrewrite -} - -func (c *twoPhaseCommitter) buildPrewriteRequest(batch batchMutations, txnSize uint64) *tikvrpc.Request { - m := batch.mutations - mutations := make([]*kvrpcpb.Mutation, m.Len()) - isPessimisticLock := make([]bool, m.Len()) - for i := 0; i < m.Len(); i++ { - mutations[i] = &kvrpcpb.Mutation{ - Op: m.GetOp(i), - Key: m.GetKey(i), - Value: m.GetValue(i), - } - isPessimisticLock[i] = m.IsPessimisticLock(i) - } - c.mu.Lock() - minCommitTS := c.minCommitTS - c.mu.Unlock() - if c.forUpdateTS > 0 && c.forUpdateTS >= minCommitTS { - minCommitTS = c.forUpdateTS + 1 - } else if c.startTS >= minCommitTS { - minCommitTS = c.startTS + 1 - } - - if val, err := util.EvalFailpoint("mockZeroCommitTS"); err == nil { - // Should be val.(uint64) but failpoint doesn't support that. - if tmp, ok := val.(int); ok && uint64(tmp) == c.startTS { - minCommitTS = 0 - } - } - - ttl := c.lockTTL - - if c.sessionID > 0 { - if _, err := util.EvalFailpoint("twoPCShortLockTTL"); err == nil { - ttl = 1 - keys := make([]string, 0, len(mutations)) - for _, m := range mutations { - keys = append(keys, hex.EncodeToString(m.Key)) - } - logutil.BgLogger().Info("[failpoint] injected lock ttl = 1 on prewrite", - zap.Uint64("txnStartTS", c.startTS), zap.Strings("keys", keys)) - } - } - - req := &kvrpcpb.PrewriteRequest{ - Mutations: mutations, - PrimaryLock: c.primary(), - StartVersion: c.startTS, - LockTtl: ttl, - IsPessimisticLock: isPessimisticLock, - ForUpdateTs: c.forUpdateTS, - TxnSize: txnSize, - MinCommitTs: minCommitTS, - MaxCommitTs: c.maxCommitTS, - } - - if _, err := util.EvalFailpoint("invalidMaxCommitTS"); err == nil { - if req.MaxCommitTs > 0 { - req.MaxCommitTs = minCommitTS - 1 - } - } - - if c.isAsyncCommit() { - if batch.isPrimary { - req.Secondaries = c.asyncSecondaries() - } - req.UseAsyncCommit = true - } - - if c.isOnePC() { - req.TryOnePc = true - } - - return tikvrpc.NewRequest(tikvrpc.CmdPrewrite, req, kvrpcpb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) -} - -func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) (err error) { - // WARNING: This function only tries to send a single request to a single region, so it don't - // need to unset the `useOnePC` flag when it fails. A special case is that when TiKV returns - // regionErr, it's uncertain if the request will be splitted into multiple and sent to multiple - // regions. It invokes `prewriteMutations` recursively here, and the number of batches will be - // checked there. - - if c.sessionID > 0 { - if batch.isPrimary { - if _, err := util.EvalFailpoint("prewritePrimaryFail"); err == nil { - // Delay to avoid cancelling other normally ongoing prewrite requests. - time.Sleep(time.Millisecond * 50) - logutil.Logger(bo.GetCtx()).Info("[failpoint] injected error on prewriting primary batch", - zap.Uint64("txnStartTS", c.startTS)) - return errors.New("injected error on prewriting primary batch") - } - util.EvalFailpoint("prewritePrimary") // for other failures like sleep or pause - } else { - if _, err := util.EvalFailpoint("prewriteSecondaryFail"); err == nil { - // Delay to avoid cancelling other normally ongoing prewrite requests. - time.Sleep(time.Millisecond * 50) - logutil.Logger(bo.GetCtx()).Info("[failpoint] injected error on prewriting secondary batch", - zap.Uint64("txnStartTS", c.startTS)) - return errors.New("injected error on prewriting secondary batch") - } - util.EvalFailpoint("prewriteSecondary") // for other failures like sleep or pause - } - } - - txnSize := uint64(c.regionTxnSize[batch.region.GetID()]) - // When we retry because of a region miss, we don't know the transaction size. We set the transaction size here - // to MaxUint64 to avoid unexpected "resolve lock lite". - if action.retry { - txnSize = math.MaxUint64 - } - - tBegin := time.Now() - attempts := 0 - - req := c.buildPrewriteRequest(batch, txnSize) - sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) - defer func() { - if err != nil { - // If we fail to receive response for async commit prewrite, it will be undetermined whether this - // transaction has been successfully committed. - // If prewrite has been cancelled, all ongoing prewrite RPCs will become errors, we needn't set undetermined - // errors. - if (c.isAsyncCommit() || c.isOnePC()) && sender.GetRPCError() != nil && atomic.LoadUint32(&c.prewriteCancelled) == 0 { - c.setUndeterminedErr(errors.Trace(sender.GetRPCError())) - } - } - }() - for { - attempts++ - if time.Since(tBegin) > slowRequestThreshold { - logutil.BgLogger().Warn("slow prewrite request", zap.Uint64("startTS", c.startTS), zap.Stringer("region", &batch.region), zap.Int("attempts", attempts)) - tBegin = time.Now() - } - - resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) - // Unexpected error occurs, return it - if err != nil { - return errors.Trace(err) - } - - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - // For other region error and the fake region error, backoff because - // there's something wrong. - // For the real EpochNotMatch error, don't backoff. - if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - } - same, err := batch.relocate(bo, c.store.regionCache) - if err != nil { - return errors.Trace(err) - } - if _, err := util.EvalFailpoint("forceRecursion"); err == nil { - same = false - } - if same { - continue - } - err = c.doActionOnMutations(bo, actionPrewrite{true}, batch.mutations) - return errors.Trace(err) - } - - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - prewriteResp := resp.Resp.(*kvrpcpb.PrewriteResponse) - keyErrs := prewriteResp.GetErrors() - if len(keyErrs) == 0 { - if batch.isPrimary { - // After writing the primary key, if the size of the transaction is larger than 32M, - // start the ttlManager. The ttlManager will be closed in tikvTxn.Commit(). - // In this case 1PC is not expected to be used, but still check it for safety. - if int64(c.txnSize) > config.GetGlobalConfig().TiKVClient.TTLRefreshedTxnSize && - prewriteResp.OnePcCommitTs == 0 { - c.run(c, nil) - } - } - - if c.isOnePC() { - if prewriteResp.OnePcCommitTs == 0 { - if prewriteResp.MinCommitTs != 0 { - return errors.Trace(errors.New("MinCommitTs must be 0 when 1pc falls back to 2pc")) - } - logutil.Logger(bo.GetCtx()).Warn("1pc failed and fallbacks to normal commit procedure", - zap.Uint64("startTS", c.startTS)) - metrics.OnePCTxnCounterFallback.Inc() - c.setOnePC(false) - c.setAsyncCommit(false) - } else { - // For 1PC, there's no racing to access to access `onePCCommmitTS` so it's safe - // not to lock the mutex. - if c.onePCCommitTS != 0 { - logutil.Logger(bo.GetCtx()).Fatal("one pc happened multiple times", - zap.Uint64("startTS", c.startTS)) - } - c.onePCCommitTS = prewriteResp.OnePcCommitTs - } - return nil - } else if prewriteResp.OnePcCommitTs != 0 { - logutil.Logger(bo.GetCtx()).Fatal("tikv committed a non-1pc transaction with 1pc protocol", - zap.Uint64("startTS", c.startTS)) - } - if c.isAsyncCommit() { - // 0 if the min_commit_ts is not ready or any other reason that async - // commit cannot proceed. The client can then fallback to normal way to - // continue committing the transaction if prewrite are all finished. - if prewriteResp.MinCommitTs == 0 { - if c.testingKnobs.noFallBack { - return nil - } - logutil.Logger(bo.GetCtx()).Warn("async commit cannot proceed since the returned minCommitTS is zero, "+ - "fallback to normal path", zap.Uint64("startTS", c.startTS)) - c.setAsyncCommit(false) - } else { - c.mu.Lock() - if prewriteResp.MinCommitTs > c.minCommitTS { - c.minCommitTS = prewriteResp.MinCommitTs - } - c.mu.Unlock() - } - } - return nil - } - var locks []*Lock - for _, keyErr := range keyErrs { - // Check already exists error - if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil { - e := &tikverr.ErrKeyExist{AlreadyExist: alreadyExist} - err = c.extractKeyExistsErr(e) - if err != nil { - atomic.StoreUint32(&c.prewriteFailed, 1) - } - return err - } - - // Extract lock from key error - lock, err1 := extractLockFromKeyErr(keyErr) - if err1 != nil { - atomic.StoreUint32(&c.prewriteFailed, 1) - return errors.Trace(err1) - } - logutil.BgLogger().Info("prewrite encounters lock", - zap.Uint64("session", c.sessionID), - zap.Stringer("lock", lock)) - locks = append(locks, lock) - } - start := time.Now() - msBeforeExpired, err := c.store.lockResolver.resolveLocksForWrite(bo, c.startTS, locks) - if err != nil { - return errors.Trace(err) - } - atomic.AddInt64(&c.getDetail().ResolveLockTime, int64(time.Since(start))) - if msBeforeExpired > 0 { - err = bo.BackoffWithCfgAndMaxSleep(retry.BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks))) - if err != nil { - return errors.Trace(err) - } - } - } -} - -func (c *twoPhaseCommitter) prewriteMutations(bo *Backoffer, mutations CommitterMutations) error { - if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("twoPhaseCommitter.prewriteMutations", opentracing.ChildOf(span.Context())) - defer span1.Finish() - bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) - } - - // `doActionOnMutations` will unset `useOnePC` if the mutations is splitted into multiple batches. - return c.doActionOnMutations(bo, actionPrewrite{}, mutations) -} diff --git a/store/tikv/range_task.go b/store/tikv/range_task.go deleted file mode 100644 index bc9b8fa9999c1..0000000000000 --- a/store/tikv/range_task.go +++ /dev/null @@ -1,294 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "go.uber.org/zap" -) - -const ( - rangeTaskDefaultStatLogInterval = time.Minute * 10 - defaultRegionsPerTask = 128 - - lblCompletedRegions = "completed-regions" - lblFailedRegions = "failed-regions" -) - -// RangeTaskRunner splits a range into many ranges to process concurrently, and convenient to send requests to all -// regions in the range. Because of merging and splitting, it's possible that multiple requests for disjoint ranges are -// sent to the same region. -type RangeTaskRunner struct { - name string - store Storage - concurrency int - handler RangeTaskHandler - statLogInterval time.Duration - regionsPerTask int - - completedRegions int32 - failedRegions int32 -} - -// RangeTaskStat is used to count Regions that completed or failed to do the task. -type RangeTaskStat struct { - CompletedRegions int - FailedRegions int -} - -// RangeTaskHandler is the type of functions that processes a task of a key range. -// The function should calculate Regions that succeeded or failed to the task. -// Returning error from the handler means the error caused the whole task should be stopped. -type RangeTaskHandler = func(ctx context.Context, r kv.KeyRange) (RangeTaskStat, error) - -// NewRangeTaskRunner creates a RangeTaskRunner. -// -// `requestCreator` is the function used to create RPC request according to the given range. -// `responseHandler` is the function to process responses of errors. If `responseHandler` returns error, the whole job -// will be canceled. -func NewRangeTaskRunner( - name string, - store Storage, - concurrency int, - handler RangeTaskHandler, -) *RangeTaskRunner { - return &RangeTaskRunner{ - name: name, - store: store, - concurrency: concurrency, - handler: handler, - statLogInterval: rangeTaskDefaultStatLogInterval, - regionsPerTask: defaultRegionsPerTask, - } -} - -// SetRegionsPerTask sets how many regions is in a divided task. Since regions may split and merge, it's possible that -// a sub task contains not exactly specified number of regions. -func (s *RangeTaskRunner) SetRegionsPerTask(regionsPerTask int) { - if regionsPerTask < 1 { - panic("RangeTaskRunner: regionsPerTask should be at least 1") - } - s.regionsPerTask = regionsPerTask -} - -const locateRegionMaxBackoff = 20000 - -// RunOnRange runs the task on the given range. -// Empty startKey or endKey means unbounded. -func (s *RangeTaskRunner) RunOnRange(ctx context.Context, startKey, endKey []byte) error { - s.completedRegions = 0 - metrics.TiKVRangeTaskStats.WithLabelValues(s.name, lblCompletedRegions).Set(0) - - if len(endKey) != 0 && bytes.Compare(startKey, endKey) >= 0 { - logutil.Logger(ctx).Info("empty range task executed. ignored", - zap.String("name", s.name), - zap.String("startKey", kv.StrKey(startKey)), - zap.String("endKey", kv.StrKey(endKey))) - return nil - } - - logutil.Logger(ctx).Info("range task started", - zap.String("name", s.name), - zap.String("startKey", kv.StrKey(startKey)), - zap.String("endKey", kv.StrKey(endKey)), - zap.Int("concurrency", s.concurrency)) - - // Periodically log the progress - statLogTicker := time.NewTicker(s.statLogInterval) - - ctx, cancel := context.WithCancel(ctx) - taskCh := make(chan *kv.KeyRange, s.concurrency) - var wg sync.WaitGroup - - // Create workers that concurrently process the whole range. - workers := make([]*rangeTaskWorker, 0, s.concurrency) - for i := 0; i < s.concurrency; i++ { - w := s.createWorker(taskCh, &wg) - workers = append(workers, w) - wg.Add(1) - go w.run(ctx, cancel) - } - - startTime := time.Now() - - // Make sure taskCh is closed exactly once - isClosed := false - defer func() { - if !isClosed { - close(taskCh) - wg.Wait() - } - statLogTicker.Stop() - cancel() - metrics.TiKVRangeTaskStats.WithLabelValues(s.name, lblCompletedRegions).Set(0) - }() - - // Iterate all regions and send each region's range as a task to the workers. - key := startKey -Loop: - for { - select { - case <-statLogTicker.C: - logutil.Logger(ctx).Info("range task in progress", - zap.String("name", s.name), - zap.String("startKey", kv.StrKey(startKey)), - zap.String("endKey", kv.StrKey(endKey)), - zap.Int("concurrency", s.concurrency), - zap.Duration("cost time", time.Since(startTime)), - zap.Int("completed regions", s.CompletedRegions())) - default: - } - - bo := retry.NewBackofferWithVars(ctx, locateRegionMaxBackoff, nil) - - rangeEndKey, err := s.store.GetRegionCache().BatchLoadRegionsFromKey(bo, key, s.regionsPerTask) - if err != nil { - logutil.Logger(ctx).Info("range task failed", - zap.String("name", s.name), - zap.String("startKey", kv.StrKey(startKey)), - zap.String("endKey", kv.StrKey(endKey)), - zap.Duration("cost time", time.Since(startTime)), - zap.Error(err)) - return errors.Trace(err) - } - task := &kv.KeyRange{ - StartKey: key, - EndKey: rangeEndKey, - } - - isLast := len(task.EndKey) == 0 || (len(endKey) > 0 && bytes.Compare(task.EndKey, endKey) >= 0) - // Let task.EndKey = min(endKey, loc.EndKey) - if isLast { - task.EndKey = endKey - } - - pushTaskStartTime := time.Now() - - select { - case taskCh <- task: - case <-ctx.Done(): - break Loop - } - metrics.TiKVRangeTaskPushDuration.WithLabelValues(s.name).Observe(time.Since(pushTaskStartTime).Seconds()) - - if isLast { - break - } - - key = task.EndKey - } - - isClosed = true - close(taskCh) - wg.Wait() - for _, w := range workers { - if w.err != nil { - logutil.Logger(ctx).Info("range task failed", - zap.String("name", s.name), - zap.String("startKey", kv.StrKey(startKey)), - zap.String("endKey", kv.StrKey(endKey)), - zap.Duration("cost time", time.Since(startTime)), - zap.Error(w.err)) - return errors.Trace(w.err) - } - } - - logutil.Logger(ctx).Info("range task finished", - zap.String("name", s.name), - zap.String("startKey", kv.StrKey(startKey)), - zap.String("endKey", kv.StrKey(endKey)), - zap.Duration("cost time", time.Since(startTime)), - zap.Int("completed regions", s.CompletedRegions())) - - return nil -} - -// createWorker creates a worker that can process tasks from the given channel. -func (s *RangeTaskRunner) createWorker(taskCh chan *kv.KeyRange, wg *sync.WaitGroup) *rangeTaskWorker { - return &rangeTaskWorker{ - name: s.name, - store: s.store, - handler: s.handler, - taskCh: taskCh, - wg: wg, - - completedRegions: &s.completedRegions, - failedRegions: &s.failedRegions, - } -} - -// CompletedRegions returns how many regions has been sent requests. -func (s *RangeTaskRunner) CompletedRegions() int { - return int(atomic.LoadInt32(&s.completedRegions)) -} - -// FailedRegions returns how many regions has failed to do the task. -func (s *RangeTaskRunner) FailedRegions() int { - return int(atomic.LoadInt32(&s.failedRegions)) -} - -// rangeTaskWorker is used by RangeTaskRunner to process tasks concurrently. -type rangeTaskWorker struct { - name string - store Storage - handler RangeTaskHandler - taskCh chan *kv.KeyRange - wg *sync.WaitGroup - - err error - - completedRegions *int32 - failedRegions *int32 -} - -// run starts the worker. It collects all objects from `w.taskCh` and process them one by one. -func (w *rangeTaskWorker) run(ctx context.Context, cancel context.CancelFunc) { - defer w.wg.Done() - for r := range w.taskCh { - select { - case <-ctx.Done(): - w.err = ctx.Err() - return - default: - } - - stat, err := w.handler(ctx, *r) - - atomic.AddInt32(w.completedRegions, int32(stat.CompletedRegions)) - atomic.AddInt32(w.failedRegions, int32(stat.FailedRegions)) - metrics.TiKVRangeTaskStats.WithLabelValues(w.name, lblCompletedRegions).Add(float64(stat.CompletedRegions)) - metrics.TiKVRangeTaskStats.WithLabelValues(w.name, lblFailedRegions).Add(float64(stat.FailedRegions)) - - if err != nil { - logutil.Logger(ctx).Info("canceling range task because of error", - zap.String("name", w.name), - zap.String("startKey", kv.StrKey(r.StartKey)), - zap.String("endKey", kv.StrKey(r.EndKey)), - zap.Error(err)) - w.err = err - cancel() - break - } - } -} diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go deleted file mode 100644 index 4627e9b02ac63..0000000000000 --- a/store/tikv/rawkv.go +++ /dev/null @@ -1,657 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/config" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - pd "github.com/tikv/pd/client" -) - -var ( - // MaxRawKVScanLimit is the maximum scan limit for rawkv Scan. - MaxRawKVScanLimit = 10240 - // ErrMaxScanLimitExceeded is returned when the limit for rawkv Scan is to large. - ErrMaxScanLimitExceeded = errors.New("limit should be less than MaxRawKVScanLimit") -) - -const ( - // rawBatchPutSize is the maximum size limit for rawkv each batch put request. - rawBatchPutSize = 16 * 1024 - // rawBatchPairCount is the maximum limit for rawkv each batch get/delete request. - rawBatchPairCount = 512 -) - -// RawKVClient is a client of TiKV server which is used as a key-value storage, -// only GET/PUT/DELETE commands are supported. -type RawKVClient struct { - clusterID uint64 - regionCache *locate.RegionCache - pdClient pd.Client - rpcClient Client -} - -// NewRawKVClient creates a client with PD cluster addrs. -func NewRawKVClient(pdAddrs []string, security config.Security, opts ...pd.ClientOption) (*RawKVClient, error) { - pdCli, err := pd.NewClient(pdAddrs, pd.SecurityOption{ - CAPath: security.ClusterSSLCA, - CertPath: security.ClusterSSLCert, - KeyPath: security.ClusterSSLKey, - }, opts...) - if err != nil { - return nil, errors.Trace(err) - } - return &RawKVClient{ - clusterID: pdCli.GetClusterID(context.TODO()), - regionCache: locate.NewRegionCache(pdCli), - pdClient: pdCli, - rpcClient: client.NewRPCClient(security), - }, nil -} - -// Close closes the client. -func (c *RawKVClient) Close() error { - if c.pdClient != nil { - c.pdClient.Close() - } - if c.regionCache != nil { - c.regionCache.Close() - } - if c.rpcClient == nil { - return nil - } - return c.rpcClient.Close() -} - -// ClusterID returns the TiKV cluster ID. -func (c *RawKVClient) ClusterID() uint64 { - return c.clusterID -} - -// Get queries value with the key. When the key does not exist, it returns `nil, nil`. -func (c *RawKVClient) Get(key []byte) ([]byte, error) { - start := time.Now() - defer func() { metrics.RawkvCmdHistogramWithGet.Observe(time.Since(start).Seconds()) }() - - req := tikvrpc.NewRequest(tikvrpc.CmdRawGet, &kvrpcpb.RawGetRequest{Key: key}) - resp, _, err := c.sendReq(key, req, false) - if err != nil { - return nil, errors.Trace(err) - } - if resp.Resp == nil { - return nil, errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawGetResponse) - if cmdResp.GetError() != "" { - return nil, errors.New(cmdResp.GetError()) - } - if len(cmdResp.Value) == 0 { - return nil, nil - } - return cmdResp.Value, nil -} - -const rawkvMaxBackoff = 20000 - -// BatchGet queries values with the keys. -func (c *RawKVClient) BatchGet(keys [][]byte) ([][]byte, error) { - start := time.Now() - defer func() { - metrics.RawkvCmdHistogramWithBatchGet.Observe(time.Since(start).Seconds()) - }() - - bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) - resp, err := c.sendBatchReq(bo, keys, tikvrpc.CmdRawBatchGet) - if err != nil { - return nil, errors.Trace(err) - } - - if resp.Resp == nil { - return nil, errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawBatchGetResponse) - - keyToValue := make(map[string][]byte, len(keys)) - for _, pair := range cmdResp.Pairs { - keyToValue[string(pair.Key)] = pair.Value - } - - values := make([][]byte, len(keys)) - for i, key := range keys { - values[i] = keyToValue[string(key)] - } - return values, nil -} - -// Put stores a key-value pair to TiKV. -func (c *RawKVClient) Put(key, value []byte) error { - start := time.Now() - defer func() { metrics.RawkvCmdHistogramWithBatchPut.Observe(time.Since(start).Seconds()) }() - metrics.RawkvSizeHistogramWithKey.Observe(float64(len(key))) - metrics.RawkvSizeHistogramWithValue.Observe(float64(len(value))) - - if len(value) == 0 { - return errors.New("empty value is not supported") - } - - req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ - Key: key, - Value: value, - }) - resp, _, err := c.sendReq(key, req, false) - if err != nil { - return errors.Trace(err) - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawPutResponse) - if cmdResp.GetError() != "" { - return errors.New(cmdResp.GetError()) - } - return nil -} - -// BatchPut stores key-value pairs to TiKV. -func (c *RawKVClient) BatchPut(keys, values [][]byte) error { - start := time.Now() - defer func() { - metrics.RawkvCmdHistogramWithBatchPut.Observe(time.Since(start).Seconds()) - }() - - if len(keys) != len(values) { - return errors.New("the len of keys is not equal to the len of values") - } - for _, value := range values { - if len(value) == 0 { - return errors.New("empty value is not supported") - } - } - bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) - err := c.sendBatchPut(bo, keys, values) - return errors.Trace(err) -} - -// Delete deletes a key-value pair from TiKV. -func (c *RawKVClient) Delete(key []byte) error { - start := time.Now() - defer func() { metrics.RawkvCmdHistogramWithDelete.Observe(time.Since(start).Seconds()) }() - - req := tikvrpc.NewRequest(tikvrpc.CmdRawDelete, &kvrpcpb.RawDeleteRequest{ - Key: key, - }) - resp, _, err := c.sendReq(key, req, false) - if err != nil { - return errors.Trace(err) - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawDeleteResponse) - if cmdResp.GetError() != "" { - return errors.New(cmdResp.GetError()) - } - return nil -} - -// BatchDelete deletes key-value pairs from TiKV -func (c *RawKVClient) BatchDelete(keys [][]byte) error { - start := time.Now() - defer func() { - metrics.RawkvCmdHistogramWithBatchDelete.Observe(time.Since(start).Seconds()) - }() - - bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) - resp, err := c.sendBatchReq(bo, keys, tikvrpc.CmdRawBatchDelete) - if err != nil { - return errors.Trace(err) - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawBatchDeleteResponse) - if cmdResp.GetError() != "" { - return errors.New(cmdResp.GetError()) - } - return nil -} - -// DeleteRange deletes all key-value pairs in a range from TiKV -func (c *RawKVClient) DeleteRange(startKey []byte, endKey []byte) error { - start := time.Now() - var err error - defer func() { - var label = "delete_range" - if err != nil { - label += "_error" - } - metrics.TiKVRawkvCmdHistogram.WithLabelValues(label).Observe(time.Since(start).Seconds()) - }() - - // Process each affected region respectively - for !bytes.Equal(startKey, endKey) { - var resp *tikvrpc.Response - var actualEndKey []byte - resp, actualEndKey, err = c.sendDeleteRangeReq(startKey, endKey) - if err != nil { - return errors.Trace(err) - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawDeleteRangeResponse) - if cmdResp.GetError() != "" { - return errors.New(cmdResp.GetError()) - } - startKey = actualEndKey - } - - return nil -} - -// Scan queries continuous kv pairs in range [startKey, endKey), up to limit pairs. -// If endKey is empty, it means unbounded. -// If you want to exclude the startKey or include the endKey, push a '\0' to the key. For example, to scan -// (startKey, endKey], you can write: -// `Scan(push(startKey, '\0'), push(endKey, '\0'), limit)`. -func (c *RawKVClient) Scan(startKey, endKey []byte, limit int) (keys [][]byte, values [][]byte, err error) { - start := time.Now() - defer func() { metrics.RawkvCmdHistogramWithRawScan.Observe(time.Since(start).Seconds()) }() - - if limit > MaxRawKVScanLimit { - return nil, nil, errors.Trace(ErrMaxScanLimitExceeded) - } - - for len(keys) < limit && (len(endKey) == 0 || bytes.Compare(startKey, endKey) < 0) { - req := tikvrpc.NewRequest(tikvrpc.CmdRawScan, &kvrpcpb.RawScanRequest{ - StartKey: startKey, - EndKey: endKey, - Limit: uint32(limit - len(keys)), - }) - resp, loc, err := c.sendReq(startKey, req, false) - if err != nil { - return nil, nil, errors.Trace(err) - } - if resp.Resp == nil { - return nil, nil, errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawScanResponse) - for _, pair := range cmdResp.Kvs { - keys = append(keys, pair.Key) - values = append(values, pair.Value) - } - startKey = loc.EndKey - if len(startKey) == 0 { - break - } - } - return -} - -// ReverseScan queries continuous kv pairs in range [endKey, startKey), up to limit pairs. -// Direction is different from Scan, upper to lower. -// If endKey is empty, it means unbounded. -// If you want to include the startKey or exclude the endKey, push a '\0' to the key. For example, to scan -// (endKey, startKey], you can write: -// `ReverseScan(push(startKey, '\0'), push(endKey, '\0'), limit)`. -// It doesn't support Scanning from "", because locating the last Region is not yet implemented. -func (c *RawKVClient) ReverseScan(startKey, endKey []byte, limit int) (keys [][]byte, values [][]byte, err error) { - start := time.Now() - defer func() { - metrics.RawkvCmdHistogramWithRawReversScan.Observe(time.Since(start).Seconds()) - }() - - if limit > MaxRawKVScanLimit { - return nil, nil, errors.Trace(ErrMaxScanLimitExceeded) - } - - for len(keys) < limit && bytes.Compare(startKey, endKey) > 0 { - req := tikvrpc.NewRequest(tikvrpc.CmdRawScan, &kvrpcpb.RawScanRequest{ - StartKey: startKey, - EndKey: endKey, - Limit: uint32(limit - len(keys)), - Reverse: true, - }) - resp, loc, err := c.sendReq(startKey, req, true) - if err != nil { - return nil, nil, errors.Trace(err) - } - if resp.Resp == nil { - return nil, nil, errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawScanResponse) - for _, pair := range cmdResp.Kvs { - keys = append(keys, pair.Key) - values = append(values, pair.Value) - } - startKey = loc.StartKey - if len(startKey) == 0 { - break - } - } - return -} - -func (c *RawKVClient) sendReq(key []byte, req *tikvrpc.Request, reverse bool) (*tikvrpc.Response, *locate.KeyLocation, error) { - bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) - sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient) - for { - var loc *locate.KeyLocation - var err error - if reverse { - loc, err = c.regionCache.LocateEndKey(bo, key) - } else { - loc, err = c.regionCache.LocateKey(bo, key) - } - if err != nil { - return nil, nil, errors.Trace(err) - } - resp, err := sender.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) - if err != nil { - return nil, nil, errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return nil, nil, errors.Trace(err) - } - if regionErr != nil { - err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return nil, nil, errors.Trace(err) - } - continue - } - return resp, loc, nil - } -} - -func (c *RawKVClient) sendBatchReq(bo *Backoffer, keys [][]byte, cmdType tikvrpc.CmdType) (*tikvrpc.Response, error) { // split the keys - groups, _, err := c.regionCache.GroupKeysByRegion(bo, keys, nil) - if err != nil { - return nil, errors.Trace(err) - } - - var batches []batch - for regionID, groupKeys := range groups { - batches = appendKeyBatches(batches, regionID, groupKeys, rawBatchPairCount) - } - bo, cancel := bo.Fork() - ches := make(chan singleBatchResp, len(batches)) - for _, batch := range batches { - batch1 := batch - go func() { - singleBatchBackoffer, singleBatchCancel := bo.Fork() - defer singleBatchCancel() - ches <- c.doBatchReq(singleBatchBackoffer, batch1, cmdType) - }() - } - - var firstError error - var resp *tikvrpc.Response - switch cmdType { - case tikvrpc.CmdRawBatchGet: - resp = &tikvrpc.Response{Resp: &kvrpcpb.RawBatchGetResponse{}} - case tikvrpc.CmdRawBatchDelete: - resp = &tikvrpc.Response{Resp: &kvrpcpb.RawBatchDeleteResponse{}} - } - for i := 0; i < len(batches); i++ { - singleResp, ok := <-ches - if ok { - if singleResp.err != nil { - cancel() - if firstError == nil { - firstError = singleResp.err - } - } else if cmdType == tikvrpc.CmdRawBatchGet { - cmdResp := singleResp.resp.Resp.(*kvrpcpb.RawBatchGetResponse) - resp.Resp.(*kvrpcpb.RawBatchGetResponse).Pairs = append(resp.Resp.(*kvrpcpb.RawBatchGetResponse).Pairs, cmdResp.Pairs...) - } - } - } - - return resp, firstError -} - -func (c *RawKVClient) doBatchReq(bo *Backoffer, batch batch, cmdType tikvrpc.CmdType) singleBatchResp { - var req *tikvrpc.Request - switch cmdType { - case tikvrpc.CmdRawBatchGet: - req = tikvrpc.NewRequest(cmdType, &kvrpcpb.RawBatchGetRequest{ - Keys: batch.keys, - }) - case tikvrpc.CmdRawBatchDelete: - req = tikvrpc.NewRequest(cmdType, &kvrpcpb.RawBatchDeleteRequest{ - Keys: batch.keys, - }) - } - - sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient) - resp, err := sender.SendReq(bo, req, batch.regionID, client.ReadTimeoutShort) - - batchResp := singleBatchResp{} - if err != nil { - batchResp.err = errors.Trace(err) - return batchResp - } - regionErr, err := resp.GetRegionError() - if err != nil { - batchResp.err = errors.Trace(err) - return batchResp - } - if regionErr != nil { - err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - batchResp.err = errors.Trace(err) - return batchResp - } - resp, err = c.sendBatchReq(bo, batch.keys, cmdType) - batchResp.resp = resp - batchResp.err = err - return batchResp - } - - switch cmdType { - case tikvrpc.CmdRawBatchGet: - batchResp.resp = resp - case tikvrpc.CmdRawBatchDelete: - if resp.Resp == nil { - batchResp.err = errors.Trace(tikverr.ErrBodyMissing) - return batchResp - } - cmdResp := resp.Resp.(*kvrpcpb.RawBatchDeleteResponse) - if cmdResp.GetError() != "" { - batchResp.err = errors.New(cmdResp.GetError()) - return batchResp - } - batchResp.resp = resp - } - return batchResp -} - -// sendDeleteRangeReq sends a raw delete range request and returns the response and the actual endKey. -// If the given range spans over more than one regions, the actual endKey is the end of the first region. -// We can't use sendReq directly, because we need to know the end of the region before we send the request -// TODO: Is there any better way to avoid duplicating code with func `sendReq` ? -func (c *RawKVClient) sendDeleteRangeReq(startKey []byte, endKey []byte) (*tikvrpc.Response, []byte, error) { - bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) - sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient) - for { - loc, err := c.regionCache.LocateKey(bo, startKey) - if err != nil { - return nil, nil, errors.Trace(err) - } - - actualEndKey := endKey - if len(loc.EndKey) > 0 && bytes.Compare(loc.EndKey, endKey) < 0 { - actualEndKey = loc.EndKey - } - - req := tikvrpc.NewRequest(tikvrpc.CmdRawDeleteRange, &kvrpcpb.RawDeleteRangeRequest{ - StartKey: startKey, - EndKey: actualEndKey, - }) - - resp, err := sender.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) - if err != nil { - return nil, nil, errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return nil, nil, errors.Trace(err) - } - if regionErr != nil { - err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return nil, nil, errors.Trace(err) - } - continue - } - return resp, actualEndKey, nil - } -} - -func (c *RawKVClient) sendBatchPut(bo *Backoffer, keys, values [][]byte) error { - keyToValue := make(map[string][]byte, len(keys)) - for i, key := range keys { - keyToValue[string(key)] = values[i] - } - groups, _, err := c.regionCache.GroupKeysByRegion(bo, keys, nil) - if err != nil { - return errors.Trace(err) - } - var batches []batch - // split the keys by size and RegionVerID - for regionID, groupKeys := range groups { - batches = appendBatches(batches, regionID, groupKeys, keyToValue, rawBatchPutSize) - } - bo, cancel := bo.Fork() - ch := make(chan error, len(batches)) - for _, batch := range batches { - batch1 := batch - go func() { - singleBatchBackoffer, singleBatchCancel := bo.Fork() - defer singleBatchCancel() - ch <- c.doBatchPut(singleBatchBackoffer, batch1) - }() - } - - for i := 0; i < len(batches); i++ { - if e := <-ch; e != nil { - cancel() - // catch the first error - if err == nil { - err = e - } - } - } - return errors.Trace(err) -} - -func appendKeyBatches(batches []batch, regionID locate.RegionVerID, groupKeys [][]byte, limit int) []batch { - var keys [][]byte - for start, count := 0, 0; start < len(groupKeys); start++ { - if count > limit { - batches = append(batches, batch{regionID: regionID, keys: keys}) - keys = make([][]byte, 0, limit) - count = 0 - } - keys = append(keys, groupKeys[start]) - count++ - } - if len(keys) != 0 { - batches = append(batches, batch{regionID: regionID, keys: keys}) - } - return batches -} - -func appendBatches(batches []batch, regionID locate.RegionVerID, groupKeys [][]byte, keyToValue map[string][]byte, limit int) []batch { - var start, size int - var keys, values [][]byte - for start = 0; start < len(groupKeys); start++ { - if size >= limit { - batches = append(batches, batch{regionID: regionID, keys: keys, values: values}) - keys = make([][]byte, 0) - values = make([][]byte, 0) - size = 0 - } - key := groupKeys[start] - value := keyToValue[string(key)] - keys = append(keys, key) - values = append(values, value) - size += len(key) - size += len(value) - } - if len(keys) != 0 { - batches = append(batches, batch{regionID: regionID, keys: keys, values: values}) - } - return batches -} - -func (c *RawKVClient) doBatchPut(bo *Backoffer, batch batch) error { - kvPair := make([]*kvrpcpb.KvPair, 0, len(batch.keys)) - for i, key := range batch.keys { - kvPair = append(kvPair, &kvrpcpb.KvPair{Key: key, Value: batch.values[i]}) - } - - req := tikvrpc.NewRequest(tikvrpc.CmdRawBatchPut, &kvrpcpb.RawBatchPutRequest{Pairs: kvPair}) - - sender := locate.NewRegionRequestSender(c.regionCache, c.rpcClient) - resp, err := sender.SendReq(bo, req, batch.regionID, client.ReadTimeoutShort) - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - // recursive call - return c.sendBatchPut(bo, batch.keys, batch.values) - } - - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdResp := resp.Resp.(*kvrpcpb.RawBatchPutResponse) - if cmdResp.GetError() != "" { - return errors.New(cmdResp.GetError()) - } - return nil -} - -type batch struct { - regionID locate.RegionVerID - keys [][]byte - values [][]byte -} - -type singleBatchResp struct { - resp *tikvrpc.Response - err error -} diff --git a/store/tikv/rawkv_test.go b/store/tikv/rawkv_test.go deleted file mode 100644 index 77d8dce25ce9a..0000000000000 --- a/store/tikv/rawkv_test.go +++ /dev/null @@ -1,169 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "context" - "fmt" - - . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" - "github.com/pingcap/tidb/store/tikv/retry" -) - -type testRawkvSuite struct { - OneByOneSuite - cluster *mocktikv.Cluster - store1 uint64 // store1 is leader - store2 uint64 // store2 is follower - peer1 uint64 // peer1 is leader - peer2 uint64 // peer2 is follower - region1 uint64 - bo *retry.Backoffer -} - -var _ = Suite(&testRawkvSuite{}) - -func (s *testRawkvSuite) SetUpTest(c *C) { - s.cluster = mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) - storeIDs, peerIDs, regionID, _ := mocktikv.BootstrapWithMultiStores(s.cluster, 2) - s.region1 = regionID - s.store1 = storeIDs[0] - s.store2 = storeIDs[1] - s.peer1 = peerIDs[0] - s.peer2 = peerIDs[1] - s.bo = retry.NewBackofferWithVars(context.Background(), 5000, nil) -} - -func (s *testRawkvSuite) storeAddr(id uint64) string { - return fmt.Sprintf("store%d", id) -} - -func (s *testRawkvSuite) TestReplaceAddrWithNewStore(c *C) { - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - - client := &RawKVClient{ - clusterID: 0, - regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), - } - defer client.Close() - testKey := []byte("test_key") - testValue := []byte("test_value") - err := client.Put(testKey, testValue) - c.Assert(err, IsNil) - - // make store2 using store1's addr and store1 offline - store1Addr := s.storeAddr(s.store1) - s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) - s.cluster.UpdateStoreAddr(s.store2, store1Addr) - s.cluster.RemoveStore(s.store1) - s.cluster.ChangeLeader(s.region1, s.peer2) - s.cluster.RemovePeer(s.region1, s.peer1) - - getVal, err := client.Get(testKey) - - c.Assert(err, IsNil) - c.Assert(getVal, BytesEquals, testValue) -} - -func (s *testRawkvSuite) TestUpdateStoreAddr(c *C) { - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - - client := &RawKVClient{ - clusterID: 0, - regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), - } - defer client.Close() - testKey := []byte("test_key") - testValue := []byte("test_value") - err := client.Put(testKey, testValue) - c.Assert(err, IsNil) - // tikv-server reports `StoreNotMatch` And retry - store1Addr := s.storeAddr(s.store1) - s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) - s.cluster.UpdateStoreAddr(s.store2, store1Addr) - - getVal, err := client.Get(testKey) - - c.Assert(err, IsNil) - c.Assert(getVal, BytesEquals, testValue) -} - -func (s *testRawkvSuite) TestReplaceNewAddrAndOldOfflineImmediately(c *C) { - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - - client := &RawKVClient{ - clusterID: 0, - regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), - } - defer client.Close() - testKey := []byte("test_key") - testValue := []byte("test_value") - err := client.Put(testKey, testValue) - c.Assert(err, IsNil) - - // pre-load store2's address into cache via follower-read. - loc, err := client.regionCache.LocateKey(s.bo, testKey) - c.Assert(err, IsNil) - fctx, err := client.regionCache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, 0) - c.Assert(err, IsNil) - c.Assert(fctx.Store.StoreID(), Equals, s.store2) - c.Assert(fctx.Addr, Equals, "store2") - - // make store2 using store1's addr and store1 offline - store1Addr := s.storeAddr(s.store1) - s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) - s.cluster.UpdateStoreAddr(s.store2, store1Addr) - s.cluster.RemoveStore(s.store1) - s.cluster.ChangeLeader(s.region1, s.peer2) - s.cluster.RemovePeer(s.region1, s.peer1) - - getVal, err := client.Get(testKey) - c.Assert(err, IsNil) - c.Assert(getVal, BytesEquals, testValue) -} - -func (s *testRawkvSuite) TestReplaceStore(c *C) { - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - - client := &RawKVClient{ - clusterID: 0, - regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), - } - defer client.Close() - testKey := []byte("test_key") - testValue := []byte("test_value") - err := client.Put(testKey, testValue) - c.Assert(err, IsNil) - - s.cluster.MarkTombstone(s.store1) - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(s.store1)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.RemovePeer(s.region1, s.peer1) - s.cluster.ChangeLeader(s.region1, peer3) - - err = client.Put(testKey, testValue) - c.Assert(err, IsNil) -} diff --git a/store/tikv/region.go b/store/tikv/region.go deleted file mode 100644 index 906fd2808bd25..0000000000000 --- a/store/tikv/region.go +++ /dev/null @@ -1,140 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "time" - - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/tikv/client" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - pd "github.com/tikv/pd/client" -) - -// RPCContext contains data that is needed to send RPC to a region. -type RPCContext = locate.RPCContext - -// RPCCanceller is rpc send cancelFunc collector. -type RPCCanceller = locate.RPCCanceller - -// RegionVerID is a unique ID that can identify a Region at a specific version. -type RegionVerID = locate.RegionVerID - -// RegionCache caches Regions loaded from PD. -type RegionCache = locate.RegionCache - -// KeyLocation is the region and range that a key is located. -type KeyLocation = locate.KeyLocation - -// RPCCancellerCtxKey is context key attach rpc send cancelFunc collector to ctx. -type RPCCancellerCtxKey = locate.RPCCancellerCtxKey - -// RegionRequestSender sends KV/Cop requests to tikv server. It handles network -// errors and some region errors internally. -// -// Typically, a KV/Cop request is bind to a region, all keys that are involved -// in the request should be located in the region. -// The sending process begins with looking for the address of leader store's -// address of the target region from cache, and the request is then sent to the -// destination tikv server over TCP connection. -// If region is updated, can be caused by leader transfer, region split, region -// merge, or region balance, tikv server may not able to process request and -// send back a RegionError. -// RegionRequestSender takes care of errors that does not relevant to region -// range, such as 'I/O timeout', 'NotLeader', and 'ServerIsBusy'. For other -// errors, since region range have changed, the request may need to split, so we -// simply return the error to caller. -type RegionRequestSender = locate.RegionRequestSender - -// StoreSelectorOption configures storeSelectorOp. -type StoreSelectorOption = locate.StoreSelectorOption - -// RegionRequestRuntimeStats records the runtime stats of send region requests. -type RegionRequestRuntimeStats = locate.RegionRequestRuntimeStats - -// RPCRuntimeStats indicates the RPC request count and consume time. -type RPCRuntimeStats = locate.RPCRuntimeStats - -// CodecPDClient wraps a PD Client to decode the encoded keys in region meta. -type CodecPDClient = locate.CodecPDClient - -// RecordRegionRequestRuntimeStats records request runtime stats. -func RecordRegionRequestRuntimeStats(stats map[tikvrpc.CmdType]*locate.RPCRuntimeStats, cmd tikvrpc.CmdType, d time.Duration) { - locate.RecordRegionRequestRuntimeStats(stats, cmd, d) -} - -// Store contains a kv process's address. -type Store = locate.Store - -// Region presents kv region -type Region = locate.Region - -// EpochNotMatch indicates it's invalidated due to epoch not match -const EpochNotMatch = locate.EpochNotMatch - -// NewRPCanceller creates RPCCanceller with init state. -func NewRPCanceller() *RPCCanceller { - return locate.NewRPCanceller() -} - -// NewRegionVerID creates a region ver id, which used for invalidating regions. -func NewRegionVerID(id, confVer, ver uint64) RegionVerID { - return locate.NewRegionVerID(id, confVer, ver) -} - -// GetStoreTypeByMeta gets store type by store meta pb. -func GetStoreTypeByMeta(store *metapb.Store) tikvrpc.EndpointType { - return tikvrpc.GetStoreTypeByMeta(store) -} - -// NewRegionRequestSender creates a new sender. -func NewRegionRequestSender(regionCache *RegionCache, client client.Client) *RegionRequestSender { - return locate.NewRegionRequestSender(regionCache, client) -} - -// LoadShuttingDown atomically loads ShuttingDown. -func LoadShuttingDown() uint32 { - return locate.LoadShuttingDown() -} - -// StoreShuttingDown atomically stores ShuttingDown into v. -func StoreShuttingDown(v uint32) { - locate.StoreShuttingDown(v) -} - -// WithMatchLabels indicates selecting stores with matched labels -func WithMatchLabels(labels []*metapb.StoreLabel) StoreSelectorOption { - return locate.WithMatchLabels(labels) -} - -// NewRegionRequestRuntimeStats returns a new RegionRequestRuntimeStats. -func NewRegionRequestRuntimeStats() RegionRequestRuntimeStats { - return locate.NewRegionRequestRuntimeStats() -} - -// SetRegionCacheTTLSec sets regionCacheTTLSec to t. -func SetRegionCacheTTLSec(t int64) { - locate.SetRegionCacheTTLSec(t) -} - -// SetStoreLivenessTimeout sets storeLivenessTimeout to t. -func SetStoreLivenessTimeout(t time.Duration) { - locate.SetStoreLivenessTimeout(t) -} - -// NewRegionCache creates a RegionCache. -func NewRegionCache(pdClient pd.Client) *locate.RegionCache { - return locate.NewRegionCache(pdClient) -} diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go deleted file mode 100644 index 485a179ced6c9..0000000000000 --- a/store/tikv/retry/backoff.go +++ /dev/null @@ -1,289 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package retry - -import ( - "context" - "fmt" - "math" - "strings" - "sync/atomic" - "time" - - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/log" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/util" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" -) - -// Backoffer is a utility for retrying queries. -type Backoffer struct { - ctx context.Context - - fn map[string]backoffFn - maxSleep int - totalSleep int - - vars *kv.Variables - noop bool - - errors []error - configs []*Config - backoffSleepMS map[string]int - backoffTimes map[string]int - parent *Backoffer -} - -type txnStartCtxKeyType struct{} - -// TxnStartKey is a key for transaction start_ts info in context.Context. -var TxnStartKey interface{} = txnStartCtxKeyType{} - -// NewBackoffer (Deprecated) creates a Backoffer with maximum sleep time(in ms). -func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { - return &Backoffer{ - ctx: ctx, - maxSleep: maxSleep, - vars: kv.DefaultVars, - } -} - -// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. -func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { - return NewBackoffer(ctx, maxSleep).withVars(vars) -} - -// NewNoopBackoff create a Backoffer do nothing just return error directly -func NewNoopBackoff(ctx context.Context) *Backoffer { - return &Backoffer{ctx: ctx, noop: true} -} - -// withVars sets the kv.Variables to the Backoffer and return it. -func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer { - if vars != nil { - b.vars = vars - } - // maxSleep is the max sleep time in millisecond. - // When it is multiplied by BackOffWeight, it should not be greater than MaxInt32. - if b.maxSleep > 0 && math.MaxInt32/b.vars.BackOffWeight >= b.maxSleep { - b.maxSleep *= b.vars.BackOffWeight - } - return b -} - -// Backoff sleeps a while base on the Config and records the error message. -// It returns a retryable error if total sleep time exceeds maxSleep. -func (b *Backoffer) Backoff(cfg *Config, err error) error { - if span := opentracing.SpanFromContext(b.ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan(fmt.Sprintf("tikv.backoff.%s", cfg), opentracing.ChildOf(span.Context())) - defer span1.Finish() - opentracing.ContextWithSpan(b.ctx, span1) - } - return b.BackoffWithCfgAndMaxSleep(cfg, -1, err) -} - -// BackoffWithMaxSleepTxnLockFast sleeps a while base on the MaxSleepTxnLock and records the error message -// and never sleep more than maxSleepMs for each sleep. -func (b *Backoffer) BackoffWithMaxSleepTxnLockFast(maxSleepMs int, err error) error { - cfg := BoTxnLockFast - return b.BackoffWithCfgAndMaxSleep(cfg, maxSleepMs, err) -} - -// BackoffWithCfgAndMaxSleep sleeps a while base on the Config and records the error message -// and never sleep more than maxSleepMs for each sleep. -func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err error) error { - if strings.Contains(err.Error(), tikverr.MismatchClusterID) { - logutil.BgLogger().Fatal("critical error", zap.Error(err)) - } - select { - case <-b.ctx.Done(): - return errors.Trace(err) - default: - } - - b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano))) - b.configs = append(b.configs, cfg) - if b.noop || (b.maxSleep > 0 && b.totalSleep >= b.maxSleep) { - errMsg := fmt.Sprintf("%s backoffer.maxSleep %dms is exceeded, errors:", cfg.String(), b.maxSleep) - for i, err := range b.errors { - // Print only last 3 errors for non-DEBUG log levels. - if log.GetLevel() == zapcore.DebugLevel || i >= len(b.errors)-3 { - errMsg += "\n" + err.Error() - } - } - logutil.BgLogger().Warn(errMsg) - // Use the first backoff type to generate a MySQL error. - return b.configs[0].err - } - - // Lazy initialize. - if b.fn == nil { - b.fn = make(map[string]backoffFn) - } - f, ok := b.fn[cfg.name] - if !ok { - f = cfg.createBackoffFn(b.vars) - b.fn[cfg.name] = f - } - realSleep := f(b.ctx, maxSleepMs) - if cfg.metric != nil { - (*cfg.metric).Observe(float64(realSleep) / 1000) - } - b.totalSleep += realSleep - if b.backoffSleepMS == nil { - b.backoffSleepMS = make(map[string]int) - } - b.backoffSleepMS[cfg.name] += realSleep - if b.backoffTimes == nil { - b.backoffTimes = make(map[string]int) - } - b.backoffTimes[cfg.name]++ - - stmtExec := b.ctx.Value(util.ExecDetailsKey) - if stmtExec != nil { - detail := stmtExec.(*util.ExecDetails) - atomic.AddInt64(&detail.BackoffDuration, int64(realSleep)*int64(time.Millisecond)) - atomic.AddInt64(&detail.BackoffCount, 1) - } - - if b.vars != nil && b.vars.Killed != nil { - if atomic.LoadUint32(b.vars.Killed) == 1 { - return tikverr.ErrQueryInterrupted - } - } - - var startTs interface{} - if ts := b.ctx.Value(TxnStartKey); ts != nil { - startTs = ts - } - logutil.Logger(b.ctx).Debug("retry later", - zap.Error(err), - zap.Int("totalSleep", b.totalSleep), - zap.Int("maxSleep", b.maxSleep), - zap.Stringer("type", cfg), - zap.Reflect("txnStartTS", startTs)) - return nil -} - -func (b *Backoffer) String() string { - if b.totalSleep == 0 { - return "" - } - return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.configs) -} - -// Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares -// current Backoffer's context. -func (b *Backoffer) Clone() *Backoffer { - return &Backoffer{ - ctx: b.ctx, - maxSleep: b.maxSleep, - totalSleep: b.totalSleep, - errors: b.errors, - vars: b.vars, - parent: b.parent, - } -} - -// Fork creates a new Backoffer which keeps current Backoffer's sleep time and errors, and holds -// a child context of current Backoffer's context. -func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) { - ctx, cancel := context.WithCancel(b.ctx) - return &Backoffer{ - ctx: ctx, - maxSleep: b.maxSleep, - totalSleep: b.totalSleep, - errors: b.errors, - vars: b.vars, - parent: b, - }, cancel -} - -// GetVars returns the binded vars. -func (b *Backoffer) GetVars() *kv.Variables { - return b.vars -} - -// GetTotalSleep returns total sleep time. -func (b *Backoffer) GetTotalSleep() int { - return b.totalSleep -} - -// GetTypes returns type list of this backoff and all its ancestors. -func (b *Backoffer) GetTypes() []string { - typs := make([]string, 0, len(b.configs)) - for b != nil { - for _, cfg := range b.configs { - typs = append(typs, cfg.String()) - } - b = b.parent - } - return typs -} - -// GetCtx returns the binded context. -func (b *Backoffer) GetCtx() context.Context { - return b.ctx -} - -// SetCtx sets the binded context to ctx. -func (b *Backoffer) SetCtx(ctx context.Context) { - b.ctx = ctx -} - -// GetBackoffTimes returns a map contains backoff time count by type. -func (b *Backoffer) GetBackoffTimes() map[string]int { - return b.backoffTimes -} - -// GetTotalBackoffTimes returns the total backoff times of the backoffer. -func (b *Backoffer) GetTotalBackoffTimes() int { - total := 0 - for _, time := range b.backoffTimes { - total += time - } - return total -} - -// GetBackoffSleepMS returns a map contains backoff sleep time by type. -func (b *Backoffer) GetBackoffSleepMS() map[string]int { - return b.backoffSleepMS -} - -// ErrorsNum returns the number of errors. -func (b *Backoffer) ErrorsNum() int { - return len(b.errors) -} - -// Reset resets the sleep state of the backoffer, so that following backoff -// can sleep shorter. The reason why we don't create a new backoffer is that -// backoffer is similar to context and it records some metrics that we -// want to record for an entire process which is composed of serveral stages. -func (b *Backoffer) Reset() { - b.fn = nil - b.totalSleep = 0 -} - -// ResetMaxSleep resets the sleep state and max sleep limit of the backoffer. -// It's used when switches to the next stage of the process. -func (b *Backoffer) ResetMaxSleep(maxSleep int) { - b.Reset() - b.maxSleep = maxSleep - b.withVars(b.vars) -} diff --git a/store/tikv/retry/backoff_test.go b/store/tikv/retry/backoff_test.go deleted file mode 100644 index a0a566499b10f..0000000000000 --- a/store/tikv/retry/backoff_test.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2019 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package retry - -import ( - "context" - "errors" - - . "github.com/pingcap/check" -) - -type testBackoffSuite struct { -} - -var _ = Suite(&testBackoffSuite{}) - -func (s *testBackoffSuite) TestBackoffWithMax(c *C) { - b := NewBackofferWithVars(context.TODO(), 2000, nil) - err := b.BackoffWithMaxSleepTxnLockFast(30, errors.New("test")) - c.Assert(err, IsNil) - c.Assert(b.totalSleep, Equals, 30) -} diff --git a/store/tikv/retry/config.go b/store/tikv/retry/config.go deleted file mode 100644 index 7222260b1fee2..0000000000000 --- a/store/tikv/retry/config.go +++ /dev/null @@ -1,162 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package retry - -import ( - "context" - "math" - "math/rand" - "strings" - "time" - - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -// Config is the configuration of the Backoff function. -type Config struct { - name string - metric *prometheus.Observer - fnCfg *BackoffFnCfg - err error -} - -// backoffFn is the backoff function which compute the sleep time and do sleep. -type backoffFn func(ctx context.Context, maxSleepMs int) int - -func (c *Config) createBackoffFn(vars *kv.Variables) backoffFn { - if strings.EqualFold(c.name, txnLockFastName) { - return newBackoffFn(vars.BackoffLockFast, c.fnCfg.cap, c.fnCfg.jitter) - } - return newBackoffFn(c.fnCfg.base, c.fnCfg.cap, c.fnCfg.jitter) -} - -// BackoffFnCfg is the configuration for the backoff func which implements exponential backoff with -// optional jitters. -// See http://www.awsarchitectureblog.com/2015/03/backoff.html -type BackoffFnCfg struct { - base int - cap int - jitter int -} - -// NewBackoffFnCfg creates the config for BackoffFn. -func NewBackoffFnCfg(base, cap, jitter int) *BackoffFnCfg { - return &BackoffFnCfg{ - base, - cap, - jitter, - } -} - -// NewConfig creates a new Config for the Backoff operation. -func NewConfig(name string, metric *prometheus.Observer, backoffFnCfg *BackoffFnCfg, err error) *Config { - return &Config{ - name: name, - metric: metric, - fnCfg: backoffFnCfg, - err: err, - } -} - -func (c *Config) String() string { - return c.name -} - -const txnLockFastName = "txnLockFast" - -// Backoff Config variables. -var ( - // TODO: distinguish tikv and tiflash in metrics - BoTiKVRPC = NewConfig("tikvRPC", &metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiKVServerTimeout) - BoTiFlashRPC = NewConfig("tiflashRPC", &metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiFlashServerTimeout) - BoTxnLock = NewConfig("txnLock", &metrics.BackoffHistogramLock, NewBackoffFnCfg(200, 3000, EqualJitter), tikverr.ErrResolveLockTimeout) - BoPDRPC = NewConfig("pdRPC", &metrics.BackoffHistogramPD, NewBackoffFnCfg(500, 3000, EqualJitter), tikverr.NewErrPDServerTimeout("")) - // change base time to 2ms, because it may recover soon. - BoRegionMiss = NewConfig("regionMiss", &metrics.BackoffHistogramRegionMiss, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable) - BoRegionScheduling = NewConfig("regionScheduling", &metrics.BackoffHistogramRegionScheduling, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable) - BoTiKVServerBusy = NewConfig("tikvServerBusy", &metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiKVServerBusy) - BoTiFlashServerBusy = NewConfig("tiflashServerBusy", &metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiFlashServerBusy) - BoTxnNotFound = NewConfig("txnNotFound", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrResolveLockTimeout) - BoStaleCmd = NewConfig("staleCommand", &metrics.BackoffHistogramStaleCmd, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrTiKVStaleCommand) - BoMaxTsNotSynced = NewConfig("maxTsNotSynced", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrTiKVMaxTimestampNotSynced) - BoMaxDataNotReady = NewConfig("dataNotReady", &metrics.BackoffHistogramDataNotReady, NewBackoffFnCfg(100, 2000, NoJitter), tikverr.ErrRegionDataNotReady) - BoMaxRegionNotInitialized = NewConfig("regionNotInitialized", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrRegionNotInitialized) - // TxnLockFast's `base` load from vars.BackoffLockFast when create BackoffFn. - BoTxnLockFast = NewConfig(txnLockFastName, &metrics.BackoffHistogramLockFast, NewBackoffFnCfg(2, 3000, EqualJitter), tikverr.ErrResolveLockTimeout) -) - -const ( - // NoJitter makes the backoff sequence strict exponential. - NoJitter = 1 + iota - // FullJitter applies random factors to strict exponential. - FullJitter - // EqualJitter is also randomized, but prevents very short sleeps. - EqualJitter - // DecorrJitter increases the maximum jitter based on the last random value. - DecorrJitter -) - -// newBackoffFn creates a backoff func which implements exponential backoff with -// optional jitters. -// See http://www.awsarchitectureblog.com/2015/03/backoff.html -func newBackoffFn(base, cap, jitter int) backoffFn { - if base < 2 { - // Top prevent panic in 'rand.Intn'. - base = 2 - } - attempts := 0 - lastSleep := base - return func(ctx context.Context, maxSleepMs int) int { - var sleep int - switch jitter { - case NoJitter: - sleep = expo(base, cap, attempts) - case FullJitter: - v := expo(base, cap, attempts) - sleep = rand.Intn(v) - case EqualJitter: - v := expo(base, cap, attempts) - sleep = v/2 + rand.Intn(v/2) - case DecorrJitter: - sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base)))) - } - logutil.BgLogger().Debug("backoff", - zap.Int("base", base), - zap.Int("sleep", sleep), - zap.Int("attempts", attempts)) - - realSleep := sleep - // when set maxSleepMs >= 0 in `tikv.BackoffWithMaxSleep` will force sleep maxSleepMs milliseconds. - if maxSleepMs >= 0 && realSleep > maxSleepMs { - realSleep = maxSleepMs - } - select { - case <-time.After(time.Duration(realSleep) * time.Millisecond): - attempts++ - lastSleep = sleep - return realSleep - case <-ctx.Done(): - return 0 - } - } -} - -func expo(base, cap, n int) int { - return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n)))) -} diff --git a/store/tikv/safepoint.go b/store/tikv/safepoint.go deleted file mode 100644 index 3aa8b5ece26e5..0000000000000 --- a/store/tikv/safepoint.go +++ /dev/null @@ -1,180 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "context" - "crypto/tls" - "strconv" - "strings" - "sync" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv/logutil" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" - "go.uber.org/zap" -) - -// Safe point constants. -const ( - // This is almost the same as 'tikv_gc_safe_point' in the table 'mysql.tidb', - // save this to pd instead of tikv, because we can't use interface of table - // if the safepoint on tidb is expired. - GcSavedSafePoint = "/tidb/store/gcworker/saved_safe_point" - - GcSafePointCacheInterval = time.Second * 100 - gcCPUTimeInaccuracyBound = time.Second - gcSafePointUpdateInterval = time.Second * 10 - gcSafePointQuickRepeatInterval = time.Second -) - -// SafePointKV is used for a seamingless integration for mockTest and runtime. -type SafePointKV interface { - Put(k string, v string) error - Get(k string) (string, error) - GetWithPrefix(k string) ([]*mvccpb.KeyValue, error) - Close() error -} - -// MockSafePointKV implements SafePointKV at mock test -type MockSafePointKV struct { - store map[string]string - mockLock sync.RWMutex -} - -// NewMockSafePointKV creates an instance of MockSafePointKV -func NewMockSafePointKV() *MockSafePointKV { - return &MockSafePointKV{ - store: make(map[string]string), - } -} - -// Put implements the Put method for SafePointKV -func (w *MockSafePointKV) Put(k string, v string) error { - w.mockLock.Lock() - defer w.mockLock.Unlock() - w.store[k] = v - return nil -} - -// Get implements the Get method for SafePointKV -func (w *MockSafePointKV) Get(k string) (string, error) { - w.mockLock.RLock() - defer w.mockLock.RUnlock() - elem := w.store[k] - return elem, nil -} - -// GetWithPrefix implements the Get method for SafePointKV -func (w *MockSafePointKV) GetWithPrefix(prefix string) ([]*mvccpb.KeyValue, error) { - w.mockLock.RLock() - defer w.mockLock.RUnlock() - kvs := make([]*mvccpb.KeyValue, 0, len(w.store)) - for k, v := range w.store { - if strings.HasPrefix(k, prefix) { - kvs = append(kvs, &mvccpb.KeyValue{Key: []byte(k), Value: []byte(v)}) - } - } - return kvs, nil -} - -// Close implements the Close method for SafePointKV -func (w *MockSafePointKV) Close() error { - return nil -} - -// EtcdSafePointKV implements SafePointKV at runtime -type EtcdSafePointKV struct { - cli *clientv3.Client -} - -// NewEtcdSafePointKV creates an instance of EtcdSafePointKV -func NewEtcdSafePointKV(addrs []string, tlsConfig *tls.Config) (*EtcdSafePointKV, error) { - etcdCli, err := createEtcdKV(addrs, tlsConfig) - if err != nil { - return nil, errors.Trace(err) - } - return &EtcdSafePointKV{cli: etcdCli}, nil -} - -// Put implements the Put method for SafePointKV -func (w *EtcdSafePointKV) Put(k string, v string) error { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) - _, err := w.cli.Put(ctx, k, v) - cancel() - if err != nil { - return errors.Trace(err) - } - return nil -} - -// Get implements the Get method for SafePointKV -func (w *EtcdSafePointKV) Get(k string) (string, error) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) - resp, err := w.cli.Get(ctx, k) - cancel() - if err != nil { - return "", errors.Trace(err) - } - if len(resp.Kvs) > 0 { - return string(resp.Kvs[0].Value), nil - } - return "", nil -} - -// GetWithPrefix implements the GetWithPrefix for SafePointKV -func (w *EtcdSafePointKV) GetWithPrefix(k string) ([]*mvccpb.KeyValue, error) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*15) - resp, err := w.cli.Get(ctx, k, clientv3.WithPrefix()) - cancel() - if err != nil { - return nil, errors.Trace(err) - } - return resp.Kvs, nil -} - -// Close implements the Close for SafePointKV -func (w *EtcdSafePointKV) Close() error { - return errors.Trace(w.cli.Close()) -} - -func saveSafePoint(kv SafePointKV, t uint64) error { - s := strconv.FormatUint(t, 10) - err := kv.Put(GcSavedSafePoint, s) - if err != nil { - logutil.BgLogger().Error("save safepoint failed", zap.Error(err)) - return errors.Trace(err) - } - return nil -} - -func loadSafePoint(kv SafePointKV) (uint64, error) { - str, err := kv.Get(GcSavedSafePoint) - - if err != nil { - return 0, errors.Trace(err) - } - - if str == "" { - return 0, nil - } - - t, err := strconv.ParseUint(str, 10, 64) - if err != nil { - return 0, errors.Trace(err) - } - return t, nil -} diff --git a/store/tikv/scan.go b/store/tikv/scan.go deleted file mode 100644 index 335f760ec09eb..0000000000000 --- a/store/tikv/scan.go +++ /dev/null @@ -1,314 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/store/tikv/client" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "go.uber.org/zap" -) - -// Scanner support tikv scan -type Scanner struct { - snapshot *KVSnapshot - batchSize int - cache []*kvrpcpb.KvPair - idx int - nextStartKey []byte - endKey []byte - - // Use for reverse scan. - nextEndKey []byte - reverse bool - - valid bool - eof bool -} - -func newScanner(snapshot *KVSnapshot, startKey []byte, endKey []byte, batchSize int, reverse bool) (*Scanner, error) { - // It must be > 1. Otherwise scanner won't skipFirst. - if batchSize <= 1 { - batchSize = scanBatchSize - } - scanner := &Scanner{ - snapshot: snapshot, - batchSize: batchSize, - valid: true, - nextStartKey: startKey, - endKey: endKey, - reverse: reverse, - nextEndKey: endKey, - } - err := scanner.Next() - if tikverr.IsErrNotFound(err) { - return scanner, nil - } - return scanner, errors.Trace(err) -} - -// Valid return valid. -func (s *Scanner) Valid() bool { - return s.valid -} - -// Key return key. -func (s *Scanner) Key() []byte { - if s.valid { - return s.cache[s.idx].Key - } - return nil -} - -// Value return value. -func (s *Scanner) Value() []byte { - if s.valid { - return s.cache[s.idx].Value - } - return nil -} - -const scannerNextMaxBackoff = 600000 // 10 minutes - -// Next return next element. -func (s *Scanner) Next() error { - bo := retry.NewBackofferWithVars(context.WithValue(context.Background(), retry.TxnStartKey, s.snapshot.version), scannerNextMaxBackoff, s.snapshot.vars) - if !s.valid { - return errors.New("scanner iterator is invalid") - } - var err error - for { - s.idx++ - if s.idx >= len(s.cache) { - if s.eof { - s.Close() - return nil - } - err = s.getData(bo) - if err != nil { - s.Close() - return errors.Trace(err) - } - if s.idx >= len(s.cache) { - continue - } - } - - current := s.cache[s.idx] - if (!s.reverse && (len(s.endKey) > 0 && kv.CmpKey(current.Key, s.endKey) >= 0)) || - (s.reverse && len(s.nextStartKey) > 0 && kv.CmpKey(current.Key, s.nextStartKey) < 0) { - s.eof = true - s.Close() - return nil - } - // Try to resolve the lock - if current.GetError() != nil { - // 'current' would be modified if the lock being resolved - if err := s.resolveCurrentLock(bo, current); err != nil { - s.Close() - return errors.Trace(err) - } - - // The check here does not violate the KeyOnly semantic, because current's value - // is filled by resolveCurrentLock which fetches the value by snapshot.get, so an empty - // value stands for NotExist - if len(current.Value) == 0 { - continue - } - } - return nil - } -} - -// Close close iterator. -func (s *Scanner) Close() { - s.valid = false -} - -func (s *Scanner) startTS() uint64 { - return s.snapshot.version -} - -func (s *Scanner) resolveCurrentLock(bo *Backoffer, current *kvrpcpb.KvPair) error { - ctx := context.Background() - val, err := s.snapshot.get(ctx, bo, current.Key) - if err != nil { - return errors.Trace(err) - } - current.Error = nil - current.Value = val - return nil -} - -func (s *Scanner) getData(bo *Backoffer) error { - logutil.BgLogger().Debug("txn getData", - zap.String("nextStartKey", kv.StrKey(s.nextStartKey)), - zap.String("nextEndKey", kv.StrKey(s.nextEndKey)), - zap.Bool("reverse", s.reverse), - zap.Uint64("txnStartTS", s.startTS())) - sender := locate.NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.GetTiKVClient()) - var reqEndKey, reqStartKey []byte - var loc *locate.KeyLocation - var err error - for { - if !s.reverse { - loc, err = s.snapshot.store.regionCache.LocateKey(bo, s.nextStartKey) - } else { - loc, err = s.snapshot.store.regionCache.LocateEndKey(bo, s.nextEndKey) - } - if err != nil { - return errors.Trace(err) - } - - if !s.reverse { - reqEndKey = s.endKey - if len(reqEndKey) > 0 && len(loc.EndKey) > 0 && bytes.Compare(loc.EndKey, reqEndKey) < 0 { - reqEndKey = loc.EndKey - } - } else { - reqStartKey = s.nextStartKey - if len(reqStartKey) == 0 || - (len(loc.StartKey) > 0 && bytes.Compare(loc.StartKey, reqStartKey) > 0) { - reqStartKey = loc.StartKey - } - } - sreq := &kvrpcpb.ScanRequest{ - Context: &kvrpcpb.Context{ - Priority: s.snapshot.priority.ToPB(), - NotFillCache: s.snapshot.notFillCache, - IsolationLevel: s.snapshot.isolationLevel.ToPB(), - ResourceGroupTag: s.snapshot.resourceGroupTag, - }, - StartKey: s.nextStartKey, - EndKey: reqEndKey, - Limit: uint32(s.batchSize), - Version: s.startTS(), - KeyOnly: s.snapshot.keyOnly, - SampleStep: s.snapshot.sampleStep, - } - if s.reverse { - sreq.StartKey = s.nextEndKey - sreq.EndKey = reqStartKey - sreq.Reverse = true - } - s.snapshot.mu.RLock() - req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdScan, sreq, s.snapshot.mu.replicaRead, &s.snapshot.replicaReadSeed, kvrpcpb.Context{ - Priority: s.snapshot.priority.ToPB(), - NotFillCache: s.snapshot.notFillCache, - TaskId: s.snapshot.mu.taskID, - ResourceGroupTag: s.snapshot.resourceGroupTag, - }) - s.snapshot.mu.RUnlock() - resp, err := sender.SendReq(bo, req, loc.Region, client.ReadTimeoutMedium) - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - logutil.BgLogger().Debug("scanner getData failed", - zap.Stringer("regionErr", regionErr)) - // For other region error and the fake region error, backoff because - // there's something wrong. - // For the real EpochNotMatch error, don't backoff. - if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - } - continue - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - cmdScanResp := resp.Resp.(*kvrpcpb.ScanResponse) - - err = s.snapshot.store.CheckVisibility(s.startTS()) - if err != nil { - return errors.Trace(err) - } - - // When there is a response-level key error, the returned pairs are incomplete. - // We should resolve the lock first and then retry the same request. - if keyErr := cmdScanResp.GetError(); keyErr != nil { - lock, err := extractLockFromKeyErr(keyErr) - if err != nil { - return errors.Trace(err) - } - msBeforeExpired, _, err := newLockResolver(s.snapshot.store).ResolveLocks(bo, s.snapshot.version, []*Lock{lock}) - if err != nil { - return errors.Trace(err) - } - if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleepTxnLockFast(int(msBeforeExpired), errors.Errorf("key is locked during scanning")) - if err != nil { - return errors.Trace(err) - } - } - continue - } - - kvPairs := cmdScanResp.Pairs - // Check if kvPair contains error, it should be a Lock. - for _, pair := range kvPairs { - if keyErr := pair.GetError(); keyErr != nil && len(pair.Key) == 0 { - lock, err := extractLockFromKeyErr(keyErr) - if err != nil { - return errors.Trace(err) - } - pair.Key = lock.Key - } - } - - s.cache, s.idx = kvPairs, 0 - if len(kvPairs) < s.batchSize { - // No more data in current Region. Next getData() starts - // from current Region's endKey. - if !s.reverse { - s.nextStartKey = loc.EndKey - } else { - s.nextEndKey = reqStartKey - } - if (!s.reverse && (len(loc.EndKey) == 0 || (len(s.endKey) > 0 && kv.CmpKey(s.nextStartKey, s.endKey) >= 0))) || - (s.reverse && (len(loc.StartKey) == 0 || (len(s.nextStartKey) > 0 && kv.CmpKey(s.nextStartKey, s.nextEndKey) >= 0))) { - // Current Region is the last one. - s.eof = true - } - return nil - } - // next getData() starts from the last key in kvPairs (but skip - // it by appending a '\x00' to the key). Note that next getData() - // may get an empty response if the Region in fact does not have - // more data. - lastKey := kvPairs[len(kvPairs)-1].GetKey() - if !s.reverse { - s.nextStartKey = kv.NextKey(lastKey) - } else { - s.nextEndKey = lastKey - } - return nil - } -} diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go deleted file mode 100644 index 15c33a5944269..0000000000000 --- a/store/tikv/snapshot.go +++ /dev/null @@ -1,859 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - "fmt" - "math" - "sync" - "sync/atomic" - "time" - - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/tikv/client" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/unionstore" - "github.com/pingcap/tidb/store/tikv/util" - "go.uber.org/zap" -) - -const ( - scanBatchSize = 256 - batchGetSize = 5120 - maxTimestamp = math.MaxUint64 -) - -// Priority is the priority for tikv to execute a command. -type Priority kvrpcpb.CommandPri - -// Priority value for transaction priority. -const ( - PriorityNormal = Priority(kvrpcpb.CommandPri_Normal) - PriorityLow = Priority(kvrpcpb.CommandPri_Low) - PriorityHigh = Priority(kvrpcpb.CommandPri_High) -) - -// ToPB converts priority to wire type. -func (p Priority) ToPB() kvrpcpb.CommandPri { - return kvrpcpb.CommandPri(p) -} - -// IsoLevel is the transaction's isolation level. -type IsoLevel kvrpcpb.IsolationLevel - -const ( - // SI stands for 'snapshot isolation'. - SI IsoLevel = IsoLevel(kvrpcpb.IsolationLevel_SI) - // RC stands for 'read committed'. - RC IsoLevel = IsoLevel(kvrpcpb.IsolationLevel_RC) -) - -// ToPB converts isolation level to wire type. -func (l IsoLevel) ToPB() kvrpcpb.IsolationLevel { - return kvrpcpb.IsolationLevel(l) -} - -// KVSnapshot implements the tidbkv.Snapshot interface. -type KVSnapshot struct { - store *KVStore - version uint64 - isolationLevel IsoLevel - priority Priority - notFillCache bool - keyOnly bool - vars *kv.Variables - replicaReadSeed uint32 - resolvedLocks *util.TSSet - - // Cache the result of BatchGet. - // The invariance is that calling BatchGet multiple times using the same start ts, - // the result should not change. - // NOTE: This representation here is different from the BatchGet API. - // cached use len(value)=0 to represent a key-value entry doesn't exist (a reliable truth from TiKV). - // In the BatchGet API, it use no key-value entry to represent non-exist. - // It's OK as long as there are no zero-byte values in the protocol. - mu struct { - sync.RWMutex - hitCnt int64 - cached map[string][]byte - cachedSize int - stats *SnapshotRuntimeStats - replicaRead kv.ReplicaReadType - taskID uint64 - isStaleness bool - txnScope string - // MatchStoreLabels indicates the labels the store should be matched - matchStoreLabels []*metapb.StoreLabel - } - sampleStep uint32 - // resourceGroupTag is use to set the kv request resource group tag. - resourceGroupTag []byte -} - -// newTiKVSnapshot creates a snapshot of an TiKV store. -func newTiKVSnapshot(store *KVStore, ts uint64, replicaReadSeed uint32) *KVSnapshot { - // Sanity check for snapshot version. - if ts >= math.MaxInt64 && ts != math.MaxUint64 { - err := errors.Errorf("try to get snapshot with a large ts %d", ts) - panic(err) - } - return &KVSnapshot{ - store: store, - version: ts, - priority: PriorityNormal, - vars: kv.DefaultVars, - replicaReadSeed: replicaReadSeed, - resolvedLocks: util.NewTSSet(5), - } -} - -const batchGetMaxBackoff = 600000 // 10 minutes - -// SetSnapshotTS resets the timestamp for reads. -func (s *KVSnapshot) SetSnapshotTS(ts uint64) { - // Sanity check for snapshot version. - if ts >= math.MaxInt64 && ts != math.MaxUint64 { - err := errors.Errorf("try to get snapshot with a large ts %d", ts) - panic(err) - } - // Invalidate cache if the snapshotTS change! - s.version = ts - s.mu.Lock() - s.mu.cached = nil - s.mu.Unlock() - // And also the minCommitTS pushed information. - s.resolvedLocks = util.NewTSSet(5) -} - -// BatchGet gets all the keys' value from kv-server and returns a map contains key/value pairs. -// The map will not contain nonexistent keys. -// NOTE: Don't modify keys. Some codes rely on the order of keys. -func (s *KVSnapshot) BatchGet(ctx context.Context, keys [][]byte) (map[string][]byte, error) { - // Check the cached value first. - m := make(map[string][]byte) - s.mu.RLock() - if s.mu.cached != nil { - tmp := make([][]byte, 0, len(keys)) - for _, key := range keys { - if val, ok := s.mu.cached[string(key)]; ok { - atomic.AddInt64(&s.mu.hitCnt, 1) - if len(val) > 0 { - m[string(key)] = val - } - } else { - tmp = append(tmp, key) - } - } - keys = tmp - } - s.mu.RUnlock() - - if len(keys) == 0 { - return m, nil - } - - ctx = context.WithValue(ctx, retry.TxnStartKey, s.version) - bo := retry.NewBackofferWithVars(ctx, batchGetMaxBackoff, s.vars) - - // Create a map to collect key-values from region servers. - var mu sync.Mutex - err := s.batchGetKeysByRegions(bo, keys, func(k, v []byte) { - if len(v) == 0 { - return - } - - mu.Lock() - m[string(k)] = v - mu.Unlock() - }) - s.recordBackoffInfo(bo) - if err != nil { - return nil, errors.Trace(err) - } - - err = s.store.CheckVisibility(s.version) - if err != nil { - return nil, errors.Trace(err) - } - - // Update the cache. - s.mu.Lock() - if s.mu.cached == nil { - s.mu.cached = make(map[string][]byte, len(m)) - } - for _, key := range keys { - val := m[string(key)] - s.mu.cachedSize += len(key) + len(val) - s.mu.cached[string(key)] = val - } - - const cachedSizeLimit = 10 << 30 - if s.mu.cachedSize >= cachedSizeLimit { - for k, v := range s.mu.cached { - if _, needed := m[k]; needed { - continue - } - delete(s.mu.cached, k) - s.mu.cachedSize -= len(k) + len(v) - if s.mu.cachedSize < cachedSizeLimit { - break - } - } - } - s.mu.Unlock() - - return m, nil -} - -type batchKeys struct { - region locate.RegionVerID - keys [][]byte -} - -func (b *batchKeys) relocate(bo *Backoffer, c *RegionCache) (bool, error) { - loc, err := c.LocateKey(bo, b.keys[0]) - if err != nil { - return false, errors.Trace(err) - } - // keys is not in order, so we have to iterate all keys. - for i := 1; i < len(b.keys); i++ { - if !loc.Contains(b.keys[i]) { - return false, nil - } - } - b.region = loc.Region - return true, nil -} - -// appendBatchKeysBySize appends keys to b. It may split the keys to make -// sure each batch's size does not exceed the limit. -func appendBatchKeysBySize(b []batchKeys, region locate.RegionVerID, keys [][]byte, sizeFn func([]byte) int, limit int) []batchKeys { - var start, end int - for start = 0; start < len(keys); start = end { - var size int - for end = start; end < len(keys) && size < limit; end++ { - size += sizeFn(keys[end]) - } - b = append(b, batchKeys{ - region: region, - keys: keys[start:end], - }) - } - return b -} - -func (s *KVSnapshot) batchGetKeysByRegions(bo *Backoffer, keys [][]byte, collectF func(k, v []byte)) error { - defer func(start time.Time) { - metrics.TxnCmdHistogramWithBatchGet.Observe(time.Since(start).Seconds()) - }(time.Now()) - groups, _, err := s.store.regionCache.GroupKeysByRegion(bo, keys, nil) - if err != nil { - return errors.Trace(err) - } - - metrics.TxnRegionsNumHistogramWithSnapshot.Observe(float64(len(groups))) - - var batches []batchKeys - for id, g := range groups { - batches = appendBatchKeysBySize(batches, id, g, func([]byte) int { return 1 }, batchGetSize) - } - - if len(batches) == 0 { - return nil - } - if len(batches) == 1 { - return errors.Trace(s.batchGetSingleRegion(bo, batches[0], collectF)) - } - ch := make(chan error) - for _, batch1 := range batches { - batch := batch1 - go func() { - backoffer, cancel := bo.Fork() - defer cancel() - ch <- s.batchGetSingleRegion(backoffer, batch, collectF) - }() - } - for i := 0; i < len(batches); i++ { - if e := <-ch; e != nil { - logutil.BgLogger().Debug("snapshot batchGet failed", - zap.Error(e), - zap.Uint64("txnStartTS", s.version)) - err = e - } - } - return errors.Trace(err) -} - -func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collectF func(k, v []byte)) error { - cli := NewClientHelper(s.store, s.resolvedLocks) - s.mu.RLock() - if s.mu.stats != nil { - cli.Stats = make(map[tikvrpc.CmdType]*locate.RPCRuntimeStats) - defer func() { - s.mergeRegionRequestStats(cli.Stats) - }() - } - s.mu.RUnlock() - - pending := batch.keys - for { - s.mu.RLock() - req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdBatchGet, &kvrpcpb.BatchGetRequest{ - Keys: pending, - Version: s.version, - }, s.mu.replicaRead, &s.replicaReadSeed, kvrpcpb.Context{ - Priority: s.priority.ToPB(), - NotFillCache: s.notFillCache, - TaskId: s.mu.taskID, - ResourceGroupTag: s.resourceGroupTag, - }) - txnScope := s.mu.txnScope - isStaleness := s.mu.isStaleness - matchStoreLabels := s.mu.matchStoreLabels - s.mu.RUnlock() - req.TxnScope = txnScope - if isStaleness { - req.EnableStaleRead() - } - ops := make([]StoreSelectorOption, 0, 2) - if len(matchStoreLabels) > 0 { - ops = append(ops, locate.WithMatchLabels(matchStoreLabels)) - } - resp, _, _, err := cli.SendReqCtx(bo, req, batch.region, client.ReadTimeoutMedium, tikvrpc.TiKV, "", ops...) - - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - // For other region error and the fake region error, backoff because - // there's something wrong. - // For the real EpochNotMatch error, don't backoff. - if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) - } - } - same, err := batch.relocate(bo, cli.regionCache) - if err != nil { - return errors.Trace(err) - } - if same { - continue - } - err = s.batchGetKeysByRegions(bo, pending, collectF) - return errors.Trace(err) - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - batchGetResp := resp.Resp.(*kvrpcpb.BatchGetResponse) - var ( - lockedKeys [][]byte - locks []*Lock - ) - if keyErr := batchGetResp.GetError(); keyErr != nil { - // If a response-level error happens, skip reading pairs. - lock, err := extractLockFromKeyErr(keyErr) - if err != nil { - return errors.Trace(err) - } - lockedKeys = append(lockedKeys, lock.Key) - locks = append(locks, lock) - } else { - for _, pair := range batchGetResp.Pairs { - keyErr := pair.GetError() - if keyErr == nil { - collectF(pair.GetKey(), pair.GetValue()) - continue - } - lock, err := extractLockFromKeyErr(keyErr) - if err != nil { - return errors.Trace(err) - } - lockedKeys = append(lockedKeys, lock.Key) - locks = append(locks, lock) - } - } - if batchGetResp.ExecDetailsV2 != nil { - readKeys := len(batchGetResp.Pairs) - readTime := float64(batchGetResp.ExecDetailsV2.GetTimeDetail().GetKvReadWallTimeMs() / 1000) - metrics.ObserveReadSLI(uint64(readKeys), readTime) - s.mergeExecDetail(batchGetResp.ExecDetailsV2) - } - if len(lockedKeys) > 0 { - msBeforeExpired, err := cli.ResolveLocks(bo, s.version, locks) - if err != nil { - return errors.Trace(err) - } - if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleepTxnLockFast(int(msBeforeExpired), errors.Errorf("batchGet lockedKeys: %d", len(lockedKeys))) - if err != nil { - return errors.Trace(err) - } - } - // Only reduce pending keys when there is no response-level error. Otherwise, - // lockedKeys may be incomplete. - if batchGetResp.GetError() == nil { - pending = lockedKeys - } - continue - } - return nil - } -} - -const getMaxBackoff = 600000 // 10 minutes - -// Get gets the value for key k from snapshot. -func (s *KVSnapshot) Get(ctx context.Context, k []byte) ([]byte, error) { - - defer func(start time.Time) { - metrics.TxnCmdHistogramWithGet.Observe(time.Since(start).Seconds()) - }(time.Now()) - - ctx = context.WithValue(ctx, retry.TxnStartKey, s.version) - bo := retry.NewBackofferWithVars(ctx, getMaxBackoff, s.vars) - val, err := s.get(ctx, bo, k) - s.recordBackoffInfo(bo) - if err != nil { - return nil, errors.Trace(err) - } - err = s.store.CheckVisibility(s.version) - if err != nil { - return nil, errors.Trace(err) - } - - if len(val) == 0 { - return nil, tikverr.ErrNotExist - } - return val, nil -} - -func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, error) { - // Check the cached values first. - s.mu.RLock() - if s.mu.cached != nil { - if value, ok := s.mu.cached[string(k)]; ok { - atomic.AddInt64(&s.mu.hitCnt, 1) - s.mu.RUnlock() - return value, nil - } - } - s.mu.RUnlock() - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("tikvSnapshot.get", opentracing.ChildOf(span.Context())) - defer span1.Finish() - opentracing.ContextWithSpan(ctx, span1) - } - failpoint.Inject("snapshot-get-cache-fail", func(_ failpoint.Value) { - if bo.GetCtx().Value("TestSnapshotCache") != nil { - panic("cache miss") - } - }) - - cli := NewClientHelper(s.store, s.resolvedLocks) - - s.mu.RLock() - if s.mu.stats != nil { - cli.Stats = make(map[tikvrpc.CmdType]*locate.RPCRuntimeStats) - defer func() { - s.mergeRegionRequestStats(cli.Stats) - }() - } - req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, - &kvrpcpb.GetRequest{ - Key: k, - Version: s.version, - }, s.mu.replicaRead, &s.replicaReadSeed, kvrpcpb.Context{ - Priority: s.priority.ToPB(), - NotFillCache: s.notFillCache, - TaskId: s.mu.taskID, - ResourceGroupTag: s.resourceGroupTag, - }) - isStaleness := s.mu.isStaleness - matchStoreLabels := s.mu.matchStoreLabels - s.mu.RUnlock() - var ops []locate.StoreSelectorOption - if isStaleness { - req.EnableStaleRead() - } - if len(matchStoreLabels) > 0 { - ops = append(ops, locate.WithMatchLabels(matchStoreLabels)) - } - - var firstLock *Lock - for { - util.EvalFailpoint("beforeSendPointGet") - loc, err := s.store.regionCache.LocateKey(bo, k) - if err != nil { - return nil, errors.Trace(err) - } - resp, _, _, err := cli.SendReqCtx(bo, req, loc.Region, client.ReadTimeoutShort, tikvrpc.TiKV, "", ops...) - if err != nil { - return nil, errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return nil, errors.Trace(err) - } - if regionErr != nil { - // For other region error and the fake region error, backoff because - // there's something wrong. - // For the real EpochNotMatch error, don't backoff. - if regionErr.GetEpochNotMatch() == nil || locate.IsFakeRegionError(regionErr) { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return nil, errors.Trace(err) - } - } - continue - } - if resp.Resp == nil { - return nil, errors.Trace(tikverr.ErrBodyMissing) - } - cmdGetResp := resp.Resp.(*kvrpcpb.GetResponse) - if cmdGetResp.ExecDetailsV2 != nil { - readKeys := len(cmdGetResp.Value) - readTime := float64(cmdGetResp.ExecDetailsV2.GetTimeDetail().GetKvReadWallTimeMs() / 1000) - metrics.ObserveReadSLI(uint64(readKeys), readTime) - s.mergeExecDetail(cmdGetResp.ExecDetailsV2) - } - val := cmdGetResp.GetValue() - if keyErr := cmdGetResp.GetError(); keyErr != nil { - lock, err := extractLockFromKeyErr(keyErr) - if err != nil { - return nil, errors.Trace(err) - } - if firstLock == nil { - firstLock = lock - } else if s.version == maxTimestamp && firstLock.TxnID != lock.TxnID { - // If it is an autocommit point get, it needs to be blocked only - // by the first lock it meets. During retries, if the encountered - // lock is different from the first one, we can omit it. - cli.resolvedLocks.Put(lock.TxnID) - continue - } - - msBeforeExpired, err := cli.ResolveLocks(bo, s.version, []*Lock{lock}) - if err != nil { - return nil, errors.Trace(err) - } - if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleepTxnLockFast(int(msBeforeExpired), errors.New(keyErr.String())) - if err != nil { - return nil, errors.Trace(err) - } - } - continue - } - return val, nil - } -} - -func (s *KVSnapshot) mergeExecDetail(detail *kvrpcpb.ExecDetailsV2) { - s.mu.Lock() - defer s.mu.Unlock() - if detail == nil || s.mu.stats == nil { - return - } - if s.mu.stats.scanDetail == nil { - s.mu.stats.scanDetail = &util.ScanDetail{} - } - if s.mu.stats.timeDetail == nil { - s.mu.stats.timeDetail = &util.TimeDetail{} - } - s.mu.stats.scanDetail.MergeFromScanDetailV2(detail.ScanDetailV2) - s.mu.stats.timeDetail.MergeFromTimeDetail(detail.TimeDetail) -} - -// Iter return a list of key-value pair after `k`. -func (s *KVSnapshot) Iter(k []byte, upperBound []byte) (unionstore.Iterator, error) { - scanner, err := newScanner(s, k, upperBound, scanBatchSize, false) - return scanner, errors.Trace(err) -} - -// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. -func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { - scanner, err := newScanner(s, nil, k, scanBatchSize, true) - return scanner, errors.Trace(err) -} - -// SetNotFillCache indicates whether tikv should skip filling cache when -// loading data. -func (s *KVSnapshot) SetNotFillCache(b bool) { - s.notFillCache = b -} - -// SetKeyOnly indicates if tikv can return only keys. -func (s *KVSnapshot) SetKeyOnly(b bool) { - s.keyOnly = b -} - -// SetReplicaRead sets up the replica read type. -func (s *KVSnapshot) SetReplicaRead(readType kv.ReplicaReadType) { - s.mu.Lock() - defer s.mu.Unlock() - s.mu.replicaRead = readType -} - -// SetIsolationLevel sets the isolation level used to scan data from tikv. -func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { - s.isolationLevel = level -} - -// SetSampleStep skips 'step - 1' number of keys after each returned key. -func (s *KVSnapshot) SetSampleStep(step uint32) { - s.sampleStep = step -} - -// SetPriority sets the priority for tikv to execute commands. -func (s *KVSnapshot) SetPriority(pri Priority) { - s.priority = pri -} - -// SetTaskID marks current task's unique ID to allow TiKV to schedule -// tasks more fairly. -func (s *KVSnapshot) SetTaskID(id uint64) { - s.mu.Lock() - defer s.mu.Unlock() - s.mu.taskID = id -} - -// SetRuntimeStats sets the stats to collect runtime statistics. -// Set it to nil to clear stored stats. -func (s *KVSnapshot) SetRuntimeStats(stats *SnapshotRuntimeStats) { - s.mu.Lock() - defer s.mu.Unlock() - s.mu.stats = stats -} - -// SetTxnScope sets up the txn scope. -func (s *KVSnapshot) SetTxnScope(txnScope string) { - s.mu.Lock() - defer s.mu.Unlock() - s.mu.txnScope = txnScope -} - -// SetIsStatenessReadOnly indicates whether the transaction is staleness read only transaction -func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) { - s.mu.Lock() - defer s.mu.Unlock() - s.mu.isStaleness = b -} - -// SetMatchStoreLabels sets up labels to filter target stores. -func (s *KVSnapshot) SetMatchStoreLabels(labels []*metapb.StoreLabel) { - s.mu.Lock() - defer s.mu.Unlock() - s.mu.matchStoreLabels = labels -} - -// SetResourceGroupTag sets resource group of the kv request. -func (s *KVSnapshot) SetResourceGroupTag(tag []byte) { - s.resourceGroupTag = tag -} - -// SnapCacheHitCount gets the snapshot cache hit count. Only for test. -func (s *KVSnapshot) SnapCacheHitCount() int { - return int(atomic.LoadInt64(&s.mu.hitCnt)) -} - -// SnapCacheSize gets the snapshot cache size. Only for test. -func (s *KVSnapshot) SnapCacheSize() int { - s.mu.RLock() - defer s.mu.RLock() - return len(s.mu.cached) -} - -func extractLockFromKeyErr(keyErr *kvrpcpb.KeyError) (*Lock, error) { - if locked := keyErr.GetLocked(); locked != nil { - return NewLock(locked), nil - } - return nil, extractKeyErr(keyErr) -} - -func extractKeyErr(keyErr *kvrpcpb.KeyError) error { - if val, err := util.EvalFailpoint("mockRetryableErrorResp"); err == nil { - if val.(bool) { - keyErr.Conflict = nil - keyErr.Retryable = "mock retryable error" - } - } - - if keyErr.Conflict != nil { - return &tikverr.ErrWriteConflict{WriteConflict: keyErr.GetConflict()} - } - - if keyErr.Retryable != "" { - return &tikverr.ErrRetryable{Retryable: keyErr.Retryable} - } - - if keyErr.Abort != "" { - err := errors.Errorf("tikv aborts txn: %s", keyErr.GetAbort()) - logutil.BgLogger().Warn("2PC failed", zap.Error(err)) - return errors.Trace(err) - } - if keyErr.CommitTsTooLarge != nil { - err := errors.Errorf("commit TS %v is too large", keyErr.CommitTsTooLarge.CommitTs) - logutil.BgLogger().Warn("2PC failed", zap.Error(err)) - return errors.Trace(err) - } - if keyErr.TxnNotFound != nil { - err := errors.Errorf("txn %d not found", keyErr.TxnNotFound.StartTs) - return errors.Trace(err) - } - return errors.Errorf("unexpected KeyError: %s", keyErr.String()) -} - -func (s *KVSnapshot) recordBackoffInfo(bo *Backoffer) { - s.mu.RLock() - if s.mu.stats == nil || bo.GetTotalSleep() == 0 { - s.mu.RUnlock() - return - } - s.mu.RUnlock() - s.mu.Lock() - defer s.mu.Unlock() - if s.mu.stats == nil { - return - } - if s.mu.stats.backoffSleepMS == nil { - s.mu.stats.backoffSleepMS = bo.GetBackoffSleepMS() - s.mu.stats.backoffTimes = bo.GetBackoffTimes() - return - } - for k, v := range bo.GetBackoffSleepMS() { - s.mu.stats.backoffSleepMS[k] += v - } - for k, v := range bo.GetBackoffTimes() { - s.mu.stats.backoffTimes[k] += v - } -} - -func (s *KVSnapshot) mergeRegionRequestStats(stats map[tikvrpc.CmdType]*locate.RPCRuntimeStats) { - s.mu.Lock() - defer s.mu.Unlock() - if s.mu.stats == nil { - return - } - if s.mu.stats.rpcStats.Stats == nil { - s.mu.stats.rpcStats.Stats = stats - return - } - for k, v := range stats { - stat, ok := s.mu.stats.rpcStats.Stats[k] - if !ok { - s.mu.stats.rpcStats.Stats[k] = v - continue - } - stat.Count += v.Count - stat.Consume += v.Consume - } -} - -// SnapshotRuntimeStats records the runtime stats of snapshot. -type SnapshotRuntimeStats struct { - rpcStats locate.RegionRequestRuntimeStats - backoffSleepMS map[string]int - backoffTimes map[string]int - scanDetail *util.ScanDetail - timeDetail *util.TimeDetail -} - -// Clone implements the RuntimeStats interface. -func (rs *SnapshotRuntimeStats) Clone() *SnapshotRuntimeStats { - newRs := SnapshotRuntimeStats{rpcStats: locate.NewRegionRequestRuntimeStats()} - if rs.rpcStats.Stats != nil { - for k, v := range rs.rpcStats.Stats { - newRs.rpcStats.Stats[k] = v - } - } - if len(rs.backoffSleepMS) > 0 { - newRs.backoffSleepMS = make(map[string]int) - newRs.backoffTimes = make(map[string]int) - for k, v := range rs.backoffSleepMS { - newRs.backoffSleepMS[k] += v - } - for k, v := range rs.backoffTimes { - newRs.backoffTimes[k] += v - } - } - return &newRs -} - -// Merge implements the RuntimeStats interface. -func (rs *SnapshotRuntimeStats) Merge(other *SnapshotRuntimeStats) { - if other.rpcStats.Stats != nil { - if rs.rpcStats.Stats == nil { - rs.rpcStats.Stats = make(map[tikvrpc.CmdType]*locate.RPCRuntimeStats, len(other.rpcStats.Stats)) - } - rs.rpcStats.Merge(other.rpcStats) - } - if len(other.backoffSleepMS) > 0 { - if rs.backoffSleepMS == nil { - rs.backoffSleepMS = make(map[string]int) - } - if rs.backoffTimes == nil { - rs.backoffTimes = make(map[string]int) - } - for k, v := range other.backoffSleepMS { - rs.backoffSleepMS[k] += v - } - for k, v := range other.backoffTimes { - rs.backoffTimes[k] += v - } - } -} - -// String implements fmt.Stringer interface. -func (rs *SnapshotRuntimeStats) String() string { - var buf bytes.Buffer - buf.WriteString(rs.rpcStats.String()) - for k, v := range rs.backoffTimes { - if buf.Len() > 0 { - buf.WriteByte(',') - } - ms := rs.backoffSleepMS[k] - d := time.Duration(ms) * time.Millisecond - buf.WriteString(fmt.Sprintf("%s_backoff:{num:%d, total_time:%s}", k, v, util.FormatDuration(d))) - } - timeDetail := rs.timeDetail.String() - if timeDetail != "" { - buf.WriteString(", ") - buf.WriteString(timeDetail) - } - scanDetail := rs.scanDetail.String() - if scanDetail != "" { - buf.WriteString(", ") - buf.WriteString(scanDetail) - } - return buf.String() -} diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go deleted file mode 100644 index 8be35a61e9e03..0000000000000 --- a/store/tikv/split_region.go +++ /dev/null @@ -1,352 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - "fmt" - "math" - "sync/atomic" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/pingcap/tidb/store/tikv/client" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - pd "github.com/tikv/pd/client" - "go.uber.org/zap" -) - -const splitBatchRegionLimit = 16 - -func equalRegionStartKey(key, regionStartKey []byte) bool { - return bytes.Equal(key, regionStartKey) -} - -func (s *KVStore) splitBatchRegionsReq(bo *Backoffer, keys [][]byte, scatter bool, tableID *int64) (*tikvrpc.Response, error) { - // equalRegionStartKey is used to filter split keys. - // If the split key is equal to the start key of the region, then the key has been split, we need to skip the split key. - groups, _, err := s.regionCache.GroupKeysByRegion(bo, keys, equalRegionStartKey) - if err != nil { - return nil, errors.Trace(err) - } - - var batches []batch - for regionID, groupKeys := range groups { - batches = appendKeyBatches(batches, regionID, groupKeys, splitBatchRegionLimit) - } - - if len(batches) == 0 { - return nil, nil - } - // The first time it enters this function. - if bo.GetTotalSleep() == 0 { - logutil.BgLogger().Info("split batch regions request", - zap.Int("split key count", len(keys)), - zap.Int("batch count", len(batches)), - zap.Uint64("first batch, region ID", batches[0].regionID.GetID()), - zap.String("first split key", kv.StrKey(batches[0].keys[0]))) - } - if len(batches) == 1 { - resp := s.batchSendSingleRegion(bo, batches[0], scatter, tableID) - return resp.resp, errors.Trace(resp.err) - } - ch := make(chan singleBatchResp, len(batches)) - for _, batch1 := range batches { - go func(b batch) { - backoffer, cancel := bo.Fork() - defer cancel() - - util.WithRecovery(func() { - select { - case ch <- s.batchSendSingleRegion(backoffer, b, scatter, tableID): - case <-bo.GetCtx().Done(): - ch <- singleBatchResp{err: bo.GetCtx().Err()} - } - }, func(r interface{}) { - if r != nil { - ch <- singleBatchResp{err: errors.Errorf("%v", r)} - } - }) - }(batch1) - } - - srResp := &kvrpcpb.SplitRegionResponse{Regions: make([]*metapb.Region, 0, len(keys)*2)} - for i := 0; i < len(batches); i++ { - batchResp := <-ch - if batchResp.err != nil { - logutil.BgLogger().Info("batch split regions failed", zap.Error(batchResp.err)) - if err == nil { - err = batchResp.err - } - } - - // If the split succeeds and the scatter fails, we also need to add the region IDs. - if batchResp.resp != nil { - spResp := batchResp.resp.Resp.(*kvrpcpb.SplitRegionResponse) - regions := spResp.GetRegions() - srResp.Regions = append(srResp.Regions, regions...) - } - } - return &tikvrpc.Response{Resp: srResp}, errors.Trace(err) -} - -func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool, tableID *int64) singleBatchResp { - if val, err := util.EvalFailpoint("mockSplitRegionTimeout"); err == nil { - if val.(bool) { - if _, ok := bo.GetCtx().Deadline(); ok { - <-bo.GetCtx().Done() - } - } - } - - req := tikvrpc.NewRequest(tikvrpc.CmdSplitRegion, &kvrpcpb.SplitRegionRequest{ - SplitKeys: batch.keys, - }, kvrpcpb.Context{ - Priority: kvrpcpb.CommandPri_Normal, - }) - - sender := locate.NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) - resp, err := sender.SendReq(bo, req, batch.regionID, client.ReadTimeoutShort) - - batchResp := singleBatchResp{resp: resp} - if err != nil { - batchResp.err = errors.Trace(err) - return batchResp - } - regionErr, err := resp.GetRegionError() - if err != nil { - batchResp.err = errors.Trace(err) - return batchResp - } - if regionErr != nil { - err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - batchResp.err = errors.Trace(err) - return batchResp - } - resp, err = s.splitBatchRegionsReq(bo, batch.keys, scatter, tableID) - batchResp.resp = resp - batchResp.err = err - return batchResp - } - - spResp := resp.Resp.(*kvrpcpb.SplitRegionResponse) - regions := spResp.GetRegions() - if len(regions) > 0 { - // Divide a region into n, one of them may not need to be scattered, - // so n-1 needs to be scattered to other stores. - spResp.Regions = regions[:len(regions)-1] - } - var newRegionLeft string - if len(spResp.Regions) > 0 { - newRegionLeft = logutil.Hex(spResp.Regions[0]).String() - } - logutil.BgLogger().Info("batch split regions complete", - zap.Uint64("batch region ID", batch.regionID.GetID()), - zap.String("first at", kv.StrKey(batch.keys[0])), - zap.String("first new region left", newRegionLeft), - zap.Int("new region count", len(spResp.Regions))) - - if !scatter { - return batchResp - } - - for i, r := range spResp.Regions { - if err = s.scatterRegion(bo, r.Id, tableID); err == nil { - logutil.BgLogger().Info("batch split regions, scatter region complete", - zap.Uint64("batch region ID", batch.regionID.GetID()), - zap.String("at", kv.StrKey(batch.keys[i])), - zap.Stringer("new region left", logutil.Hex(r))) - continue - } - - logutil.BgLogger().Info("batch split regions, scatter region failed", - zap.Uint64("batch region ID", batch.regionID.GetID()), - zap.String("at", kv.StrKey(batch.keys[i])), - zap.Stringer("new region left", logutil.Hex(r)), - zap.Error(err)) - if batchResp.err == nil { - batchResp.err = err - } - if _, ok := err.(*tikverr.ErrPDServerTimeout); ok { - break - } - } - return batchResp -} - -const ( - splitRegionBackoff = 20000 - maxSplitRegionsBackoff = 120000 -) - -// SplitRegions splits regions by splitKeys. -func (s *KVStore) SplitRegions(ctx context.Context, splitKeys [][]byte, scatter bool, tableID *int64) (regionIDs []uint64, err error) { - bo := retry.NewBackofferWithVars(ctx, int(math.Min(float64(len(splitKeys))*splitRegionBackoff, maxSplitRegionsBackoff)), nil) - resp, err := s.splitBatchRegionsReq(bo, splitKeys, scatter, tableID) - regionIDs = make([]uint64, 0, len(splitKeys)) - if resp != nil && resp.Resp != nil { - spResp := resp.Resp.(*kvrpcpb.SplitRegionResponse) - for _, r := range spResp.Regions { - regionIDs = append(regionIDs, r.Id) - } - logutil.BgLogger().Info("split regions complete", zap.Int("region count", len(regionIDs)), zap.Uint64s("region IDs", regionIDs)) - } - return regionIDs, errors.Trace(err) -} - -func (s *KVStore) scatterRegion(bo *Backoffer, regionID uint64, tableID *int64) error { - logutil.BgLogger().Info("start scatter region", - zap.Uint64("regionID", regionID)) - for { - opts := make([]pd.RegionsOption, 0, 1) - if tableID != nil { - opts = append(opts, pd.WithGroup(fmt.Sprintf("%v", *tableID))) - } - _, err := s.pdClient.ScatterRegions(bo.GetCtx(), []uint64{regionID}, opts...) - - if val, err2 := util.EvalFailpoint("mockScatterRegionTimeout"); err2 == nil { - if val.(bool) { - err = tikverr.NewErrPDServerTimeout("") - } - } - - if err == nil { - break - } - err = bo.Backoff(retry.BoPDRPC, errors.New(err.Error())) - if err != nil { - return errors.Trace(err) - } - } - logutil.BgLogger().Debug("scatter region complete", - zap.Uint64("regionID", regionID)) - return nil -} - -func (s *KVStore) preSplitRegion(ctx context.Context, group groupedMutations) bool { - splitKeys := make([][]byte, 0, 4) - - preSplitSizeThresholdVal := atomic.LoadUint32(&preSplitSizeThreshold) - regionSize := 0 - keysLength := group.mutations.Len() - // The value length maybe zero for pessimistic lock keys - for i := 0; i < keysLength; i++ { - regionSize = regionSize + len(group.mutations.GetKey(i)) + len(group.mutations.GetValue(i)) - // The second condition is used for testing. - if regionSize >= int(preSplitSizeThresholdVal) { - regionSize = 0 - splitKeys = append(splitKeys, group.mutations.GetKey(i)) - } - } - if len(splitKeys) == 0 { - return false - } - - regionIDs, err := s.SplitRegions(ctx, splitKeys, true, nil) - if err != nil { - logutil.BgLogger().Warn("2PC split regions failed", zap.Uint64("regionID", group.region.GetID()), - zap.Int("keys count", keysLength), zap.Error(err)) - return false - } - - for _, regionID := range regionIDs { - err := s.WaitScatterRegionFinish(ctx, regionID, 0) - if err != nil { - logutil.BgLogger().Warn("2PC wait scatter region failed", zap.Uint64("regionID", regionID), zap.Error(err)) - } - } - // Invalidate the old region cache information. - s.regionCache.InvalidateCachedRegion(group.region) - return true -} - -const waitScatterRegionFinishBackoff = 120000 - -// WaitScatterRegionFinish implements SplittableStore interface. -// backOff is the back off time of the wait scatter region.(Milliseconds) -// if backOff <= 0, the default wait scatter back off time will be used. -func (s *KVStore) WaitScatterRegionFinish(ctx context.Context, regionID uint64, backOff int) error { - if backOff <= 0 { - backOff = waitScatterRegionFinishBackoff - } - logutil.BgLogger().Info("wait scatter region", - zap.Uint64("regionID", regionID), zap.Int("backoff(ms)", backOff)) - - bo := retry.NewBackofferWithVars(ctx, backOff, nil) - logFreq := 0 - for { - resp, err := s.pdClient.GetOperator(ctx, regionID) - if err == nil && resp != nil { - if !bytes.Equal(resp.Desc, []byte("scatter-region")) || resp.Status != pdpb.OperatorStatus_RUNNING { - logutil.BgLogger().Info("wait scatter region finished", - zap.Uint64("regionID", regionID)) - return nil - } - if resp.GetHeader().GetError() != nil { - err = errors.AddStack(&tikverr.PDError{ - Err: resp.Header.Error, - }) - logutil.BgLogger().Warn("wait scatter region error", - zap.Uint64("regionID", regionID), zap.Error(err)) - return err - } - if logFreq%10 == 0 { - logutil.BgLogger().Info("wait scatter region", - zap.Uint64("regionID", regionID), - zap.String("reverse", string(resp.Desc)), - zap.String("status", pdpb.OperatorStatus_name[int32(resp.Status)])) - } - logFreq++ - } - if err != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(err.Error())) - } else { - err = bo.Backoff(retry.BoRegionMiss, errors.New("wait scatter region timeout")) - } - if err != nil { - return errors.Trace(err) - } - } -} - -// CheckRegionInScattering uses to check whether scatter region finished. -func (s *KVStore) CheckRegionInScattering(regionID uint64) (bool, error) { - bo := retry.NewBackofferWithVars(context.Background(), locateRegionMaxBackoff, nil) - for { - resp, err := s.pdClient.GetOperator(context.Background(), regionID) - if err == nil && resp != nil { - if !bytes.Equal(resp.Desc, []byte("scatter-region")) || resp.Status != pdpb.OperatorStatus_RUNNING { - return false, nil - } - } - if err != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(err.Error())) - } else { - return true, nil - } - if err != nil { - return true, errors.Trace(err) - } - } -} diff --git a/store/tikv/test_probe.go b/store/tikv/test_probe.go deleted file mode 100644 index 90d0b9f7eb6cc..0000000000000 --- a/store/tikv/test_probe.go +++ /dev/null @@ -1,586 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/tikv/locate" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/unionstore" - pd "github.com/tikv/pd/client" -) - -// StoreProbe wraps KVSTore and exposes internal states for testing purpose. -type StoreProbe struct { - *KVStore -} - -// NewLockResolver creates a new LockResolver instance. -func (s StoreProbe) NewLockResolver() LockResolverProbe { - return LockResolverProbe{LockResolver: newLockResolver(s.KVStore)} -} - -// GetTimestampWithRetry returns latest timestamp. -func (s StoreProbe) GetTimestampWithRetry(bo *Backoffer, scope string) (uint64, error) { - return s.getTimestampWithRetry(bo, scope) -} - -// Begin starts a transaction. -func (s StoreProbe) Begin() (TxnProbe, error) { - txn, err := s.KVStore.Begin() - return TxnProbe{KVTxn: txn}, err -} - -// GetSnapshot returns a snapshot. -func (s StoreProbe) GetSnapshot(ts uint64) SnapshotProbe { - snap := s.KVStore.GetSnapshot(ts) - return SnapshotProbe{KVSnapshot: snap} -} - -// SetRegionCachePDClient replaces pd client inside region cache. -func (s StoreProbe) SetRegionCachePDClient(client pd.Client) { - s.regionCache.SetPDClient(client) -} - -// ClearTxnLatches clears store's txn latch scheduler. -func (s StoreProbe) ClearTxnLatches() { - s.txnLatches = nil -} - -// SendTxnHeartbeat renews a txn's ttl. -func (s StoreProbe) SendTxnHeartbeat(ctx context.Context, key []byte, startTS uint64, ttl uint64) (uint64, error) { - bo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil) - newTTL, _, err := sendTxnHeartBeat(bo, s.KVStore, key, startTS, ttl) - return newTTL, err -} - -// LoadSafePoint from safepoint kv. -func (s StoreProbe) LoadSafePoint() (uint64, error) { - return loadSafePoint(s.GetSafePointKV()) -} - -// SaveSafePoint saves safepoint to kv. -func (s StoreProbe) SaveSafePoint(v uint64) error { - return saveSafePoint(s.GetSafePointKV(), v) -} - -// 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) -} - -// SetSafeTS is used to set safeTS for the store with `storeID` -func (s StoreProbe) SetSafeTS(storeID, safeTS uint64) { - s.setSafeTS(storeID, safeTS) -} - -// TxnProbe wraps a txn and exports internal states for testing purpose. -type TxnProbe struct { - *KVTxn -} - -// SetStartTS resets the txn's start ts. -func (txn TxnProbe) SetStartTS(ts uint64) { - txn.startTS = ts -} - -// GetCommitTS returns the commit ts. -func (txn TxnProbe) GetCommitTS() uint64 { - return txn.commitTS -} - -// GetUnionStore returns transaction's embedded unionstore. -func (txn TxnProbe) GetUnionStore() *unionstore.KVUnionStore { - return txn.us -} - -// IsAsyncCommit returns if the txn is committed using async commit. -func (txn TxnProbe) IsAsyncCommit() bool { - return txn.committer.isAsyncCommit() -} - -// NewCommitter creates an committer. -func (txn TxnProbe) NewCommitter(sessionID uint64) (CommitterProbe, error) { - committer, err := newTwoPhaseCommitterWithInit(txn.KVTxn, sessionID) - return CommitterProbe{twoPhaseCommitter: committer}, err -} - -// GetCommitter returns the transaction committer. -func (txn TxnProbe) GetCommitter() CommitterProbe { - return CommitterProbe{txn.committer} -} - -// SetCommitter sets the bind committer of a transaction. -func (txn TxnProbe) SetCommitter(committer CommitterProbe) { - txn.committer = committer.twoPhaseCommitter -} - -// CollectLockedKeys returns all locked keys of a transaction. -func (txn TxnProbe) CollectLockedKeys() [][]byte { - return txn.collectLockedKeys() -} - -// BatchGetSingleRegion gets a batch of keys from a region. -func (txn TxnProbe) BatchGetSingleRegion(bo *Backoffer, region locate.RegionVerID, keys [][]byte, collect func([]byte, []byte)) error { - snapshot := txn.GetSnapshot() - return snapshot.batchGetSingleRegion(bo, batchKeys{region: region, keys: keys}, collect) -} - -// NewScanner returns a scanner to iterate given key range. -func (txn TxnProbe) NewScanner(start, end []byte, batchSize int, reverse bool) (*Scanner, error) { - return newScanner(txn.GetSnapshot(), start, end, batchSize, reverse) -} - -// GetStartTime returns the time when txn starts. -func (txn TxnProbe) GetStartTime() time.Time { - return txn.startTime -} - -func newTwoPhaseCommitterWithInit(txn *KVTxn, sessionID uint64) (*twoPhaseCommitter, error) { - c, err := newTwoPhaseCommitter(txn, sessionID) - if err != nil { - return nil, errors.Trace(err) - } - if err = c.initKeysAndMutations(); err != nil { - return nil, errors.Trace(err) - } - return c, nil -} - -// CommitterProbe wraps a 2PC committer and exports internal states for testing purpose. -type CommitterProbe struct { - *twoPhaseCommitter -} - -// InitKeysAndMutations prepares the committer for commit. -func (c CommitterProbe) InitKeysAndMutations() error { - return c.initKeysAndMutations() -} - -// SetPrimaryKey resets the committer's commit ts. -func (c CommitterProbe) SetPrimaryKey(key []byte) { - c.primaryKey = key -} - -// GetPrimaryKey returns primary key of the committer. -func (c CommitterProbe) GetPrimaryKey() []byte { - return c.primaryKey -} - -// GetMutations returns the mutation buffer to commit. -func (c CommitterProbe) GetMutations() CommitterMutations { - return c.mutations -} - -// SetMutations replace the mutation buffer. -func (c CommitterProbe) SetMutations(muts CommitterMutations) { - c.mutations = muts.(*memBufferMutations) -} - -// SetCommitTS resets the committer's commit ts. -func (c CommitterProbe) SetCommitTS(ts uint64) { - atomic.StoreUint64(&c.commitTS, ts) -} - -// GetCommitTS returns the commit ts of the committer. -func (c CommitterProbe) GetCommitTS() uint64 { - return atomic.LoadUint64(&c.commitTS) -} - -// GetMinCommitTS returns the minimal commit ts can be used. -func (c CommitterProbe) GetMinCommitTS() uint64 { - return c.minCommitTS -} - -// SetMinCommitTS sets the minimal commit ts can be used. -func (c CommitterProbe) SetMinCommitTS(ts uint64) { - c.minCommitTS = ts -} - -// SetMaxCommitTS sets the max commit ts can be used. -func (c CommitterProbe) SetMaxCommitTS(ts uint64) { - c.maxCommitTS = ts -} - -// SetSessionID sets the session id of the committer. -func (c CommitterProbe) SetSessionID(id uint64) { - c.sessionID = id -} - -// GetForUpdateTS returns the pessimistic ForUpdate ts. -func (c CommitterProbe) GetForUpdateTS() uint64 { - return c.forUpdateTS -} - -// SetForUpdateTS sets pessimistic ForUpdate ts. -func (c CommitterProbe) SetForUpdateTS(ts uint64) { - c.forUpdateTS = ts -} - -// GetStartTS returns the start ts of the transaction. -func (c CommitterProbe) GetStartTS() uint64 { - return c.startTS -} - -// GetLockTTL returns the lock ttl duration of the transaction. -func (c CommitterProbe) GetLockTTL() uint64 { - return c.lockTTL -} - -// SetLockTTL sets the lock ttl duration. -func (c CommitterProbe) SetLockTTL(ttl uint64) { - c.lockTTL = ttl -} - -// SetLockTTLByTimeAndSize sets the lock ttl duration by time and size. -func (c CommitterProbe) SetLockTTLByTimeAndSize(start time.Time, size int) { - c.lockTTL = txnLockTTL(start, size) -} - -// SetTxnSize resets the txn size of the committer and updates lock TTL. -func (c CommitterProbe) SetTxnSize(sz int) { - c.txnSize = sz - c.lockTTL = txnLockTTL(c.txn.startTime, sz) -} - -// SetUseAsyncCommit enables async commit feature. -func (c CommitterProbe) SetUseAsyncCommit() { - c.useAsyncCommit = 1 -} - -// Execute runs the commit process. -func (c CommitterProbe) Execute(ctx context.Context) error { - return c.execute(ctx) -} - -// PrewriteAllMutations performs the first phase of commit. -func (c CommitterProbe) PrewriteAllMutations(ctx context.Context) error { - return c.PrewriteMutations(ctx, c.mutations) -} - -// PrewriteMutations performs the first phase of commit for given keys. -func (c CommitterProbe) PrewriteMutations(ctx context.Context, mutations CommitterMutations) error { - return c.prewriteMutations(retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), mutations) -} - -// CommitMutations performs the second phase of commit. -func (c CommitterProbe) CommitMutations(ctx context.Context) error { - return c.commitMutations(retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), nil), c.mutationsOfKeys([][]byte{c.primaryKey})) -} - -// MutationsOfKeys returns mutations match the keys. -func (c CommitterProbe) MutationsOfKeys(keys [][]byte) CommitterMutations { - return c.mutationsOfKeys(keys) -} - -// PessimisticRollbackMutations rolls mutations back. -func (c CommitterProbe) PessimisticRollbackMutations(ctx context.Context, muts CommitterMutations) error { - return c.pessimisticRollbackMutations(retry.NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, nil), muts) -} - -// Cleanup cleans dirty data of a committer. -func (c CommitterProbe) Cleanup(ctx context.Context) { - c.cleanup(ctx) - c.cleanWg.Wait() -} - -// WaitCleanup waits for the committer to complete. -func (c CommitterProbe) WaitCleanup() { - c.cleanWg.Wait() -} - -// IsOnePC returns if the committer is using one PC. -func (c CommitterProbe) IsOnePC() bool { - return c.isOnePC() -} - -// BuildPrewriteRequest builds rpc request for mutation. -func (c CommitterProbe) BuildPrewriteRequest(regionID, regionConf, regionVersion uint64, mutations CommitterMutations, txnSize uint64) *tikvrpc.Request { - var batch batchMutations - batch.mutations = mutations - batch.region = locate.NewRegionVerID(regionID, regionConf, regionVersion) - for _, key := range mutations.GetKeys() { - if bytes.Equal(key, c.primary()) { - batch.isPrimary = true - break - } - } - return c.buildPrewriteRequest(batch, txnSize) -} - -// IsAsyncCommit returns if the committer uses async commit. -func (c CommitterProbe) IsAsyncCommit() bool { - return c.isAsyncCommit() -} - -// CheckAsyncCommit returns if async commit is available. -func (c CommitterProbe) CheckAsyncCommit() bool { - return c.checkAsyncCommit() -} - -// GetOnePCCommitTS returns the commit ts of one pc. -func (c CommitterProbe) GetOnePCCommitTS() uint64 { - return c.onePCCommitTS -} - -// IsTTLUninitialized returns if the TTL manager is uninitialized. -func (c CommitterProbe) IsTTLUninitialized() bool { - state := atomic.LoadUint32((*uint32)(&c.ttlManager.state)) - return state == uint32(stateUninitialized) -} - -// IsTTLRunning returns if the TTL manager is running state. -func (c CommitterProbe) IsTTLRunning() bool { - state := atomic.LoadUint32((*uint32)(&c.ttlManager.state)) - return state == uint32(stateRunning) -} - -// CloseTTLManager closes the TTL manager. -func (c CommitterProbe) CloseTTLManager() { - c.ttlManager.close() -} - -// GetUndeterminedErr returns the encountered undetermined error (if any). -func (c CommitterProbe) GetUndeterminedErr() error { - c.mu.RLock() - defer c.mu.RUnlock() - return c.mu.undeterminedErr -} - -// SetNoFallBack disallows async commit to fall back to normal mode. -func (c CommitterProbe) SetNoFallBack() { - c.testingKnobs.noFallBack = true -} - -// SetPrimaryKeyBlocker is used to block committer after primary is sent. -func (c CommitterProbe) SetPrimaryKeyBlocker(ac, bk chan struct{}) { - c.testingKnobs.acAfterCommitPrimary = ac - c.testingKnobs.bkAfterCommitPrimary = bk -} - -// CleanupMutations performs the clean up phase. -func (c CommitterProbe) CleanupMutations(ctx context.Context) error { - bo := retry.NewBackofferWithVars(ctx, cleanupMaxBackoff, nil) - return c.cleanupMutations(bo, c.mutations) -} - -// SnapshotProbe exposes some snapshot utilities for testing purpose. -type SnapshotProbe struct { - *KVSnapshot -} - -// MergeRegionRequestStats merges RPC runtime stats into snapshot's stats. -func (s SnapshotProbe) MergeRegionRequestStats(stats map[tikvrpc.CmdType]*locate.RPCRuntimeStats) { - s.mergeRegionRequestStats(stats) -} - -// RecordBackoffInfo records backoff stats into snapshot's stats. -func (s SnapshotProbe) RecordBackoffInfo(bo *Backoffer) { - s.recordBackoffInfo(bo) -} - -// MergeExecDetail merges exec stats into snapshot's stats. -func (s SnapshotProbe) MergeExecDetail(detail *kvrpcpb.ExecDetailsV2) { - s.mergeExecDetail(detail) -} - -// FormatStats dumps information of stats. -func (s SnapshotProbe) FormatStats() string { - s.mu.Lock() - defer s.mu.Unlock() - return s.mu.stats.String() -} - -// LockProbe exposes some lock utilities for testing purpose. -type LockProbe struct { -} - -// ExtractLockFromKeyErr makes a Lock based on a key error. -func (l LockProbe) ExtractLockFromKeyErr(err *kvrpcpb.KeyError) (*Lock, error) { - return extractLockFromKeyErr(err) -} - -// NewLockStatus returns a txn state that has been locked. -func (l LockProbe) NewLockStatus(keys [][]byte, useAsyncCommit bool, minCommitTS uint64) TxnStatus { - return TxnStatus{ - primaryLock: &kvrpcpb.LockInfo{ - Secondaries: keys, - UseAsyncCommit: useAsyncCommit, - MinCommitTs: minCommitTS, - }, - } -} - -// GetPrimaryKeyFromTxnStatus returns the primary key of the transaction. -func (l LockProbe) GetPrimaryKeyFromTxnStatus(s TxnStatus) []byte { - return s.primaryLock.Key -} - -// LockResolverProbe wraps a LockResolver and exposes internal stats for testing purpose. -type LockResolverProbe struct { - *LockResolver -} - -// ResolveLockAsync tries to resolve a lock using the txn states. -func (l LockResolverProbe) ResolveLockAsync(bo *Backoffer, lock *Lock, status TxnStatus) error { - return l.resolveLockAsync(bo, lock, status) -} - -// ResolveLock resolves single lock. -func (l LockResolverProbe) ResolveLock(ctx context.Context, lock *Lock) error { - bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) - return l.resolveLock(bo, lock, TxnStatus{}, false, make(map[locate.RegionVerID]struct{})) -} - -// ResolvePessimisticLock resolves single pessimistic lock. -func (l LockResolverProbe) ResolvePessimisticLock(ctx context.Context, lock *Lock) error { - bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) - return l.resolvePessimisticLock(bo, lock, make(map[locate.RegionVerID]struct{})) -} - -// GetTxnStatus sends the CheckTxnStatus request to the TiKV server. -func (l LockResolverProbe) GetTxnStatus(bo *Backoffer, txnID uint64, primary []byte, - callerStartTS, currentTS uint64, rollbackIfNotExist bool, forceSyncCommit bool, lockInfo *Lock) (TxnStatus, error) { - return l.getTxnStatus(bo, txnID, primary, callerStartTS, currentTS, rollbackIfNotExist, forceSyncCommit, lockInfo) -} - -// GetTxnStatusFromLock queries tikv for a txn's status. -func (l LockResolverProbe) GetTxnStatusFromLock(bo *Backoffer, lock *Lock, callerStartTS uint64, forceSyncCommit bool) (TxnStatus, error) { - return l.getTxnStatusFromLock(bo, lock, callerStartTS, forceSyncCommit) -} - -// GetSecondariesFromTxnStatus returns the secondary locks from txn status. -func (l LockResolverProbe) GetSecondariesFromTxnStatus(status TxnStatus) [][]byte { - return status.primaryLock.GetSecondaries() -} - -// SetMeetLockCallback is called whenever it meets locks. -func (l LockResolverProbe) SetMeetLockCallback(f func([]*Lock)) { - l.testingKnobs.meetLock = f -} - -// CheckAllSecondaries checks the secondary locks of an async commit transaction to find out the final -// status of the transaction. -func (l LockResolverProbe) CheckAllSecondaries(bo *Backoffer, lock *Lock, status *TxnStatus) error { - _, err := l.checkAllSecondaries(bo, lock, status) - return err -} - -// IsErrorNotFound checks if an error is caused by txnNotFoundErr. -func (l LockResolverProbe) IsErrorNotFound(err error) bool { - _, ok := errors.Cause(err).(txnNotFoundErr) - return ok -} - -// IsNonAsyncCommitLock checks if an error is nonAsyncCommitLock error. -func (l LockResolverProbe) IsNonAsyncCommitLock(err error) bool { - _, ok := errors.Cause(err).(*nonAsyncCommitLock) - return ok -} - -// ConfigProbe exposes configurations and global variables for testing purpose. -type ConfigProbe struct{} - -// GetTxnCommitBatchSize returns the batch size to commit txn. -func (c ConfigProbe) GetTxnCommitBatchSize() uint64 { - return txnCommitBatchSize -} - -// GetBigTxnThreshold returns the txn size to be considered as big txn. -func (c ConfigProbe) GetBigTxnThreshold() int { - return bigTxnThreshold -} - -// GetScanBatchSize returns the batch size to scan ranges. -func (c ConfigProbe) GetScanBatchSize() int { - return scanBatchSize -} - -// GetDefaultLockTTL returns the default lock TTL. -func (c ConfigProbe) GetDefaultLockTTL() uint64 { - return defaultLockTTL -} - -// GetTTLFactor returns the factor to calculate txn TTL. -func (c ConfigProbe) GetTTLFactor() int { - return ttlFactor -} - -// GetGetMaxBackoff returns the max sleep for get command. -func (c ConfigProbe) GetGetMaxBackoff() int { - return getMaxBackoff -} - -// LoadPreSplitDetectThreshold returns presplit detect threshold config. -func (c ConfigProbe) LoadPreSplitDetectThreshold() uint32 { - return atomic.LoadUint32(&preSplitDetectThreshold) -} - -// StorePreSplitDetectThreshold updates presplit detect threshold config. -func (c ConfigProbe) StorePreSplitDetectThreshold(v uint32) { - atomic.StoreUint32(&preSplitDetectThreshold, v) -} - -// LoadPreSplitSizeThreshold returns presplit size threshold config. -func (c ConfigProbe) LoadPreSplitSizeThreshold() uint32 { - return atomic.LoadUint32(&preSplitSizeThreshold) -} - -// StorePreSplitSizeThreshold updates presplit size threshold config. -func (c ConfigProbe) StorePreSplitSizeThreshold(v uint32) { - atomic.StoreUint32(&preSplitSizeThreshold, v) -} - -// SetOracleUpdateInterval sets the interval of updating cached ts. -func (c ConfigProbe) SetOracleUpdateInterval(v int) { - oracleUpdateInterval = v -} - -// GetRawBatchPutSize returns the raw batch put size config. -func (c ConfigProbe) GetRawBatchPutSize() int { - return rawBatchPutSize -} - -// RawKVClientProbe wraps RawKVClient and exposes internal states for testing purpose. -type RawKVClientProbe struct { - *RawKVClient -} - -// GetRegionCache returns the internal region cache container. -func (c RawKVClientProbe) GetRegionCache() *locate.RegionCache { - return c.regionCache -} - -// SetRegionCache resets the internal region cache container. -func (c RawKVClientProbe) SetRegionCache(regionCache *locate.RegionCache) { - c.regionCache = regionCache -} - -// SetPDClient resets the interval PD client. -func (c RawKVClientProbe) SetPDClient(client pd.Client) { - c.pdClient = client -} - -// SetRPCClient resets the internal RPC client. -func (c RawKVClientProbe) SetRPCClient(client Client) { - c.rpcClient = client -} diff --git a/store/tikv/test_util.go b/store/tikv/test_util.go deleted file mode 100644 index b8d2891f4407a..0000000000000 --- a/store/tikv/test_util.go +++ /dev/null @@ -1,65 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "sync/atomic" - - "github.com/google/uuid" - "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv/locate" - pd "github.com/tikv/pd/client" -) - -// NewTestTiKVStore creates a test store with Option -func NewTestTiKVStore(client Client, pdClient pd.Client, clientHijack func(Client) Client, pdClientHijack func(pd.Client) pd.Client, txnLocalLatches uint) (*KVStore, error) { - if clientHijack != nil { - client = clientHijack(client) - } - - pdCli := pd.Client(locate.NewCodeCPDClient(pdClient)) - if pdClientHijack != nil { - pdCli = pdClientHijack(pdCli) - } - - // Make sure the uuid is unique. - uid := uuid.New().String() - spkv := NewMockSafePointKV() - tikvStore, err := NewKVStore(uid, pdCli, spkv, client) - - if txnLocalLatches > 0 { - tikvStore.EnableTxnLocalLatches(txnLocalLatches) - } - - tikvStore.mock = true - return tikvStore, errors.Trace(err) -} - -// mockCommitErrorEnable uses to enable `mockCommitError` and only mock error once. -var mockCommitErrorEnable = int64(0) - -// MockCommitErrorEnable exports for gofail testing. -func MockCommitErrorEnable() { - atomic.StoreInt64(&mockCommitErrorEnable, 1) -} - -// MockCommitErrorDisable exports for gofail testing. -func MockCommitErrorDisable() { - atomic.StoreInt64(&mockCommitErrorEnable, 0) -} - -// IsMockCommitErrorEnable exports for gofail testing. -func IsMockCommitErrorEnable() bool { - return atomic.LoadInt64(&mockCommitErrorEnable) == 1 -} diff --git a/store/tikv/tests/client_fp_test.go b/store/tikv/tests/client_fp_test.go index 24e8e7a56499b..da6a23140bc4b 100644 --- a/store/tikv/tests/client_fp_test.go +++ b/store/tikv/tests/client_fp_test.go @@ -20,9 +20,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/store/tikv/util" "github.com/tikv/client-go/v2/client" "github.com/tikv/client-go/v2/tikvrpc" + "github.com/tikv/client-go/v2/util" ) // mock TiKV RPC client that hooks message by failpoint diff --git a/store/tikv/tikv_test.go b/store/tikv/tikv_test.go deleted file mode 100644 index 8d25938cf0df2..0000000000000 --- a/store/tikv/tikv_test.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2018 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "testing" - - . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/tikv/mockstore" -) - -type OneByOneSuite = mockstore.OneByOneSuite -type testTiKVSuite struct { - OneByOneSuite -} - -func TestT(t *testing.T) { - CustomVerboseFlag = true - TestingT(t) -} - -var _ = Suite(&testTiKVSuite{}) - -func (s *testTiKVSuite) TestBasicFunc(c *C) { - if IsMockCommitErrorEnable() { - defer MockCommitErrorEnable() - } else { - defer MockCommitErrorDisable() - } - - MockCommitErrorEnable() - c.Assert(IsMockCommitErrorEnable(), IsTrue) - MockCommitErrorDisable() - c.Assert(IsMockCommitErrorEnable(), IsFalse) -} diff --git a/store/tikv/tikvrpc/endpoint.go b/store/tikv/tikvrpc/endpoint.go deleted file mode 100644 index 3cd6fabbea794..0000000000000 --- a/store/tikv/tikvrpc/endpoint.go +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikvrpc - -import "github.com/pingcap/kvproto/pkg/metapb" - -// EndpointType represents the type of a remote endpoint.. -type EndpointType uint8 - -// EndpointType type enums. -const ( - TiKV EndpointType = iota - TiFlash - TiDB -) - -// Name returns the name of endpoint type. -func (t EndpointType) Name() string { - switch t { - case TiKV: - return "tikv" - case TiFlash: - return "tiflash" - case TiDB: - return "tidb" - } - return "unspecified" -} - -// Constants to determine engine type. -// They should be synced with PD. -const ( - engineLabelKey = "engine" - engineLabelTiFlash = "tiflash" -) - -// GetStoreTypeByMeta gets store type by store meta pb. -func GetStoreTypeByMeta(store *metapb.Store) EndpointType { - for _, label := range store.Labels { - if label.Key == engineLabelKey && label.Value == engineLabelTiFlash { - return TiFlash - } - } - return TiKV -} diff --git a/store/tikv/tikvrpc/tikvrpc.go b/store/tikv/tikvrpc/tikvrpc.go deleted file mode 100644 index ae450b6019799..0000000000000 --- a/store/tikv/tikvrpc/tikvrpc.go +++ /dev/null @@ -1,1111 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikvrpc - -import ( - "context" - "fmt" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/debugpb" - "github.com/pingcap/kvproto/pkg/errorpb" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/mpp" - "github.com/pingcap/kvproto/pkg/tikvpb" - "github.com/pingcap/tidb/store/tikv/kv" -) - -// CmdType represents the concrete request type in Request or response type in Response. -type CmdType uint16 - -// CmdType values. -const ( - CmdGet CmdType = 1 + iota - CmdScan - CmdPrewrite - CmdCommit - CmdCleanup - CmdBatchGet - CmdBatchRollback - CmdScanLock - CmdResolveLock - CmdGC - CmdDeleteRange - CmdPessimisticLock - CmdPessimisticRollback - CmdTxnHeartBeat - CmdCheckTxnStatus - CmdCheckSecondaryLocks - - CmdRawGet CmdType = 256 + iota - CmdRawBatchGet - CmdRawPut - CmdRawBatchPut - CmdRawDelete - CmdRawBatchDelete - CmdRawDeleteRange - CmdRawScan - - CmdUnsafeDestroyRange - - CmdRegisterLockObserver - CmdCheckLockObserver - CmdRemoveLockObserver - CmdPhysicalScanLock - - CmdStoreSafeTS - CmdLockWaitInfo - - CmdCop CmdType = 512 + iota - CmdCopStream - CmdBatchCop - CmdMPPTask - CmdMPPConn - CmdMPPCancel - - CmdMvccGetByKey CmdType = 1024 + iota - CmdMvccGetByStartTs - CmdSplitRegion - - CmdDebugGetRegionProperties CmdType = 2048 + iota - - CmdEmpty CmdType = 3072 + iota -) - -func (t CmdType) String() string { - switch t { - case CmdGet: - return "Get" - case CmdScan: - return "Scan" - case CmdPrewrite: - return "Prewrite" - case CmdPessimisticLock: - return "PessimisticLock" - case CmdPessimisticRollback: - return "PessimisticRollback" - case CmdCommit: - return "Commit" - case CmdCleanup: - return "Cleanup" - case CmdBatchGet: - return "BatchGet" - case CmdBatchRollback: - return "BatchRollback" - case CmdScanLock: - return "ScanLock" - case CmdResolveLock: - return "ResolveLock" - case CmdGC: - return "GC" - case CmdDeleteRange: - return "DeleteRange" - case CmdRawGet: - return "RawGet" - case CmdRawBatchGet: - return "RawBatchGet" - case CmdRawPut: - return "RawPut" - case CmdRawBatchPut: - return "RawBatchPut" - case CmdRawDelete: - return "RawDelete" - case CmdRawBatchDelete: - return "RawBatchDelete" - case CmdRawDeleteRange: - return "RawDeleteRange" - case CmdRawScan: - return "RawScan" - case CmdUnsafeDestroyRange: - return "UnsafeDestroyRange" - case CmdRegisterLockObserver: - return "RegisterLockObserver" - case CmdCheckLockObserver: - return "CheckLockObserver" - case CmdRemoveLockObserver: - return "RemoveLockObserver" - case CmdPhysicalScanLock: - return "PhysicalScanLock" - case CmdCop: - return "Cop" - case CmdCopStream: - return "CopStream" - case CmdBatchCop: - return "BatchCop" - case CmdMPPTask: - return "DispatchMPPTask" - case CmdMPPConn: - return "EstablishMPPConnection" - case CmdMPPCancel: - return "CancelMPPTask" - case CmdMvccGetByKey: - return "MvccGetByKey" - case CmdMvccGetByStartTs: - return "MvccGetByStartTS" - case CmdSplitRegion: - return "SplitRegion" - case CmdCheckTxnStatus: - return "CheckTxnStatus" - case CmdCheckSecondaryLocks: - return "CheckSecondaryLocks" - case CmdDebugGetRegionProperties: - return "DebugGetRegionProperties" - case CmdTxnHeartBeat: - return "TxnHeartBeat" - case CmdStoreSafeTS: - return "StoreSafeTS" - case CmdLockWaitInfo: - return "LockWaitInfo" - } - return "Unknown" -} - -// Request wraps all kv/coprocessor requests. -type Request struct { - Type CmdType - Req interface{} - kvrpcpb.Context - TxnScope string - ReplicaReadType kv.ReplicaReadType // different from `kvrpcpb.Context.ReplicaRead` - ReplicaReadSeed *uint32 // pointer to follower read seed in snapshot/coprocessor - StoreTp EndpointType - // ForwardedHost is the address of a store which will handle the request. It's different from - // the address the request sent to. - // If it's not empty, the store which receive the request will forward it to - // the forwarded host. It's useful when network partition occurs. - ForwardedHost string -} - -// NewRequest returns new kv rpc request. -func NewRequest(typ CmdType, pointer interface{}, ctxs ...kvrpcpb.Context) *Request { - if len(ctxs) > 0 { - return &Request{ - Type: typ, - Req: pointer, - Context: ctxs[0], - } - } - return &Request{ - Type: typ, - Req: pointer, - } -} - -// NewReplicaReadRequest returns new kv rpc request with replica read. -func NewReplicaReadRequest(typ CmdType, pointer interface{}, replicaReadType kv.ReplicaReadType, replicaReadSeed *uint32, ctxs ...kvrpcpb.Context) *Request { - req := NewRequest(typ, pointer, ctxs...) - req.ReplicaRead = replicaReadType.IsFollowerRead() - req.ReplicaReadType = replicaReadType - req.ReplicaReadSeed = replicaReadSeed - return req -} - -// GetReplicaReadSeed returns ReplicaReadSeed pointer. -func (req *Request) GetReplicaReadSeed() *uint32 { - if req != nil { - return req.ReplicaReadSeed - } - return nil -} - -// EnableStaleRead enables stale read -func (req *Request) EnableStaleRead() { - req.StaleRead = true - req.ReplicaReadType = kv.ReplicaReadMixed - req.ReplicaRead = false -} - -// IsDebugReq check whether the req is debug req. -func (req *Request) IsDebugReq() bool { - switch req.Type { - case CmdDebugGetRegionProperties: - return true - } - return false -} - -// Get returns GetRequest in request. -func (req *Request) Get() *kvrpcpb.GetRequest { - return req.Req.(*kvrpcpb.GetRequest) -} - -// Scan returns ScanRequest in request. -func (req *Request) Scan() *kvrpcpb.ScanRequest { - return req.Req.(*kvrpcpb.ScanRequest) -} - -// Prewrite returns PrewriteRequest in request. -func (req *Request) Prewrite() *kvrpcpb.PrewriteRequest { - return req.Req.(*kvrpcpb.PrewriteRequest) -} - -// Commit returns CommitRequest in request. -func (req *Request) Commit() *kvrpcpb.CommitRequest { - return req.Req.(*kvrpcpb.CommitRequest) -} - -// Cleanup returns CleanupRequest in request. -func (req *Request) Cleanup() *kvrpcpb.CleanupRequest { - return req.Req.(*kvrpcpb.CleanupRequest) -} - -// BatchGet returns BatchGetRequest in request. -func (req *Request) BatchGet() *kvrpcpb.BatchGetRequest { - return req.Req.(*kvrpcpb.BatchGetRequest) -} - -// BatchRollback returns BatchRollbackRequest in request. -func (req *Request) BatchRollback() *kvrpcpb.BatchRollbackRequest { - return req.Req.(*kvrpcpb.BatchRollbackRequest) -} - -// ScanLock returns ScanLockRequest in request. -func (req *Request) ScanLock() *kvrpcpb.ScanLockRequest { - return req.Req.(*kvrpcpb.ScanLockRequest) -} - -// ResolveLock returns ResolveLockRequest in request. -func (req *Request) ResolveLock() *kvrpcpb.ResolveLockRequest { - return req.Req.(*kvrpcpb.ResolveLockRequest) -} - -// GC returns GCRequest in request. -func (req *Request) GC() *kvrpcpb.GCRequest { - return req.Req.(*kvrpcpb.GCRequest) -} - -// DeleteRange returns DeleteRangeRequest in request. -func (req *Request) DeleteRange() *kvrpcpb.DeleteRangeRequest { - return req.Req.(*kvrpcpb.DeleteRangeRequest) -} - -// RawGet returns RawGetRequest in request. -func (req *Request) RawGet() *kvrpcpb.RawGetRequest { - return req.Req.(*kvrpcpb.RawGetRequest) -} - -// RawBatchGet returns RawBatchGetRequest in request. -func (req *Request) RawBatchGet() *kvrpcpb.RawBatchGetRequest { - return req.Req.(*kvrpcpb.RawBatchGetRequest) -} - -// RawPut returns RawPutRequest in request. -func (req *Request) RawPut() *kvrpcpb.RawPutRequest { - return req.Req.(*kvrpcpb.RawPutRequest) -} - -// RawBatchPut returns RawBatchPutRequest in request. -func (req *Request) RawBatchPut() *kvrpcpb.RawBatchPutRequest { - return req.Req.(*kvrpcpb.RawBatchPutRequest) -} - -// RawDelete returns PrewriteRequest in request. -func (req *Request) RawDelete() *kvrpcpb.RawDeleteRequest { - return req.Req.(*kvrpcpb.RawDeleteRequest) -} - -// RawBatchDelete returns RawBatchDeleteRequest in request. -func (req *Request) RawBatchDelete() *kvrpcpb.RawBatchDeleteRequest { - return req.Req.(*kvrpcpb.RawBatchDeleteRequest) -} - -// RawDeleteRange returns RawDeleteRangeRequest in request. -func (req *Request) RawDeleteRange() *kvrpcpb.RawDeleteRangeRequest { - return req.Req.(*kvrpcpb.RawDeleteRangeRequest) -} - -// RawScan returns RawScanRequest in request. -func (req *Request) RawScan() *kvrpcpb.RawScanRequest { - return req.Req.(*kvrpcpb.RawScanRequest) -} - -// UnsafeDestroyRange returns UnsafeDestroyRangeRequest in request. -func (req *Request) UnsafeDestroyRange() *kvrpcpb.UnsafeDestroyRangeRequest { - return req.Req.(*kvrpcpb.UnsafeDestroyRangeRequest) -} - -// RegisterLockObserver returns RegisterLockObserverRequest in request. -func (req *Request) RegisterLockObserver() *kvrpcpb.RegisterLockObserverRequest { - return req.Req.(*kvrpcpb.RegisterLockObserverRequest) -} - -// CheckLockObserver returns CheckLockObserverRequest in request. -func (req *Request) CheckLockObserver() *kvrpcpb.CheckLockObserverRequest { - return req.Req.(*kvrpcpb.CheckLockObserverRequest) -} - -// RemoveLockObserver returns RemoveLockObserverRequest in request. -func (req *Request) RemoveLockObserver() *kvrpcpb.RemoveLockObserverRequest { - return req.Req.(*kvrpcpb.RemoveLockObserverRequest) -} - -// PhysicalScanLock returns PhysicalScanLockRequest in request. -func (req *Request) PhysicalScanLock() *kvrpcpb.PhysicalScanLockRequest { - return req.Req.(*kvrpcpb.PhysicalScanLockRequest) -} - -// Cop returns coprocessor request in request. -func (req *Request) Cop() *coprocessor.Request { - return req.Req.(*coprocessor.Request) -} - -// BatchCop returns BatchCop request in request. -func (req *Request) BatchCop() *coprocessor.BatchRequest { - return req.Req.(*coprocessor.BatchRequest) -} - -// DispatchMPPTask returns dispatch task request in request. -func (req *Request) DispatchMPPTask() *mpp.DispatchTaskRequest { - return req.Req.(*mpp.DispatchTaskRequest) -} - -// EstablishMPPConn returns EstablishMPPConnectionRequest in request. -func (req *Request) EstablishMPPConn() *mpp.EstablishMPPConnectionRequest { - return req.Req.(*mpp.EstablishMPPConnectionRequest) -} - -// CancelMPPTask returns canceling task in request -func (req *Request) CancelMPPTask() *mpp.CancelTaskRequest { - return req.Req.(*mpp.CancelTaskRequest) -} - -// MvccGetByKey returns MvccGetByKeyRequest in request. -func (req *Request) MvccGetByKey() *kvrpcpb.MvccGetByKeyRequest { - return req.Req.(*kvrpcpb.MvccGetByKeyRequest) -} - -// MvccGetByStartTs returns MvccGetByStartTsRequest in request. -func (req *Request) MvccGetByStartTs() *kvrpcpb.MvccGetByStartTsRequest { - return req.Req.(*kvrpcpb.MvccGetByStartTsRequest) -} - -// SplitRegion returns SplitRegionRequest in request. -func (req *Request) SplitRegion() *kvrpcpb.SplitRegionRequest { - return req.Req.(*kvrpcpb.SplitRegionRequest) -} - -// PessimisticLock returns PessimisticLockRequest in request. -func (req *Request) PessimisticLock() *kvrpcpb.PessimisticLockRequest { - return req.Req.(*kvrpcpb.PessimisticLockRequest) -} - -// PessimisticRollback returns PessimisticRollbackRequest in request. -func (req *Request) PessimisticRollback() *kvrpcpb.PessimisticRollbackRequest { - return req.Req.(*kvrpcpb.PessimisticRollbackRequest) -} - -// DebugGetRegionProperties returns GetRegionPropertiesRequest in request. -func (req *Request) DebugGetRegionProperties() *debugpb.GetRegionPropertiesRequest { - return req.Req.(*debugpb.GetRegionPropertiesRequest) -} - -// Empty returns BatchCommandsEmptyRequest in request. -func (req *Request) Empty() *tikvpb.BatchCommandsEmptyRequest { - return req.Req.(*tikvpb.BatchCommandsEmptyRequest) -} - -// CheckTxnStatus returns CheckTxnStatusRequest in request. -func (req *Request) CheckTxnStatus() *kvrpcpb.CheckTxnStatusRequest { - return req.Req.(*kvrpcpb.CheckTxnStatusRequest) -} - -// CheckSecondaryLocks returns CheckSecondaryLocksRequest in request. -func (req *Request) CheckSecondaryLocks() *kvrpcpb.CheckSecondaryLocksRequest { - return req.Req.(*kvrpcpb.CheckSecondaryLocksRequest) -} - -// TxnHeartBeat returns TxnHeartBeatRequest in request. -func (req *Request) TxnHeartBeat() *kvrpcpb.TxnHeartBeatRequest { - return req.Req.(*kvrpcpb.TxnHeartBeatRequest) -} - -// StoreSafeTS returns StoreSafeTSRequest in request. -func (req *Request) StoreSafeTS() *kvrpcpb.StoreSafeTSRequest { - return req.Req.(*kvrpcpb.StoreSafeTSRequest) -} - -// LockWaitInfo returns GetLockWaitInfoRequest in request. -func (req *Request) LockWaitInfo() *kvrpcpb.GetLockWaitInfoRequest { - return req.Req.(*kvrpcpb.GetLockWaitInfoRequest) -} - -// ToBatchCommandsRequest converts the request to an entry in BatchCommands request. -func (req *Request) ToBatchCommandsRequest() *tikvpb.BatchCommandsRequest_Request { - switch req.Type { - case CmdGet: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Get{Get: req.Get()}} - case CmdScan: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Scan{Scan: req.Scan()}} - case CmdPrewrite: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Prewrite{Prewrite: req.Prewrite()}} - case CmdCommit: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Commit{Commit: req.Commit()}} - case CmdCleanup: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Cleanup{Cleanup: req.Cleanup()}} - case CmdBatchGet: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_BatchGet{BatchGet: req.BatchGet()}} - case CmdBatchRollback: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_BatchRollback{BatchRollback: req.BatchRollback()}} - case CmdScanLock: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_ScanLock{ScanLock: req.ScanLock()}} - case CmdResolveLock: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_ResolveLock{ResolveLock: req.ResolveLock()}} - case CmdGC: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_GC{GC: req.GC()}} - case CmdDeleteRange: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_DeleteRange{DeleteRange: req.DeleteRange()}} - case CmdRawGet: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawGet{RawGet: req.RawGet()}} - case CmdRawBatchGet: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawBatchGet{RawBatchGet: req.RawBatchGet()}} - case CmdRawPut: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawPut{RawPut: req.RawPut()}} - case CmdRawBatchPut: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawBatchPut{RawBatchPut: req.RawBatchPut()}} - case CmdRawDelete: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawDelete{RawDelete: req.RawDelete()}} - case CmdRawBatchDelete: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawBatchDelete{RawBatchDelete: req.RawBatchDelete()}} - case CmdRawDeleteRange: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawDeleteRange{RawDeleteRange: req.RawDeleteRange()}} - case CmdRawScan: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_RawScan{RawScan: req.RawScan()}} - case CmdCop: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Coprocessor{Coprocessor: req.Cop()}} - case CmdPessimisticLock: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_PessimisticLock{PessimisticLock: req.PessimisticLock()}} - case CmdPessimisticRollback: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_PessimisticRollback{PessimisticRollback: req.PessimisticRollback()}} - case CmdEmpty: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_Empty{Empty: req.Empty()}} - case CmdCheckTxnStatus: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_CheckTxnStatus{CheckTxnStatus: req.CheckTxnStatus()}} - case CmdCheckSecondaryLocks: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_CheckSecondaryLocks{CheckSecondaryLocks: req.CheckSecondaryLocks()}} - case CmdTxnHeartBeat: - return &tikvpb.BatchCommandsRequest_Request{Cmd: &tikvpb.BatchCommandsRequest_Request_TxnHeartBeat{TxnHeartBeat: req.TxnHeartBeat()}} - } - return nil -} - -// Response wraps all kv/coprocessor responses. -type Response struct { - Resp interface{} -} - -// FromBatchCommandsResponse converts a BatchCommands response to Response. -func FromBatchCommandsResponse(res *tikvpb.BatchCommandsResponse_Response) (*Response, error) { - if res.GetCmd() == nil { - return nil, errors.New("Unknown command response") - } - switch res := res.GetCmd().(type) { - case *tikvpb.BatchCommandsResponse_Response_Get: - return &Response{Resp: res.Get}, nil - case *tikvpb.BatchCommandsResponse_Response_Scan: - return &Response{Resp: res.Scan}, nil - case *tikvpb.BatchCommandsResponse_Response_Prewrite: - return &Response{Resp: res.Prewrite}, nil - case *tikvpb.BatchCommandsResponse_Response_Commit: - return &Response{Resp: res.Commit}, nil - case *tikvpb.BatchCommandsResponse_Response_Cleanup: - return &Response{Resp: res.Cleanup}, nil - case *tikvpb.BatchCommandsResponse_Response_BatchGet: - return &Response{Resp: res.BatchGet}, nil - case *tikvpb.BatchCommandsResponse_Response_BatchRollback: - return &Response{Resp: res.BatchRollback}, nil - case *tikvpb.BatchCommandsResponse_Response_ScanLock: - return &Response{Resp: res.ScanLock}, nil - case *tikvpb.BatchCommandsResponse_Response_ResolveLock: - return &Response{Resp: res.ResolveLock}, nil - case *tikvpb.BatchCommandsResponse_Response_GC: - return &Response{Resp: res.GC}, nil - case *tikvpb.BatchCommandsResponse_Response_DeleteRange: - return &Response{Resp: res.DeleteRange}, nil - case *tikvpb.BatchCommandsResponse_Response_RawGet: - return &Response{Resp: res.RawGet}, nil - case *tikvpb.BatchCommandsResponse_Response_RawBatchGet: - return &Response{Resp: res.RawBatchGet}, nil - case *tikvpb.BatchCommandsResponse_Response_RawPut: - return &Response{Resp: res.RawPut}, nil - case *tikvpb.BatchCommandsResponse_Response_RawBatchPut: - return &Response{Resp: res.RawBatchPut}, nil - case *tikvpb.BatchCommandsResponse_Response_RawDelete: - return &Response{Resp: res.RawDelete}, nil - case *tikvpb.BatchCommandsResponse_Response_RawBatchDelete: - return &Response{Resp: res.RawBatchDelete}, nil - case *tikvpb.BatchCommandsResponse_Response_RawDeleteRange: - return &Response{Resp: res.RawDeleteRange}, nil - case *tikvpb.BatchCommandsResponse_Response_RawScan: - return &Response{Resp: res.RawScan}, nil - case *tikvpb.BatchCommandsResponse_Response_Coprocessor: - return &Response{Resp: res.Coprocessor}, nil - case *tikvpb.BatchCommandsResponse_Response_PessimisticLock: - return &Response{Resp: res.PessimisticLock}, nil - case *tikvpb.BatchCommandsResponse_Response_PessimisticRollback: - return &Response{Resp: res.PessimisticRollback}, nil - case *tikvpb.BatchCommandsResponse_Response_Empty: - return &Response{Resp: res.Empty}, nil - case *tikvpb.BatchCommandsResponse_Response_TxnHeartBeat: - return &Response{Resp: res.TxnHeartBeat}, nil - case *tikvpb.BatchCommandsResponse_Response_CheckTxnStatus: - return &Response{Resp: res.CheckTxnStatus}, nil - case *tikvpb.BatchCommandsResponse_Response_CheckSecondaryLocks: - return &Response{Resp: res.CheckSecondaryLocks}, nil - } - panic("unreachable") -} - -// CopStreamResponse combines tikvpb.Tikv_CoprocessorStreamClient and the first Recv() result together. -// In streaming API, get grpc stream client may not involve any network packet, then region error have -// to be handled in Recv() function. This struct facilitates the error handling. -type CopStreamResponse struct { - tikvpb.Tikv_CoprocessorStreamClient - *coprocessor.Response // The first result of Recv() - Timeout time.Duration - Lease // Shared by this object and a background goroutine. -} - -// BatchCopStreamResponse comprises the BatchCoprocessorClient , the first result and timeout detector. -type BatchCopStreamResponse struct { - tikvpb.Tikv_BatchCoprocessorClient - *coprocessor.BatchResponse - Timeout time.Duration - Lease // Shared by this object and a background goroutine. -} - -// MPPStreamResponse is indeed a wrapped client that can receive data packet from tiflash mpp server. -type MPPStreamResponse struct { - tikvpb.Tikv_EstablishMPPConnectionClient - *mpp.MPPDataPacket - Timeout time.Duration - Lease -} - -// SetContext set the Context field for the given req to the specified ctx. -func SetContext(req *Request, region *metapb.Region, peer *metapb.Peer) error { - ctx := &req.Context - if region != nil { - ctx.RegionId = region.Id - ctx.RegionEpoch = region.RegionEpoch - } - ctx.Peer = peer - - switch req.Type { - case CmdGet: - req.Get().Context = ctx - case CmdScan: - req.Scan().Context = ctx - case CmdPrewrite: - req.Prewrite().Context = ctx - case CmdPessimisticLock: - req.PessimisticLock().Context = ctx - case CmdPessimisticRollback: - req.PessimisticRollback().Context = ctx - case CmdCommit: - req.Commit().Context = ctx - case CmdCleanup: - req.Cleanup().Context = ctx - case CmdBatchGet: - req.BatchGet().Context = ctx - case CmdBatchRollback: - req.BatchRollback().Context = ctx - case CmdScanLock: - req.ScanLock().Context = ctx - case CmdResolveLock: - req.ResolveLock().Context = ctx - case CmdGC: - req.GC().Context = ctx - case CmdDeleteRange: - req.DeleteRange().Context = ctx - case CmdRawGet: - req.RawGet().Context = ctx - case CmdRawBatchGet: - req.RawBatchGet().Context = ctx - case CmdRawPut: - req.RawPut().Context = ctx - case CmdRawBatchPut: - req.RawBatchPut().Context = ctx - case CmdRawDelete: - req.RawDelete().Context = ctx - case CmdRawBatchDelete: - req.RawBatchDelete().Context = ctx - case CmdRawDeleteRange: - req.RawDeleteRange().Context = ctx - case CmdRawScan: - req.RawScan().Context = ctx - case CmdUnsafeDestroyRange: - req.UnsafeDestroyRange().Context = ctx - case CmdRegisterLockObserver: - req.RegisterLockObserver().Context = ctx - case CmdCheckLockObserver: - req.CheckLockObserver().Context = ctx - case CmdRemoveLockObserver: - req.RemoveLockObserver().Context = ctx - case CmdPhysicalScanLock: - req.PhysicalScanLock().Context = ctx - case CmdCop: - req.Cop().Context = ctx - case CmdCopStream: - req.Cop().Context = ctx - case CmdBatchCop: - req.BatchCop().Context = ctx - case CmdMPPTask: - // Dispatching MPP tasks don't need a region context, because it's a request for store but not region. - case CmdMvccGetByKey: - req.MvccGetByKey().Context = ctx - case CmdMvccGetByStartTs: - req.MvccGetByStartTs().Context = ctx - case CmdSplitRegion: - req.SplitRegion().Context = ctx - case CmdEmpty: - req.SplitRegion().Context = ctx - case CmdTxnHeartBeat: - req.TxnHeartBeat().Context = ctx - case CmdCheckTxnStatus: - req.CheckTxnStatus().Context = ctx - case CmdCheckSecondaryLocks: - req.CheckSecondaryLocks().Context = ctx - default: - return fmt.Errorf("invalid request type %v", req.Type) - } - return nil -} - -// GenRegionErrorResp returns corresponding Response with specified RegionError -// according to the given req. -func GenRegionErrorResp(req *Request, e *errorpb.Error) (*Response, error) { - var p interface{} - resp := &Response{} - switch req.Type { - case CmdGet: - p = &kvrpcpb.GetResponse{ - RegionError: e, - } - case CmdScan: - p = &kvrpcpb.ScanResponse{ - RegionError: e, - } - case CmdPrewrite: - p = &kvrpcpb.PrewriteResponse{ - RegionError: e, - } - case CmdPessimisticLock: - p = &kvrpcpb.PessimisticLockResponse{ - RegionError: e, - } - case CmdPessimisticRollback: - p = &kvrpcpb.PessimisticRollbackResponse{ - RegionError: e, - } - case CmdCommit: - p = &kvrpcpb.CommitResponse{ - RegionError: e, - } - case CmdCleanup: - p = &kvrpcpb.CleanupResponse{ - RegionError: e, - } - case CmdBatchGet: - p = &kvrpcpb.BatchGetResponse{ - RegionError: e, - } - case CmdBatchRollback: - p = &kvrpcpb.BatchRollbackResponse{ - RegionError: e, - } - case CmdScanLock: - p = &kvrpcpb.ScanLockResponse{ - RegionError: e, - } - case CmdResolveLock: - p = &kvrpcpb.ResolveLockResponse{ - RegionError: e, - } - case CmdGC: - p = &kvrpcpb.GCResponse{ - RegionError: e, - } - case CmdDeleteRange: - p = &kvrpcpb.DeleteRangeResponse{ - RegionError: e, - } - case CmdRawGet: - p = &kvrpcpb.RawGetResponse{ - RegionError: e, - } - case CmdRawBatchGet: - p = &kvrpcpb.RawBatchGetResponse{ - RegionError: e, - } - case CmdRawPut: - p = &kvrpcpb.RawPutResponse{ - RegionError: e, - } - case CmdRawBatchPut: - p = &kvrpcpb.RawBatchPutResponse{ - RegionError: e, - } - case CmdRawDelete: - p = &kvrpcpb.RawDeleteResponse{ - RegionError: e, - } - case CmdRawBatchDelete: - p = &kvrpcpb.RawBatchDeleteResponse{ - RegionError: e, - } - case CmdRawDeleteRange: - p = &kvrpcpb.RawDeleteRangeResponse{ - RegionError: e, - } - case CmdRawScan: - p = &kvrpcpb.RawScanResponse{ - RegionError: e, - } - case CmdUnsafeDestroyRange: - p = &kvrpcpb.UnsafeDestroyRangeResponse{ - RegionError: e, - } - case CmdCop: - p = &coprocessor.Response{ - RegionError: e, - } - case CmdCopStream: - p = &CopStreamResponse{ - Response: &coprocessor.Response{ - RegionError: e, - }, - } - case CmdMvccGetByKey: - p = &kvrpcpb.MvccGetByKeyResponse{ - RegionError: e, - } - case CmdMvccGetByStartTs: - p = &kvrpcpb.MvccGetByStartTsResponse{ - RegionError: e, - } - case CmdSplitRegion: - p = &kvrpcpb.SplitRegionResponse{ - RegionError: e, - } - case CmdEmpty: - case CmdTxnHeartBeat: - p = &kvrpcpb.TxnHeartBeatResponse{ - RegionError: e, - } - case CmdCheckTxnStatus: - p = &kvrpcpb.CheckTxnStatusResponse{ - RegionError: e, - } - case CmdCheckSecondaryLocks: - p = &kvrpcpb.CheckSecondaryLocksResponse{ - RegionError: e, - } - default: - return nil, fmt.Errorf("invalid request type %v", req.Type) - } - resp.Resp = p - return resp, nil -} - -type getRegionError interface { - GetRegionError() *errorpb.Error -} - -// GetRegionError returns the RegionError of the underlying concrete response. -func (resp *Response) GetRegionError() (*errorpb.Error, error) { - if resp.Resp == nil { - return nil, nil - } - err, ok := resp.Resp.(getRegionError) - if !ok { - if _, isEmpty := resp.Resp.(*tikvpb.BatchCommandsEmptyResponse); isEmpty { - return nil, nil - } - return nil, fmt.Errorf("invalid response type %v", resp) - } - return err.GetRegionError(), nil -} - -// CallRPC launches a rpc call. -// ch is needed to implement timeout for coprocessor streaming, the stream object's -// cancel function will be sent to the channel, together with a lease checked by a background goroutine. -func CallRPC(ctx context.Context, client tikvpb.TikvClient, req *Request) (*Response, error) { - resp := &Response{} - var err error - switch req.Type { - case CmdGet: - resp.Resp, err = client.KvGet(ctx, req.Get()) - case CmdScan: - resp.Resp, err = client.KvScan(ctx, req.Scan()) - case CmdPrewrite: - resp.Resp, err = client.KvPrewrite(ctx, req.Prewrite()) - case CmdPessimisticLock: - resp.Resp, err = client.KvPessimisticLock(ctx, req.PessimisticLock()) - case CmdPessimisticRollback: - resp.Resp, err = client.KVPessimisticRollback(ctx, req.PessimisticRollback()) - case CmdCommit: - resp.Resp, err = client.KvCommit(ctx, req.Commit()) - case CmdCleanup: - resp.Resp, err = client.KvCleanup(ctx, req.Cleanup()) - case CmdBatchGet: - resp.Resp, err = client.KvBatchGet(ctx, req.BatchGet()) - case CmdBatchRollback: - resp.Resp, err = client.KvBatchRollback(ctx, req.BatchRollback()) - case CmdScanLock: - resp.Resp, err = client.KvScanLock(ctx, req.ScanLock()) - case CmdResolveLock: - resp.Resp, err = client.KvResolveLock(ctx, req.ResolveLock()) - case CmdGC: - resp.Resp, err = client.KvGC(ctx, req.GC()) - case CmdDeleteRange: - resp.Resp, err = client.KvDeleteRange(ctx, req.DeleteRange()) - case CmdRawGet: - resp.Resp, err = client.RawGet(ctx, req.RawGet()) - case CmdRawBatchGet: - resp.Resp, err = client.RawBatchGet(ctx, req.RawBatchGet()) - case CmdRawPut: - resp.Resp, err = client.RawPut(ctx, req.RawPut()) - case CmdRawBatchPut: - resp.Resp, err = client.RawBatchPut(ctx, req.RawBatchPut()) - case CmdRawDelete: - resp.Resp, err = client.RawDelete(ctx, req.RawDelete()) - case CmdRawBatchDelete: - resp.Resp, err = client.RawBatchDelete(ctx, req.RawBatchDelete()) - case CmdRawDeleteRange: - resp.Resp, err = client.RawDeleteRange(ctx, req.RawDeleteRange()) - case CmdRawScan: - resp.Resp, err = client.RawScan(ctx, req.RawScan()) - case CmdUnsafeDestroyRange: - resp.Resp, err = client.UnsafeDestroyRange(ctx, req.UnsafeDestroyRange()) - case CmdRegisterLockObserver: - resp.Resp, err = client.RegisterLockObserver(ctx, req.RegisterLockObserver()) - case CmdCheckLockObserver: - resp.Resp, err = client.CheckLockObserver(ctx, req.CheckLockObserver()) - case CmdRemoveLockObserver: - resp.Resp, err = client.RemoveLockObserver(ctx, req.RemoveLockObserver()) - case CmdPhysicalScanLock: - resp.Resp, err = client.PhysicalScanLock(ctx, req.PhysicalScanLock()) - case CmdCop: - resp.Resp, err = client.Coprocessor(ctx, req.Cop()) - case CmdMPPTask: - resp.Resp, err = client.DispatchMPPTask(ctx, req.DispatchMPPTask()) - case CmdMPPConn: - var streamClient tikvpb.Tikv_EstablishMPPConnectionClient - streamClient, err = client.EstablishMPPConnection(ctx, req.EstablishMPPConn()) - resp.Resp = &MPPStreamResponse{ - Tikv_EstablishMPPConnectionClient: streamClient, - } - case CmdMPPCancel: - // it cannot use the ctx with cancel(), otherwise this cmd will fail. - resp.Resp, err = client.CancelMPPTask(ctx, req.CancelMPPTask()) - case CmdCopStream: - var streamClient tikvpb.Tikv_CoprocessorStreamClient - streamClient, err = client.CoprocessorStream(ctx, req.Cop()) - resp.Resp = &CopStreamResponse{ - Tikv_CoprocessorStreamClient: streamClient, - } - case CmdBatchCop: - var streamClient tikvpb.Tikv_BatchCoprocessorClient - streamClient, err = client.BatchCoprocessor(ctx, req.BatchCop()) - resp.Resp = &BatchCopStreamResponse{ - Tikv_BatchCoprocessorClient: streamClient, - } - case CmdMvccGetByKey: - resp.Resp, err = client.MvccGetByKey(ctx, req.MvccGetByKey()) - case CmdMvccGetByStartTs: - resp.Resp, err = client.MvccGetByStartTs(ctx, req.MvccGetByStartTs()) - case CmdSplitRegion: - resp.Resp, err = client.SplitRegion(ctx, req.SplitRegion()) - case CmdEmpty: - resp.Resp, err = &tikvpb.BatchCommandsEmptyResponse{}, nil - case CmdCheckTxnStatus: - resp.Resp, err = client.KvCheckTxnStatus(ctx, req.CheckTxnStatus()) - case CmdCheckSecondaryLocks: - resp.Resp, err = client.KvCheckSecondaryLocks(ctx, req.CheckSecondaryLocks()) - case CmdTxnHeartBeat: - resp.Resp, err = client.KvTxnHeartBeat(ctx, req.TxnHeartBeat()) - case CmdStoreSafeTS: - resp.Resp, err = client.GetStoreSafeTS(ctx, req.StoreSafeTS()) - case CmdLockWaitInfo: - resp.Resp, err = client.GetLockWaitInfo(ctx, req.LockWaitInfo()) - default: - return nil, errors.Errorf("invalid request type: %v", req.Type) - } - if err != nil { - return nil, errors.Trace(err) - } - return resp, nil -} - -// CallDebugRPC launches a debug rpc call. -func CallDebugRPC(ctx context.Context, client debugpb.DebugClient, req *Request) (*Response, error) { - resp := &Response{} - var err error - switch req.Type { - case CmdDebugGetRegionProperties: - resp.Resp, err = client.GetRegionProperties(ctx, req.DebugGetRegionProperties()) - default: - return nil, errors.Errorf("invalid request type: %v", req.Type) - } - return resp, err -} - -// Lease is used to implement grpc stream timeout. -type Lease struct { - Cancel context.CancelFunc - deadline int64 // A time.UnixNano value, if time.Now().UnixNano() > deadline, cancel() would be called. -} - -// Recv overrides the stream client Recv() function. -func (resp *CopStreamResponse) Recv() (*coprocessor.Response, error) { - deadline := time.Now().Add(resp.Timeout).UnixNano() - atomic.StoreInt64(&resp.Lease.deadline, deadline) - - ret, err := resp.Tikv_CoprocessorStreamClient.Recv() - - atomic.StoreInt64(&resp.Lease.deadline, 0) // Stop the lease check. - return ret, errors.Trace(err) -} - -// Close closes the CopStreamResponse object. -func (resp *CopStreamResponse) Close() { - atomic.StoreInt64(&resp.Lease.deadline, 1) - // We also call cancel here because CheckStreamTimeoutLoop - // is not guaranteed to cancel all items when it exits. - if resp.Lease.Cancel != nil { - resp.Lease.Cancel() - } -} - -// Recv overrides the stream client Recv() function. -func (resp *BatchCopStreamResponse) Recv() (*coprocessor.BatchResponse, error) { - deadline := time.Now().Add(resp.Timeout).UnixNano() - atomic.StoreInt64(&resp.Lease.deadline, deadline) - - ret, err := resp.Tikv_BatchCoprocessorClient.Recv() - - atomic.StoreInt64(&resp.Lease.deadline, 0) // Stop the lease check. - return ret, errors.Trace(err) -} - -// Close closes the BatchCopStreamResponse object. -func (resp *BatchCopStreamResponse) Close() { - atomic.StoreInt64(&resp.Lease.deadline, 1) - // We also call cancel here because CheckStreamTimeoutLoop - // is not guaranteed to cancel all items when it exits. - if resp.Lease.Cancel != nil { - resp.Lease.Cancel() - } -} - -// Recv overrides the stream client Recv() function. -func (resp *MPPStreamResponse) Recv() (*mpp.MPPDataPacket, error) { - deadline := time.Now().Add(resp.Timeout).UnixNano() - atomic.StoreInt64(&resp.Lease.deadline, deadline) - - ret, err := resp.Tikv_EstablishMPPConnectionClient.Recv() - - atomic.StoreInt64(&resp.Lease.deadline, 0) // Stop the lease check. - return ret, errors.Trace(err) -} - -// Close closes the MPPStreamResponse object. -func (resp *MPPStreamResponse) Close() { - atomic.StoreInt64(&resp.Lease.deadline, 1) - // We also call cancel here because CheckStreamTimeoutLoop - // is not guaranteed to cancel all items when it exits. - if resp.Lease.Cancel != nil { - resp.Lease.Cancel() - } -} - -// CheckStreamTimeoutLoop runs periodically to check is there any stream request timed out. -// Lease is an object to track stream requests, call this function with "go CheckStreamTimeoutLoop()" -// It is not guaranteed to call every Lease.Cancel() putting into channel when exits. -// If grpc-go supports SetDeadline(https://github.com/grpc/grpc-go/issues/2917), we can stop using this method. -func CheckStreamTimeoutLoop(ch <-chan *Lease, done <-chan struct{}) { - ticker := time.NewTicker(200 * time.Millisecond) - defer ticker.Stop() - array := make([]*Lease, 0, 1024) - - for { - select { - case <-done: - drainLoop: - // Try my best cleaning the channel to make SendRequest which is blocking by it continues. - for { - select { - case <-ch: - default: - break drainLoop - } - } - return - case item := <-ch: - array = append(array, item) - case now := <-ticker.C: - array = keepOnlyActive(array, now.UnixNano()) - } - } -} - -// keepOnlyActive removes completed items, call cancel function for timeout items. -func keepOnlyActive(array []*Lease, now int64) []*Lease { - idx := 0 - for i := 0; i < len(array); i++ { - item := array[i] - deadline := atomic.LoadInt64(&item.deadline) - if deadline == 0 || deadline > now { - array[idx] = array[i] - idx++ - } else { - item.Cancel() - } - } - return array[:idx] -} - -// IsGreenGCRequest checks if the request is used by Green GC's protocol. This is used for failpoints to inject errors -// to specified RPC requests. -func (req *Request) IsGreenGCRequest() bool { - if req.Type == CmdCheckLockObserver || - req.Type == CmdRegisterLockObserver || - req.Type == CmdRemoveLockObserver || - req.Type == CmdPhysicalScanLock { - return true - } - return false -} - -// IsTxnWriteRequest checks if the request is a transactional write request. This is used for failpoints to inject -// errors to specified RPC requests. -func (req *Request) IsTxnWriteRequest() bool { - if req.Type == CmdPessimisticLock || - req.Type == CmdPrewrite || - req.Type == CmdCommit || - req.Type == CmdBatchRollback || - req.Type == CmdPessimisticRollback || - req.Type == CmdCheckTxnStatus || - req.Type == CmdCheckSecondaryLocks || - req.Type == CmdCleanup || - req.Type == CmdTxnHeartBeat || - req.Type == CmdResolveLock { - return true - } - return false -} diff --git a/store/tikv/tikvrpc/tikvrpc_test.go b/store/tikv/tikvrpc/tikvrpc_test.go deleted file mode 100644 index 4ac6cba2e7143..0000000000000 --- a/store/tikv/tikvrpc/tikvrpc_test.go +++ /dev/null @@ -1,36 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikvrpc - -import ( - "testing" - - . "github.com/pingcap/check" - "github.com/pingcap/kvproto/pkg/tikvpb" -) - -func TestT(t *testing.T) { - TestingT(t) -} - -type testBatchCommand struct{} - -var _ = Suite(&testBatchCommand{}) - -func (s *testBatchCommand) TestBatchResponse(c *C) { - resp := &tikvpb.BatchCommandsResponse_Response{} - batchResp, err := FromBatchCommandsResponse(resp) - c.Assert(batchResp == nil, IsTrue) - c.Assert(err != nil, IsTrue) -} diff --git a/store/tikv/txn.go b/store/tikv/txn.go deleted file mode 100644 index 9785dfeedc814..0000000000000 --- a/store/tikv/txn.go +++ /dev/null @@ -1,760 +0,0 @@ -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv - -import ( - "bytes" - "context" - "encoding/json" - "fmt" - "math/rand" - "runtime/trace" - "sort" - "sync" - "sync/atomic" - "time" - - "github.com/dgryski/go-farm" - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - tikverr "github.com/pingcap/tidb/store/tikv/error" - tikv "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/unionstore" - "github.com/pingcap/tidb/store/tikv/util" - "go.uber.org/zap" -) - -// MaxTxnTimeUse is the max time a Txn may use (in ms) from its begin to commit. -// We use it to abort the transaction to guarantee GC worker will not influence it. -const MaxTxnTimeUse = 24 * 60 * 60 * 1000 - -// SchemaAmender is used by pessimistic transactions to amend commit mutations for schema change during 2pc. -type SchemaAmender interface { - // AmendTxn is the amend entry, new mutations will be generated based on input mutations using schema change info. - // The returned results are mutations need to prewrite and mutations need to cleanup. - AmendTxn(ctx context.Context, startInfoSchema SchemaVer, change *RelatedSchemaChange, mutations CommitterMutations) (CommitterMutations, error) -} - -// StartTSOption indicates the option when beginning a transaction -// `TxnScope` must be set for each object -// Every other fields are optional, but currently at most one of them can be set -type StartTSOption struct { - TxnScope string - StartTS *uint64 -} - -// DefaultStartTSOption creates a default StartTSOption, ie. Work in GlobalTxnScope and get start ts when got used -func DefaultStartTSOption() StartTSOption { - return StartTSOption{TxnScope: oracle.GlobalTxnScope} -} - -// SetStartTS returns a new StartTSOption with StartTS set to the given startTS -func (to StartTSOption) SetStartTS(startTS uint64) StartTSOption { - to.StartTS = &startTS - return to -} - -// SetTxnScope returns a new StartTSOption with TxnScope set to txnScope -func (to StartTSOption) SetTxnScope(txnScope string) StartTSOption { - to.TxnScope = txnScope - return to -} - -// KVTxn contains methods to interact with a TiKV transaction. -type KVTxn struct { - snapshot *KVSnapshot - us *unionstore.KVUnionStore - store *KVStore // for connection to region. - startTS uint64 - startTime time.Time // Monotonic timestamp for recording txn time consuming. - commitTS uint64 - mu sync.Mutex // For thread-safe LockKeys function. - setCnt int64 - vars *tikv.Variables - committer *twoPhaseCommitter - lockedCnt int - - valid bool - - // schemaVer is the infoSchema fetched at startTS. - schemaVer SchemaVer - // SchemaAmender is used amend pessimistic txn commit mutations for schema change - schemaAmender SchemaAmender - // commitCallback is called after current transaction gets committed - commitCallback func(info string, err error) - - binlog BinlogExecutor - schemaLeaseChecker SchemaLeaseChecker - syncLog bool - priority Priority - isPessimistic bool - enableAsyncCommit bool - enable1PC bool - causalConsistency bool - scope string - kvFilter KVFilter - resourceGroupTag []byte -} - -// ExtractStartTS use `option` to get the proper startTS for a transaction. -func ExtractStartTS(store *KVStore, option StartTSOption) (uint64, error) { - if option.StartTS != nil { - return *option.StartTS, nil - } - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - return store.getTimestampWithRetry(bo, option.TxnScope) -} - -func newTiKVTxnWithOptions(store *KVStore, options StartTSOption) (*KVTxn, error) { - if options.TxnScope == "" { - options.TxnScope = oracle.GlobalTxnScope - } - startTS, err := ExtractStartTS(store, options) - if err != nil { - return nil, errors.Trace(err) - } - snapshot := newTiKVSnapshot(store, startTS, store.nextReplicaReadSeed()) - newTiKVTxn := &KVTxn{ - snapshot: snapshot, - us: unionstore.NewUnionStore(snapshot), - store: store, - startTS: startTS, - startTime: time.Now(), - valid: true, - vars: tikv.DefaultVars, - scope: options.TxnScope, - } - return newTiKVTxn, nil -} - -// SetSuccess is used to probe if kv variables are set or not. It is ONLY used in test cases. -var SetSuccess = false - -// SetVars sets variables to the transaction. -func (txn *KVTxn) SetVars(vars *tikv.Variables) { - txn.vars = vars - txn.snapshot.vars = vars - if val, err := util.EvalFailpoint("probeSetVars"); err == nil { - if val.(bool) { - SetSuccess = true - } - } -} - -// GetVars gets variables from the transaction. -func (txn *KVTxn) GetVars() *tikv.Variables { - return txn.vars -} - -// Get implements transaction interface. -func (txn *KVTxn) Get(ctx context.Context, k []byte) ([]byte, error) { - ret, err := txn.us.Get(ctx, k) - if tikverr.IsErrNotFound(err) { - return nil, err - } - if err != nil { - return nil, errors.Trace(err) - } - - return ret, nil -} - -// Set sets the value for key k as v into kv store. -// v must NOT be nil or empty, otherwise it returns ErrCannotSetNilValue. -func (txn *KVTxn) Set(k []byte, v []byte) error { - txn.setCnt++ - return txn.us.GetMemBuffer().Set(k, v) -} - -// String implements fmt.Stringer interface. -func (txn *KVTxn) String() string { - return fmt.Sprintf("%d", txn.StartTS()) -} - -// Iter creates an Iterator positioned on the first entry that k <= entry's key. -// If such entry is not found, it returns an invalid Iterator with no error. -// It yields only keys that < upperBound. If upperBound is nil, it means the upperBound is unbounded. -// The Iterator must be Closed after use. -func (txn *KVTxn) Iter(k []byte, upperBound []byte) (unionstore.Iterator, error) { - return txn.us.Iter(k, upperBound) -} - -// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. -func (txn *KVTxn) IterReverse(k []byte) (unionstore.Iterator, error) { - return txn.us.IterReverse(k) -} - -// Delete removes the entry for key k from kv store. -func (txn *KVTxn) Delete(k []byte) error { - return txn.us.GetMemBuffer().Delete(k) -} - -// SetSchemaLeaseChecker sets a hook to check schema version. -func (txn *KVTxn) SetSchemaLeaseChecker(checker SchemaLeaseChecker) { - txn.schemaLeaseChecker = checker -} - -// EnableForceSyncLog indicates tikv to always sync log for the transaction. -func (txn *KVTxn) EnableForceSyncLog() { - txn.syncLog = true -} - -// SetPessimistic indicates if the transaction should use pessimictic lock. -func (txn *KVTxn) SetPessimistic(b bool) { - txn.isPessimistic = b -} - -// SetSchemaVer updates schema version to validate transaction. -func (txn *KVTxn) SetSchemaVer(schemaVer SchemaVer) { - txn.schemaVer = schemaVer -} - -// SetPriority sets the priority for both write and read. -func (txn *KVTxn) SetPriority(pri Priority) { - txn.priority = pri - txn.GetSnapshot().SetPriority(pri) -} - -// SetResourceGroupTag sets the resource tag for both write and read. -func (txn *KVTxn) SetResourceGroupTag(tag []byte) { - txn.resourceGroupTag = tag - txn.GetSnapshot().SetResourceGroupTag(tag) -} - -// SetSchemaAmender sets an amender to update mutations after schema change. -func (txn *KVTxn) SetSchemaAmender(sa SchemaAmender) { - txn.schemaAmender = sa -} - -// SetCommitCallback sets up a function that will be called when the transaction -// is finished. -func (txn *KVTxn) SetCommitCallback(f func(string, error)) { - txn.commitCallback = f -} - -// SetEnableAsyncCommit indicates if the transaction will try to use async commit. -func (txn *KVTxn) SetEnableAsyncCommit(b bool) { - txn.enableAsyncCommit = b -} - -// SetEnable1PC indicates if the transaction will try to use 1 phase commit. -func (txn *KVTxn) SetEnable1PC(b bool) { - txn.enable1PC = b -} - -// SetCausalConsistency indicates if the transaction does not need to -// guarantee linearizability. Default value is false which means -// linearizability is guaranteed. -func (txn *KVTxn) SetCausalConsistency(b bool) { - txn.causalConsistency = b -} - -// SetScope sets the geographical scope of the transaction. -func (txn *KVTxn) SetScope(scope string) { - txn.scope = scope -} - -// SetKVFilter sets the filter to ignore key-values in memory buffer. -func (txn *KVTxn) SetKVFilter(filter KVFilter) { - txn.kvFilter = filter -} - -// IsPessimistic returns true if it is pessimistic. -func (txn *KVTxn) IsPessimistic() bool { - return txn.isPessimistic -} - -// IsCasualConsistency returns if the transaction allows linearizability -// inconsistency. -func (txn *KVTxn) IsCasualConsistency() bool { - return txn.causalConsistency -} - -// GetScope returns the geographical scope of the transaction. -func (txn *KVTxn) GetScope() string { - return txn.scope -} - -// Commit commits the transaction operations to KV store. -func (txn *KVTxn) Commit(ctx context.Context) error { - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("tikvTxn.Commit", opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - defer trace.StartRegion(ctx, "CommitTxn").End() - - if !txn.valid { - return tikverr.ErrInvalidTxn - } - defer txn.close() - - if val, err := util.EvalFailpoint("mockCommitError"); err == nil { - if val.(bool) && IsMockCommitErrorEnable() { - MockCommitErrorDisable() - return errors.New("mock commit error") - } - } - - start := time.Now() - defer func() { metrics.TxnCmdHistogramWithCommit.Observe(time.Since(start).Seconds()) }() - - // sessionID is used for log. - var sessionID uint64 - val := ctx.Value(util.SessionID) - if val != nil { - sessionID = val.(uint64) - } - - var err error - // If the txn use pessimistic lock, committer is initialized. - committer := txn.committer - if committer == nil { - committer, err = newTwoPhaseCommitter(txn, sessionID) - if err != nil { - return errors.Trace(err) - } - txn.committer = committer - } - defer committer.ttlManager.close() - - initRegion := trace.StartRegion(ctx, "InitKeys") - err = committer.initKeysAndMutations() - initRegion.End() - if err != nil { - return errors.Trace(err) - } - if committer.mutations.Len() == 0 { - return nil - } - - defer func() { - detail := committer.getDetail() - detail.Mu.Lock() - metrics.TiKVTxnCommitBackoffSeconds.Observe(float64(detail.Mu.CommitBackoffTime) / float64(time.Second)) - metrics.TiKVTxnCommitBackoffCount.Observe(float64(len(detail.Mu.BackoffTypes))) - detail.Mu.Unlock() - - ctxValue := ctx.Value(util.CommitDetailCtxKey) - if ctxValue != nil { - commitDetail := ctxValue.(**util.CommitDetails) - if *commitDetail != nil { - (*commitDetail).TxnRetry++ - } else { - *commitDetail = detail - } - } - }() - // latches disabled - // pessimistic transaction should also bypass latch. - if txn.store.txnLatches == nil || txn.IsPessimistic() { - err = committer.execute(ctx) - if val == nil || sessionID > 0 { - txn.onCommitted(err) - } - logutil.Logger(ctx).Debug("[kv] txnLatches disabled, 2pc directly", zap.Error(err)) - return errors.Trace(err) - } - - // latches enabled - // for transactions which need to acquire latches - start = time.Now() - lock := txn.store.txnLatches.Lock(committer.startTS, committer.mutations.GetKeys()) - commitDetail := committer.getDetail() - commitDetail.LocalLatchTime = time.Since(start) - if commitDetail.LocalLatchTime > 0 { - metrics.TiKVLocalLatchWaitTimeHistogram.Observe(commitDetail.LocalLatchTime.Seconds()) - } - defer txn.store.txnLatches.UnLock(lock) - if lock.IsStale() { - return &tikverr.ErrWriteConflictInLatch{StartTS: txn.startTS} - } - err = committer.execute(ctx) - if val == nil || sessionID > 0 { - txn.onCommitted(err) - } - if err == nil { - lock.SetCommitTS(committer.commitTS) - } - logutil.Logger(ctx).Debug("[kv] txnLatches enabled while txn retryable", zap.Error(err)) - return errors.Trace(err) -} - -func (txn *KVTxn) close() { - txn.valid = false -} - -// Rollback undoes the transaction operations to KV store. -func (txn *KVTxn) Rollback() error { - if !txn.valid { - return tikverr.ErrInvalidTxn - } - start := time.Now() - // Clean up pessimistic lock. - if txn.IsPessimistic() && txn.committer != nil { - err := txn.rollbackPessimisticLocks() - txn.committer.ttlManager.close() - if err != nil { - logutil.BgLogger().Error(err.Error()) - } - } - txn.close() - logutil.BgLogger().Debug("[kv] rollback txn", zap.Uint64("txnStartTS", txn.StartTS())) - metrics.TxnCmdHistogramWithRollback.Observe(time.Since(start).Seconds()) - return nil -} - -func (txn *KVTxn) rollbackPessimisticLocks() error { - if txn.lockedCnt == 0 { - return nil - } - bo := retry.NewBackofferWithVars(context.Background(), cleanupMaxBackoff, txn.vars) - keys := txn.collectLockedKeys() - return txn.committer.pessimisticRollbackMutations(bo, &PlainMutations{keys: keys}) -} - -func (txn *KVTxn) collectLockedKeys() [][]byte { - keys := make([][]byte, 0, txn.lockedCnt) - buf := txn.GetMemBuffer() - var err error - for it := buf.IterWithFlags(nil, nil); it.Valid(); err = it.Next() { - _ = err - if it.Flags().HasLocked() { - keys = append(keys, it.Key()) - } - } - return keys -} - -// TxnInfo is used to keep track the info of a committed transaction (mainly for diagnosis and testing) -type TxnInfo struct { - TxnScope string `json:"txn_scope"` - StartTS uint64 `json:"start_ts"` - CommitTS uint64 `json:"commit_ts"` - TxnCommitMode string `json:"txn_commit_mode"` - AsyncCommitFallback bool `json:"async_commit_fallback"` - OnePCFallback bool `json:"one_pc_fallback"` - ErrMsg string `json:"error,omitempty"` -} - -func (txn *KVTxn) onCommitted(err error) { - if txn.commitCallback != nil { - isAsyncCommit := txn.committer.isAsyncCommit() - isOnePC := txn.committer.isOnePC() - - commitMode := "2pc" - if isOnePC { - commitMode = "1pc" - } else if isAsyncCommit { - commitMode = "async_commit" - } - - info := TxnInfo{ - TxnScope: txn.GetScope(), - StartTS: txn.startTS, - CommitTS: txn.commitTS, - TxnCommitMode: commitMode, - AsyncCommitFallback: txn.committer.hasTriedAsyncCommit && !isAsyncCommit, - OnePCFallback: txn.committer.hasTriedOnePC && !isOnePC, - } - if err != nil { - info.ErrMsg = err.Error() - } - infoStr, err2 := json.Marshal(info) - _ = err2 - txn.commitCallback(string(infoStr), err) - } -} - -// LockKeys tries to lock the entries with the keys in KV store. -// lockWaitTime in ms, except that kv.LockAlwaysWait(0) means always wait lock, kv.LockNowait(-1) means nowait lock -func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput ...[]byte) error { - // Exclude keys that are already locked. - var err error - keys := make([][]byte, 0, len(keysInput)) - startTime := time.Now() - txn.mu.Lock() - defer txn.mu.Unlock() - defer func() { - metrics.TxnCmdHistogramWithLockKeys.Observe(time.Since(startTime).Seconds()) - if err == nil { - if lockCtx.PessimisticLockWaited != nil { - if atomic.LoadInt32(lockCtx.PessimisticLockWaited) > 0 { - timeWaited := time.Since(lockCtx.WaitStartTime) - atomic.StoreInt64(lockCtx.LockKeysDuration, int64(timeWaited)) - metrics.TiKVPessimisticLockKeysDuration.Observe(timeWaited.Seconds()) - } - } - } - if lockCtx.LockKeysCount != nil { - *lockCtx.LockKeysCount += int32(len(keys)) - } - if lockCtx.Stats != nil { - lockCtx.Stats.TotalTime = time.Since(startTime) - ctxValue := ctx.Value(util.LockKeysDetailCtxKey) - if ctxValue != nil { - lockKeysDetail := ctxValue.(**util.LockKeysDetails) - *lockKeysDetail = lockCtx.Stats - } - } - }() - memBuf := txn.us.GetMemBuffer() - for _, key := range keysInput { - // The value of lockedMap is only used by pessimistic transactions. - var valueExist, locked, checkKeyExists bool - if flags, err := memBuf.GetFlags(key); err == nil { - locked = flags.HasLocked() - valueExist = flags.HasLockedValueExists() - checkKeyExists = flags.HasNeedCheckExists() - } - if !locked { - keys = append(keys, key) - } else if txn.IsPessimistic() { - if checkKeyExists && valueExist { - alreadyExist := kvrpcpb.AlreadyExist{Key: key} - e := &tikverr.ErrKeyExist{AlreadyExist: &alreadyExist} - return txn.committer.extractKeyExistsErr(e) - } - } - if lockCtx.ReturnValues && locked { - // An already locked key can not return values, we add an entry to let the caller get the value - // in other ways. - lockCtx.Values[string(key)] = tikv.ReturnedValue{AlreadyLocked: true} - } - } - if len(keys) == 0 { - return nil - } - keys = deduplicateKeys(keys) - if txn.IsPessimistic() && lockCtx.ForUpdateTS > 0 { - if txn.committer == nil { - // sessionID is used for log. - var sessionID uint64 - var err error - val := ctx.Value(util.SessionID) - if val != nil { - sessionID = val.(uint64) - } - txn.committer, err = newTwoPhaseCommitter(txn, sessionID) - if err != nil { - return err - } - } - var assignedPrimaryKey bool - if txn.committer.primaryKey == nil { - txn.committer.primaryKey = keys[0] - assignedPrimaryKey = true - } - - lockCtx.Stats = &util.LockKeysDetails{ - LockKeys: int32(len(keys)), - } - bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, txn.vars) - txn.committer.forUpdateTS = lockCtx.ForUpdateTS - // If the number of keys greater than 1, it can be on different region, - // concurrently execute on multiple regions may lead to deadlock. - txn.committer.isFirstLock = txn.lockedCnt == 0 && len(keys) == 1 - err = txn.committer.pessimisticLockMutations(bo, lockCtx, &PlainMutations{keys: keys}) - if bo.GetTotalSleep() > 0 { - atomic.AddInt64(&lockCtx.Stats.BackoffTime, int64(bo.GetTotalSleep())*int64(time.Millisecond)) - lockCtx.Stats.Mu.Lock() - lockCtx.Stats.Mu.BackoffTypes = append(lockCtx.Stats.Mu.BackoffTypes, bo.GetTypes()...) - lockCtx.Stats.Mu.Unlock() - } - if lockCtx.Killed != nil { - // If the kill signal is received during waiting for pessimisticLock, - // pessimisticLockKeys would handle the error but it doesn't reset the flag. - // We need to reset the killed flag here. - atomic.CompareAndSwapUint32(lockCtx.Killed, 1, 0) - } - if err != nil { - for _, key := range keys { - if txn.us.HasPresumeKeyNotExists(key) { - txn.us.UnmarkPresumeKeyNotExists(key) - } - } - keyMayBeLocked := !(tikverr.IsErrWriteConflict(err) || tikverr.IsErrKeyExist(err)) - // If there is only 1 key and lock fails, no need to do pessimistic rollback. - if len(keys) > 1 || keyMayBeLocked { - dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock) - if ok && lockCtx.OnDeadlock != nil { - // Call OnDeadlock before pessimistic rollback. - lockCtx.OnDeadlock(dl) - } - wg := txn.asyncPessimisticRollback(ctx, keys) - if ok { - logutil.Logger(ctx).Debug("deadlock error received", zap.Uint64("startTS", txn.startTS), zap.Stringer("deadlockInfo", dl)) - if hashInKeys(dl.DeadlockKeyHash, keys) { - dl.IsRetryable = true - // Wait for the pessimistic rollback to finish before we retry the statement. - wg.Wait() - // Sleep a little, wait for the other transaction that blocked by this transaction to acquire the lock. - time.Sleep(time.Millisecond * 5) - if _, err := util.EvalFailpoint("SingleStmtDeadLockRetrySleep"); err == nil { - time.Sleep(300 * time.Millisecond) - } - } - } - } - if assignedPrimaryKey { - // unset the primary key if we assigned primary key when failed to lock it. - txn.committer.primaryKey = nil - } - return err - } - if assignedPrimaryKey { - txn.committer.ttlManager.run(txn.committer, lockCtx) - } - } - for _, key := range keys { - valExists := tikv.SetKeyLockedValueExists - // PointGet and BatchPointGet will return value in pessimistic lock response, the value may not exist. - // For other lock modes, the locked key values always exist. - if lockCtx.ReturnValues { - val := lockCtx.Values[string(key)] - if len(val.Value) == 0 { - valExists = tikv.SetKeyLockedValueNotExists - } - } - memBuf.UpdateFlags(key, tikv.SetKeyLocked, tikv.DelNeedCheckExists, valExists) - } - txn.lockedCnt += len(keys) - return nil -} - -// deduplicateKeys deduplicate the keys, it use sort instead of map to avoid memory allocation. -func deduplicateKeys(keys [][]byte) [][]byte { - sort.Slice(keys, func(i, j int) bool { - return bytes.Compare(keys[i], keys[j]) < 0 - }) - deduped := keys[:1] - for i := 1; i < len(keys); i++ { - if !bytes.Equal(deduped[len(deduped)-1], keys[i]) { - deduped = append(deduped, keys[i]) - } - } - return deduped -} - -const pessimisticRollbackMaxBackoff = 20000 - -func (txn *KVTxn) asyncPessimisticRollback(ctx context.Context, keys [][]byte) *sync.WaitGroup { - // Clone a new committer for execute in background. - committer := &twoPhaseCommitter{ - store: txn.committer.store, - sessionID: txn.committer.sessionID, - startTS: txn.committer.startTS, - forUpdateTS: txn.committer.forUpdateTS, - primaryKey: txn.committer.primaryKey, - } - wg := new(sync.WaitGroup) - wg.Add(1) - go func() { - if val, err := util.EvalFailpoint("beforeAsyncPessimisticRollback"); err == nil { - if s, ok := val.(string); ok { - if s == "skip" { - logutil.Logger(ctx).Info("[failpoint] injected skip async pessimistic rollback", - zap.Uint64("txnStartTS", txn.startTS)) - wg.Done() - return - } else if s == "delay" { - duration := time.Duration(rand.Int63n(int64(time.Second) * 2)) - logutil.Logger(ctx).Info("[failpoint] injected delay before async pessimistic rollback", - zap.Uint64("txnStartTS", txn.startTS), zap.Duration("duration", duration)) - time.Sleep(duration) - } - } - } - - err := committer.pessimisticRollbackMutations(retry.NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, txn.vars), &PlainMutations{keys: keys}) - if err != nil { - logutil.Logger(ctx).Warn("[kv] pessimisticRollback failed.", zap.Error(err)) - } - wg.Done() - }() - return wg -} - -func hashInKeys(deadlockKeyHash uint64, keys [][]byte) bool { - for _, key := range keys { - if farm.Fingerprint64(key) == deadlockKeyHash { - return true - } - } - return false -} - -// IsReadOnly checks if the transaction has only performed read operations. -func (txn *KVTxn) IsReadOnly() bool { - return !txn.us.GetMemBuffer().Dirty() -} - -// StartTS returns the transaction start timestamp. -func (txn *KVTxn) StartTS() uint64 { - return txn.startTS -} - -// Valid returns if the transaction is valid. -// A transaction become invalid after commit or rollback. -func (txn *KVTxn) Valid() bool { - return txn.valid -} - -// Len returns the number of entries in the DB. -func (txn *KVTxn) Len() int { - return txn.us.GetMemBuffer().Len() -} - -// Size returns sum of keys and values length. -func (txn *KVTxn) Size() int { - return txn.us.GetMemBuffer().Size() -} - -// Reset reset the Transaction to initial states. -func (txn *KVTxn) Reset() { - txn.us.GetMemBuffer().Reset() -} - -// GetUnionStore returns the UnionStore binding to this transaction. -func (txn *KVTxn) GetUnionStore() *unionstore.KVUnionStore { - return txn.us -} - -// GetMemBuffer return the MemBuffer binding to this transaction. -func (txn *KVTxn) GetMemBuffer() *unionstore.MemDB { - return txn.us.GetMemBuffer() -} - -// GetSnapshot returns the Snapshot binding to this transaction. -func (txn *KVTxn) GetSnapshot() *KVSnapshot { - return txn.snapshot -} - -// SetBinlogExecutor sets the method to perform binlong synchronization. -func (txn *KVTxn) SetBinlogExecutor(binlog BinlogExecutor) { - txn.binlog = binlog - if txn.committer != nil { - txn.committer.binlog = binlog - } -} - -// GetClusterID returns store's cluster id. -func (txn *KVTxn) GetClusterID() uint64 { - return txn.store.clusterID -} diff --git a/store/tikv/unionstore/memdb.go b/store/tikv/unionstore/memdb.go deleted file mode 100644 index 7bfc11971e4e2..0000000000000 --- a/store/tikv/unionstore/memdb.go +++ /dev/null @@ -1,814 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "bytes" - "math" - "reflect" - "sync" - "unsafe" - - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" -) - -var tombstone = []byte{} - -// IsTombstone returns whether the value is a tombstone. -func IsTombstone(val []byte) bool { return len(val) == 0 } - -// MemKeyHandle represents a pointer for key in MemBuffer. -type MemKeyHandle struct { - // Opaque user data - UserData uint16 - idx uint16 - off uint32 -} - -func (h MemKeyHandle) toAddr() memdbArenaAddr { - return memdbArenaAddr{idx: uint32(h.idx), off: h.off} -} - -// MemDB is rollbackable Red-Black Tree optimized for TiDB's transaction states buffer use scenario. -// You can think MemDB is a combination of two separate tree map, one for key => value and another for key => keyFlags. -// -// The value map is rollbackable, that means you can use the `Staging`, `Release` and `Cleanup` API to safely modify KVs. -// -// The flags map is not rollbackable. There are two types of flag, persistent and non-persistent. -// When discarding a newly added KV in `Cleanup`, the non-persistent flags will be cleared. -// If there are persistent flags associated with key, we will keep this key in node without value. -type MemDB struct { - // This RWMutex only used to ensure memdbSnapGetter.Get will not race with - // concurrent memdb.Set, memdb.SetWithFlags, memdb.Delete and memdb.UpdateFlags. - sync.RWMutex - root memdbArenaAddr - allocator nodeAllocator - vlog memdbVlog - - entrySizeLimit uint64 - bufferSizeLimit uint64 - count int - size int - - vlogInvalid bool - dirty bool - stages []memdbCheckpoint -} - -func newMemDB() *MemDB { - db := new(MemDB) - db.allocator.init() - db.root = nullAddr - db.stages = make([]memdbCheckpoint, 0, 2) - db.entrySizeLimit = math.MaxUint64 - db.bufferSizeLimit = math.MaxUint64 - return db -} - -// Staging create a new staging buffer inside the MemBuffer. -// Subsequent writes will be temporarily stored in this new staging buffer. -// When you think all modifications looks good, you can call `Release` to public all of them to the upper level buffer. -func (db *MemDB) Staging() int { - db.Lock() - defer db.Unlock() - - db.stages = append(db.stages, db.vlog.checkpoint()) - return len(db.stages) -} - -// Release publish all modifications in the latest staging buffer to upper level. -func (db *MemDB) Release(h int) { - if h != len(db.stages) { - // This should never happens in production environment. - // Use panic to make debug easier. - panic("cannot release staging buffer") - } - - db.Lock() - defer db.Unlock() - if h == 1 { - tail := db.vlog.checkpoint() - if !db.stages[0].isSamePosition(&tail) { - db.dirty = true - } - } - db.stages = db.stages[:h-1] -} - -// Cleanup cleanup the resources referenced by the StagingHandle. -// If the changes are not published by `Release`, they will be discarded. -func (db *MemDB) Cleanup(h int) { - if h > len(db.stages) { - return - } - if h < len(db.stages) { - // This should never happens in production environment. - // Use panic to make debug easier. - panic("cannot cleanup staging buffer") - } - - db.Lock() - defer db.Unlock() - cp := &db.stages[h-1] - if !db.vlogInvalid { - curr := db.vlog.checkpoint() - if !curr.isSamePosition(cp) { - db.vlog.revertToCheckpoint(db, cp) - db.vlog.truncate(cp) - } - } - db.stages = db.stages[:h-1] -} - -// Reset resets the MemBuffer to initial states. -func (db *MemDB) Reset() { - db.root = nullAddr - db.stages = db.stages[:0] - db.dirty = false - db.vlogInvalid = false - db.size = 0 - db.count = 0 - db.vlog.reset() - db.allocator.reset() -} - -// DiscardValues releases the memory used by all values. -// NOTE: any operation need value will panic after this function. -func (db *MemDB) DiscardValues() { - db.vlogInvalid = true - db.vlog.reset() -} - -// InspectStage used to inspect the value updates in the given stage. -func (db *MemDB) InspectStage(handle int, f func([]byte, kv.KeyFlags, []byte)) { - idx := handle - 1 - tail := db.vlog.checkpoint() - head := db.stages[idx] - db.vlog.inspectKVInLog(db, &head, &tail, f) -} - -// Get gets the value for key k from kv store. -// If corresponding kv pair does not exist, it returns nil and ErrNotExist. -func (db *MemDB) Get(key []byte) ([]byte, error) { - if db.vlogInvalid { - // panic for easier debugging. - panic("vlog is resetted") - } - - x := db.traverse(key, false) - if x.isNull() { - return nil, tikverr.ErrNotExist - } - if x.vptr.isNull() { - // A flag only key, act as value not exists - return nil, tikverr.ErrNotExist - } - return db.vlog.getValue(x.vptr), nil -} - -// SelectValueHistory select the latest value which makes `predicate` returns true from the modification history. -func (db *MemDB) SelectValueHistory(key []byte, predicate func(value []byte) bool) ([]byte, error) { - x := db.traverse(key, false) - if x.isNull() { - return nil, tikverr.ErrNotExist - } - if x.vptr.isNull() { - // A flag only key, act as value not exists - return nil, tikverr.ErrNotExist - } - result := db.vlog.selectValueHistory(x.vptr, func(addr memdbArenaAddr) bool { - return predicate(db.vlog.getValue(addr)) - }) - if result.isNull() { - return nil, nil - } - return db.vlog.getValue(result), nil -} - -// GetFlags returns the latest flags associated with key. -func (db *MemDB) GetFlags(key []byte) (kv.KeyFlags, error) { - x := db.traverse(key, false) - if x.isNull() { - return 0, tikverr.ErrNotExist - } - return x.getKeyFlags(), nil -} - -// UpdateFlags update the flags associated with key. -func (db *MemDB) UpdateFlags(key []byte, ops ...kv.FlagsOp) { - err := db.set(key, nil, ops...) - _ = err // set without value will never fail -} - -// Set sets the value for key k as v into kv store. -// v must NOT be nil or empty, otherwise it returns ErrCannotSetNilValue. -func (db *MemDB) Set(key []byte, value []byte) error { - if len(value) == 0 { - return tikverr.ErrCannotSetNilValue - } - return db.set(key, value) -} - -// SetWithFlags put key-value into the last active staging buffer with the given KeyFlags. -func (db *MemDB) SetWithFlags(key []byte, value []byte, ops ...kv.FlagsOp) error { - if len(value) == 0 { - return tikverr.ErrCannotSetNilValue - } - return db.set(key, value, ops...) -} - -// Delete removes the entry for key k from kv store. -func (db *MemDB) Delete(key []byte) error { - return db.set(key, tombstone) -} - -// DeleteWithFlags delete key with the given KeyFlags -func (db *MemDB) DeleteWithFlags(key []byte, ops ...kv.FlagsOp) error { - return db.set(key, tombstone, ops...) -} - -// GetKeyByHandle returns key by handle. -func (db *MemDB) GetKeyByHandle(handle MemKeyHandle) []byte { - x := db.getNode(handle.toAddr()) - return x.getKey() -} - -// GetValueByHandle returns value by handle. -func (db *MemDB) GetValueByHandle(handle MemKeyHandle) ([]byte, bool) { - if db.vlogInvalid { - return nil, false - } - x := db.getNode(handle.toAddr()) - if x.vptr.isNull() { - return nil, false - } - return db.vlog.getValue(x.vptr), true -} - -// Len returns the number of entries in the DB. -func (db *MemDB) Len() int { - return db.count -} - -// Size returns sum of keys and values length. -func (db *MemDB) Size() int { - return db.size -} - -// Dirty returns whether the root staging buffer is updated. -func (db *MemDB) Dirty() bool { - return db.dirty -} - -func (db *MemDB) set(key []byte, value []byte, ops ...kv.FlagsOp) error { - if db.vlogInvalid { - // panic for easier debugging. - panic("vlog is resetted") - } - - if value != nil { - if size := uint64(len(key) + len(value)); size > db.entrySizeLimit { - return &tikverr.ErrEntryTooLarge{ - Limit: db.entrySizeLimit, - Size: size, - } - } - } - - db.Lock() - defer db.Unlock() - - if len(db.stages) == 0 { - db.dirty = true - } - x := db.traverse(key, true) - - if len(ops) != 0 { - flags := kv.ApplyFlagsOps(x.getKeyFlags(), ops...) - if flags.AndPersistent() != 0 { - db.dirty = true - } - x.setKeyFlags(flags) - } - - if value == nil { - return nil - } - - db.setValue(x, value) - if uint64(db.Size()) > db.bufferSizeLimit { - return &tikverr.ErrTxnTooLarge{Size: db.Size()} - } - return nil -} - -func (db *MemDB) setValue(x memdbNodeAddr, value []byte) { - var activeCp *memdbCheckpoint - if len(db.stages) > 0 { - activeCp = &db.stages[len(db.stages)-1] - } - - var oldVal []byte - if !x.vptr.isNull() { - oldVal = db.vlog.getValue(x.vptr) - } - - if len(oldVal) > 0 && db.vlog.canModify(activeCp, x.vptr) { - // For easier to implement, we only consider this case. - // It is the most common usage in TiDB's transaction buffers. - if len(oldVal) == len(value) { - copy(oldVal, value) - return - } - } - x.vptr = db.vlog.appendValue(x.addr, x.vptr, value) - db.size = db.size - len(oldVal) + len(value) -} - -// traverse search for and if not found and insert is true, will add a new node in. -// Returns a pointer to the new node, or the node found. -func (db *MemDB) traverse(key []byte, insert bool) memdbNodeAddr { - x := db.getRoot() - y := memdbNodeAddr{nil, nullAddr} - found := false - - // walk x down the tree - for !x.isNull() && !found { - y = x - cmp := bytes.Compare(key, x.getKey()) - if cmp < 0 { - x = x.getLeft(db) - } else if cmp > 0 { - x = x.getRight(db) - } else { - found = true - } - } - - if found || !insert { - return x - } - - z := db.allocNode(key) - z.up = y.addr - - if y.isNull() { - db.root = z.addr - } else { - cmp := bytes.Compare(z.getKey(), y.getKey()) - if cmp < 0 { - y.left = z.addr - } else { - y.right = z.addr - } - } - - z.left = nullAddr - z.right = nullAddr - - // colour this new node red - z.setRed() - - // Having added a red node, we must now walk back up the tree balancing it, - // by a series of rotations and changing of colours - x = z - - // While we are not at the top and our parent node is red - // NOTE: Since the root node is guaranteed black, then we - // are also going to stop if we are the child of the root - - for x.addr != db.root { - xUp := x.getUp(db) - if xUp.isBlack() { - break - } - - xUpUp := xUp.getUp(db) - // if our parent is on the left side of our grandparent - if x.up == xUpUp.left { - // get the right side of our grandparent (uncle?) - y = xUpUp.getRight(db) - if y.isRed() { - // make our parent black - xUp.setBlack() - // make our uncle black - y.setBlack() - // make our grandparent red - xUpUp.setRed() - // now consider our grandparent - x = xUp.getUp(db) - } else { - // if we are on the right side of our parent - if x.addr == xUp.right { - // Move up to our parent - x = x.getUp(db) - db.leftRotate(x) - xUp = x.getUp(db) - xUpUp = xUp.getUp(db) - } - - xUp.setBlack() - xUpUp.setRed() - db.rightRotate(xUpUp) - } - } else { - // everything here is the same as above, but exchanging left for right - y = xUpUp.getLeft(db) - if y.isRed() { - xUp.setBlack() - y.setBlack() - xUpUp.setRed() - - x = xUp.getUp(db) - } else { - if x.addr == xUp.left { - x = x.getUp(db) - db.rightRotate(x) - xUp = x.getUp(db) - xUpUp = xUp.getUp(db) - } - - xUp.setBlack() - xUpUp.setRed() - db.leftRotate(xUpUp) - } - } - } - - // Set the root node black - db.getRoot().setBlack() - - return z -} - -// -// Rotate our tree thus:- -// -// X leftRotate(X)---> Y -// / \ / \ -// A Y <---rightRotate(Y) X C -// / \ / \ -// B C A B -// -// NOTE: This does not change the ordering. -// -// We assume that neither X nor Y is NULL -// - -func (db *MemDB) leftRotate(x memdbNodeAddr) { - y := x.getRight(db) - - // Turn Y's left subtree into X's right subtree (move B) - x.right = y.left - - // If B is not null, set it's parent to be X - if !y.left.isNull() { - left := y.getLeft(db) - left.up = x.addr - } - - // Set Y's parent to be what X's parent was - y.up = x.up - - // if X was the root - if x.up.isNull() { - db.root = y.addr - } else { - xUp := x.getUp(db) - // Set X's parent's left or right pointer to be Y - if x.addr == xUp.left { - xUp.left = y.addr - } else { - xUp.right = y.addr - } - } - - // Put X on Y's left - y.left = x.addr - // Set X's parent to be Y - x.up = y.addr -} - -func (db *MemDB) rightRotate(y memdbNodeAddr) { - x := y.getLeft(db) - - // Turn X's right subtree into Y's left subtree (move B) - y.left = x.right - - // If B is not null, set it's parent to be Y - if !x.right.isNull() { - right := x.getRight(db) - right.up = y.addr - } - - // Set X's parent to be what Y's parent was - x.up = y.up - - // if Y was the root - if y.up.isNull() { - db.root = x.addr - } else { - yUp := y.getUp(db) - // Set Y's parent's left or right pointer to be X - if y.addr == yUp.left { - yUp.left = x.addr - } else { - yUp.right = x.addr - } - } - - // Put Y on X's right - x.right = y.addr - // Set Y's parent to be X - y.up = x.addr -} - -func (db *MemDB) deleteNode(z memdbNodeAddr) { - var x, y memdbNodeAddr - - db.count-- - db.size -= int(z.klen) - - if z.left.isNull() || z.right.isNull() { - y = z - } else { - y = db.successor(z) - } - - if !y.left.isNull() { - x = y.getLeft(db) - } else { - x = y.getRight(db) - } - x.up = y.up - - if y.up.isNull() { - db.root = x.addr - } else { - yUp := y.getUp(db) - if y.addr == yUp.left { - yUp.left = x.addr - } else { - yUp.right = x.addr - } - } - - needFix := y.isBlack() - - // NOTE: traditional red-black tree will copy key from Y to Z and free Y. - // We cannot do the same thing here, due to Y's pointer is stored in vlog and the space in Z may not suitable for Y. - // So we need to copy states from Z to Y, and relink all nodes formerly connected to Z. - if y != z { - db.replaceNode(z, y) - } - - if needFix { - db.deleteNodeFix(x) - } - - db.allocator.freeNode(z.addr) -} - -func (db *MemDB) replaceNode(old memdbNodeAddr, new memdbNodeAddr) { - if !old.up.isNull() { - oldUp := old.getUp(db) - if old.addr == oldUp.left { - oldUp.left = new.addr - } else { - oldUp.right = new.addr - } - } else { - db.root = new.addr - } - new.up = old.up - - left := old.getLeft(db) - left.up = new.addr - new.left = old.left - - right := old.getRight(db) - right.up = new.addr - new.right = old.right - - if old.isBlack() { - new.setBlack() - } else { - new.setRed() - } -} - -func (db *MemDB) deleteNodeFix(x memdbNodeAddr) { - for x.addr != db.root && x.isBlack() { - xUp := x.getUp(db) - if x.addr == xUp.left { - w := xUp.getRight(db) - if w.isRed() { - w.setBlack() - xUp.setRed() - db.leftRotate(xUp) - w = x.getUp(db).getRight(db) - } - - if w.getLeft(db).isBlack() && w.getRight(db).isBlack() { - w.setRed() - x = x.getUp(db) - } else { - if w.getRight(db).isBlack() { - w.getLeft(db).setBlack() - w.setRed() - db.rightRotate(w) - w = x.getUp(db).getRight(db) - } - - xUp := x.getUp(db) - if xUp.isBlack() { - w.setBlack() - } else { - w.setRed() - } - xUp.setBlack() - w.getRight(db).setBlack() - db.leftRotate(xUp) - x = db.getRoot() - } - } else { - w := xUp.getLeft(db) - if w.isRed() { - w.setBlack() - xUp.setRed() - db.rightRotate(xUp) - w = x.getUp(db).getLeft(db) - } - - if w.getRight(db).isBlack() && w.getLeft(db).isBlack() { - w.setRed() - x = x.getUp(db) - } else { - if w.getLeft(db).isBlack() { - w.getRight(db).setBlack() - w.setRed() - db.leftRotate(w) - w = x.getUp(db).getLeft(db) - } - - xUp := x.getUp(db) - if xUp.isBlack() { - w.setBlack() - } else { - w.setRed() - } - xUp.setBlack() - w.getLeft(db).setBlack() - db.rightRotate(xUp) - x = db.getRoot() - } - } - } - x.setBlack() -} - -func (db *MemDB) successor(x memdbNodeAddr) (y memdbNodeAddr) { - if !x.right.isNull() { - // If right is not NULL then go right one and - // then keep going left until we find a node with - // no left pointer. - - y = x.getRight(db) - for !y.left.isNull() { - y = y.getLeft(db) - } - return - } - - // Go up the tree until we get to a node that is on the - // left of its parent (or the root) and then return the - // parent. - - y = x.getUp(db) - for !y.isNull() && x.addr == y.right { - x = y - y = y.getUp(db) - } - return y -} - -func (db *MemDB) predecessor(x memdbNodeAddr) (y memdbNodeAddr) { - if !x.left.isNull() { - // If left is not NULL then go left one and - // then keep going right until we find a node with - // no right pointer. - - y = x.getLeft(db) - for !y.right.isNull() { - y = y.getRight(db) - } - return - } - - // Go up the tree until we get to a node that is on the - // right of its parent (or the root) and then return the - // parent. - - y = x.getUp(db) - for !y.isNull() && x.addr == y.left { - x = y - y = y.getUp(db) - } - return y -} - -func (db *MemDB) getNode(x memdbArenaAddr) memdbNodeAddr { - return memdbNodeAddr{db.allocator.getNode(x), x} -} - -func (db *MemDB) getRoot() memdbNodeAddr { - return db.getNode(db.root) -} - -func (db *MemDB) allocNode(key []byte) memdbNodeAddr { - db.size += len(key) - db.count++ - x, xn := db.allocator.allocNode(key) - return memdbNodeAddr{xn, x} -} - -type memdbNodeAddr struct { - *memdbNode - addr memdbArenaAddr -} - -func (a *memdbNodeAddr) isNull() bool { - return a.addr.isNull() -} - -func (a memdbNodeAddr) getUp(db *MemDB) memdbNodeAddr { - return db.getNode(a.up) -} - -func (a memdbNodeAddr) getLeft(db *MemDB) memdbNodeAddr { - return db.getNode(a.left) -} - -func (a memdbNodeAddr) getRight(db *MemDB) memdbNodeAddr { - return db.getNode(a.right) -} - -type memdbNode struct { - up memdbArenaAddr - left memdbArenaAddr - right memdbArenaAddr - vptr memdbArenaAddr - klen uint16 - flags uint8 -} - -func (n *memdbNode) isRed() bool { - return n.flags&nodeColorBit != 0 -} - -func (n *memdbNode) isBlack() bool { - return !n.isRed() -} - -func (n *memdbNode) setRed() { - n.flags |= nodeColorBit -} - -func (n *memdbNode) setBlack() { - n.flags &= ^nodeColorBit -} - -func (n *memdbNode) getKey() []byte { - var ret []byte - hdr := (*reflect.SliceHeader)(unsafe.Pointer(&ret)) - hdr.Data = uintptr(unsafe.Pointer(&n.flags)) + 1 - hdr.Len = int(n.klen) - hdr.Cap = int(n.klen) - return ret -} - -const ( - // bit 1 => red, bit 0 => black - nodeColorBit uint8 = 0x80 - nodeFlagsMask = ^nodeColorBit -) - -func (n *memdbNode) getKeyFlags() kv.KeyFlags { - return kv.KeyFlags(n.flags & nodeFlagsMask) -} - -func (n *memdbNode) setKeyFlags(f kv.KeyFlags) { - n.flags = (^nodeFlagsMask & n.flags) | uint8(f) -} diff --git a/store/tikv/unionstore/memdb_arena.go b/store/tikv/unionstore/memdb_arena.go deleted file mode 100644 index 45f7f6279e2e5..0000000000000 --- a/store/tikv/unionstore/memdb_arena.go +++ /dev/null @@ -1,372 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "encoding/binary" - "math" - "unsafe" - - "github.com/pingcap/tidb/store/tikv/kv" -) - -const ( - alignMask = 1<<32 - 8 // 29 bit 1 and 3 bit 0. - - nullBlockOffset = math.MaxUint32 - maxBlockSize = 128 << 20 - initBlockSize = 4 * 1024 -) - -var ( - nullAddr = memdbArenaAddr{math.MaxUint32, math.MaxUint32} - endian = binary.LittleEndian -) - -type memdbArenaAddr struct { - idx uint32 - off uint32 -} - -func (addr memdbArenaAddr) isNull() bool { - return addr == nullAddr -} - -// store and load is used by vlog, due to pointer in vlog is not aligned. - -func (addr memdbArenaAddr) store(dst []byte) { - endian.PutUint32(dst, addr.idx) - endian.PutUint32(dst[4:], addr.off) -} - -func (addr *memdbArenaAddr) load(src []byte) { - addr.idx = endian.Uint32(src) - addr.off = endian.Uint32(src[4:]) -} - -type memdbArena struct { - blockSize int - blocks []memdbArenaBlock -} - -func (a *memdbArena) alloc(size int, align bool) (memdbArenaAddr, []byte) { - if size > maxBlockSize { - panic("alloc size is larger than max block size") - } - - if len(a.blocks) == 0 { - a.enlarge(size, initBlockSize) - } - - addr, data := a.allocInLastBlock(size, align) - if !addr.isNull() { - return addr, data - } - - a.enlarge(size, a.blockSize<<1) - return a.allocInLastBlock(size, align) -} - -func (a *memdbArena) enlarge(allocSize, blockSize int) { - a.blockSize = blockSize - for a.blockSize <= allocSize { - a.blockSize <<= 1 - } - // Size will never larger than maxBlockSize. - if a.blockSize > maxBlockSize { - a.blockSize = maxBlockSize - } - a.blocks = append(a.blocks, memdbArenaBlock{ - buf: make([]byte, a.blockSize), - }) -} - -func (a *memdbArena) allocInLastBlock(size int, align bool) (memdbArenaAddr, []byte) { - idx := len(a.blocks) - 1 - offset, data := a.blocks[idx].alloc(size, align) - if offset == nullBlockOffset { - return nullAddr, nil - } - return memdbArenaAddr{uint32(idx), offset}, data -} - -func (a *memdbArena) reset() { - for i := range a.blocks { - a.blocks[i].reset() - } - a.blocks = a.blocks[:0] - a.blockSize = 0 -} - -type memdbArenaBlock struct { - buf []byte - length int -} - -func (a *memdbArenaBlock) alloc(size int, align bool) (uint32, []byte) { - offset := a.length - if align { - // We must align the allocated address for node - // to make runtime.checkptrAlignment happy. - offset = (a.length + 7) & alignMask - } - newLen := offset + size - if newLen > len(a.buf) { - return nullBlockOffset, nil - } - a.length = newLen - return uint32(offset), a.buf[offset : offset+size] -} - -func (a *memdbArenaBlock) reset() { - a.buf = nil - a.length = 0 -} - -type memdbCheckpoint struct { - blockSize int - blocks int - offsetInBlock int -} - -func (cp *memdbCheckpoint) isSamePosition(other *memdbCheckpoint) bool { - return cp.blocks == other.blocks && cp.offsetInBlock == other.offsetInBlock -} - -func (a *memdbArena) checkpoint() memdbCheckpoint { - snap := memdbCheckpoint{ - blockSize: a.blockSize, - blocks: len(a.blocks), - } - if len(a.blocks) > 0 { - snap.offsetInBlock = a.blocks[len(a.blocks)-1].length - } - return snap -} - -func (a *memdbArena) truncate(snap *memdbCheckpoint) { - for i := snap.blocks; i < len(a.blocks); i++ { - a.blocks[i] = memdbArenaBlock{} - } - a.blocks = a.blocks[:snap.blocks] - if len(a.blocks) > 0 { - a.blocks[len(a.blocks)-1].length = snap.offsetInBlock - } - a.blockSize = snap.blockSize -} - -type nodeAllocator struct { - memdbArena - - // Dummy node, so that we can make X.left.up = X. - // We then use this instead of NULL to mean the top or bottom - // end of the rb tree. It is a black node. - nullNode memdbNode -} - -func (a *nodeAllocator) init() { - a.nullNode = memdbNode{ - up: nullAddr, - left: nullAddr, - right: nullAddr, - vptr: nullAddr, - } -} - -func (a *nodeAllocator) getNode(addr memdbArenaAddr) *memdbNode { - if addr.isNull() { - return &a.nullNode - } - - return (*memdbNode)(unsafe.Pointer(&a.blocks[addr.idx].buf[addr.off])) -} - -func (a *nodeAllocator) allocNode(key []byte) (memdbArenaAddr, *memdbNode) { - nodeSize := 8*4 + 2 + 1 + len(key) - addr, mem := a.alloc(nodeSize, true) - n := (*memdbNode)(unsafe.Pointer(&mem[0])) - n.vptr = nullAddr - n.klen = uint16(len(key)) - copy(n.getKey(), key) - return addr, n -} - -var testMode = false - -func (a *nodeAllocator) freeNode(addr memdbArenaAddr) { - if testMode { - // Make it easier for debug. - n := a.getNode(addr) - badAddr := nullAddr - badAddr.idx-- - n.left = badAddr - n.right = badAddr - n.up = badAddr - n.vptr = badAddr - return - } - // TODO: reuse freed nodes. -} - -func (a *nodeAllocator) reset() { - a.memdbArena.reset() - a.init() -} - -type memdbVlog struct { - memdbArena -} - -const memdbVlogHdrSize = 8 + 8 + 4 - -type memdbVlogHdr struct { - nodeAddr memdbArenaAddr - oldValue memdbArenaAddr - valueLen uint32 -} - -func (hdr *memdbVlogHdr) store(dst []byte) { - cursor := 0 - endian.PutUint32(dst[cursor:], hdr.valueLen) - cursor += 4 - hdr.oldValue.store(dst[cursor:]) - cursor += 8 - hdr.nodeAddr.store(dst[cursor:]) -} - -func (hdr *memdbVlogHdr) load(src []byte) { - cursor := 0 - hdr.valueLen = endian.Uint32(src[cursor:]) - cursor += 4 - hdr.oldValue.load(src[cursor:]) - cursor += 8 - hdr.nodeAddr.load(src[cursor:]) -} - -func (l *memdbVlog) appendValue(nodeAddr memdbArenaAddr, oldValue memdbArenaAddr, value []byte) memdbArenaAddr { - size := memdbVlogHdrSize + len(value) - addr, mem := l.alloc(size, false) - - copy(mem, value) - hdr := memdbVlogHdr{nodeAddr, oldValue, uint32(len(value))} - hdr.store(mem[len(value):]) - - addr.off += uint32(size) - return addr -} - -func (l *memdbVlog) getValue(addr memdbArenaAddr) []byte { - lenOff := addr.off - memdbVlogHdrSize - block := l.blocks[addr.idx].buf - valueLen := endian.Uint32(block[lenOff:]) - if valueLen == 0 { - return tombstone - } - valueOff := lenOff - valueLen - return block[valueOff:lenOff:lenOff] -} - -func (l *memdbVlog) getSnapshotValue(addr memdbArenaAddr, snap *memdbCheckpoint) ([]byte, bool) { - result := l.selectValueHistory(addr, func(addr memdbArenaAddr) bool { - return !l.canModify(snap, addr) - }) - if result.isNull() { - return nil, false - } - return l.getValue(addr), true -} - -func (l *memdbVlog) selectValueHistory(addr memdbArenaAddr, predicate func(memdbArenaAddr) bool) memdbArenaAddr { - for !addr.isNull() { - if predicate(addr) { - return addr - } - var hdr memdbVlogHdr - hdr.load(l.blocks[addr.idx].buf[addr.off-memdbVlogHdrSize:]) - addr = hdr.oldValue - } - return nullAddr -} - -func (l *memdbVlog) revertToCheckpoint(db *MemDB, cp *memdbCheckpoint) { - cursor := l.checkpoint() - for !cp.isSamePosition(&cursor) { - hdrOff := cursor.offsetInBlock - memdbVlogHdrSize - block := l.blocks[cursor.blocks-1].buf - var hdr memdbVlogHdr - hdr.load(block[hdrOff:]) - node := db.getNode(hdr.nodeAddr) - - node.vptr = hdr.oldValue - db.size -= int(hdr.valueLen) - // oldValue.isNull() == true means this is a newly added value. - if hdr.oldValue.isNull() { - // If there are no flags associated with this key, we need to delete this node. - keptFlags := node.getKeyFlags().AndPersistent() - if keptFlags == 0 { - db.deleteNode(node) - } else { - node.setKeyFlags(keptFlags) - db.dirty = true - } - } else { - db.size += len(l.getValue(hdr.oldValue)) - } - - l.moveBackCursor(&cursor, &hdr) - } -} - -func (l *memdbVlog) inspectKVInLog(db *MemDB, head, tail *memdbCheckpoint, f func([]byte, kv.KeyFlags, []byte)) { - cursor := *tail - for !head.isSamePosition(&cursor) { - cursorAddr := memdbArenaAddr{idx: uint32(cursor.blocks - 1), off: uint32(cursor.offsetInBlock)} - hdrOff := cursorAddr.off - memdbVlogHdrSize - block := l.blocks[cursorAddr.idx].buf - var hdr memdbVlogHdr - hdr.load(block[hdrOff:]) - node := db.allocator.getNode(hdr.nodeAddr) - - // Skip older versions. - if node.vptr == cursorAddr { - value := block[hdrOff-hdr.valueLen : hdrOff] - f(node.getKey(), node.getKeyFlags(), value) - } - - l.moveBackCursor(&cursor, &hdr) - } -} - -func (l *memdbVlog) moveBackCursor(cursor *memdbCheckpoint, hdr *memdbVlogHdr) { - cursor.offsetInBlock -= (memdbVlogHdrSize + int(hdr.valueLen)) - if cursor.offsetInBlock == 0 { - cursor.blocks-- - if cursor.blocks > 0 { - cursor.offsetInBlock = l.blocks[cursor.blocks-1].length - } - } -} - -func (l *memdbVlog) canModify(cp *memdbCheckpoint, addr memdbArenaAddr) bool { - if cp == nil { - return true - } - if int(addr.idx) > cp.blocks-1 { - return true - } - if int(addr.idx) == cp.blocks-1 && int(addr.off) > cp.offsetInBlock { - return true - } - return false -} diff --git a/store/tikv/unionstore/memdb_bench_test.go b/store/tikv/unionstore/memdb_bench_test.go deleted file mode 100644 index 064793fc0e205..0000000000000 --- a/store/tikv/unionstore/memdb_bench_test.go +++ /dev/null @@ -1,173 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "encoding/binary" - "math/rand" - "testing" -) - -const ( - keySize = 16 - valueSize = 128 -) - -func BenchmarkLargeIndex(b *testing.B) { - buf := make([][valueSize]byte, 10000000) - for i := range buf { - binary.LittleEndian.PutUint32(buf[i][:], uint32(i)) - } - db := newMemDB() - b.ResetTimer() - - for i := range buf { - db.Set(buf[i][:keySize], buf[i][:]) - } -} - -func BenchmarkPut(b *testing.B) { - buf := make([][valueSize]byte, b.N) - for i := range buf { - binary.BigEndian.PutUint32(buf[i][:], uint32(i)) - } - - p := newMemDB() - b.ResetTimer() - - for i := range buf { - p.Set(buf[i][:keySize], buf[i][:]) - } -} - -func BenchmarkPutRandom(b *testing.B) { - buf := make([][valueSize]byte, b.N) - for i := range buf { - binary.LittleEndian.PutUint32(buf[i][:], uint32(rand.Int())) - } - - p := newMemDB() - b.ResetTimer() - - for i := range buf { - p.Set(buf[i][:keySize], buf[i][:]) - } -} - -func BenchmarkGet(b *testing.B) { - buf := make([][valueSize]byte, b.N) - for i := range buf { - binary.BigEndian.PutUint32(buf[i][:], uint32(i)) - } - - p := newMemDB() - for i := range buf { - p.Set(buf[i][:keySize], buf[i][:]) - } - - b.ResetTimer() - for i := range buf { - p.Get(buf[i][:keySize]) - } -} - -func BenchmarkGetRandom(b *testing.B) { - buf := make([][valueSize]byte, b.N) - for i := range buf { - binary.LittleEndian.PutUint32(buf[i][:], uint32(rand.Int())) - } - - p := newMemDB() - for i := range buf { - p.Set(buf[i][:keySize], buf[i][:]) - } - - b.ResetTimer() - for i := 0; i < b.N; i++ { - p.Get(buf[i][:keySize]) - } -} - -var opCnt = 100000 - -func BenchmarkMemDbBufferSequential(b *testing.B) { - data := make([][]byte, opCnt) - for i := 0; i < opCnt; i++ { - data[i] = encodeInt(i) - } - buffer := newMemDB() - benchmarkSetGet(b, buffer, data) - b.ReportAllocs() -} - -func BenchmarkMemDbBufferRandom(b *testing.B) { - data := make([][]byte, opCnt) - for i := 0; i < opCnt; i++ { - data[i] = encodeInt(i) - } - shuffle(data) - buffer := newMemDB() - benchmarkSetGet(b, buffer, data) - b.ReportAllocs() -} - -func BenchmarkMemDbIter(b *testing.B) { - buffer := newMemDB() - benchIterator(b, buffer) - b.ReportAllocs() -} - -func BenchmarkMemDbCreation(b *testing.B) { - for i := 0; i < b.N; i++ { - newMemDB() - } - b.ReportAllocs() -} - -func shuffle(slc [][]byte) { - N := len(slc) - for i := 0; i < N; i++ { - // choose index uniformly in [i, N-1] - r := i + rand.Intn(N-i) - slc[r], slc[i] = slc[i], slc[r] - } -} -func benchmarkSetGet(b *testing.B, buffer *MemDB, data [][]byte) { - b.ResetTimer() - for i := 0; i < b.N; i++ { - for _, k := range data { - buffer.Set(k, k) - } - for _, k := range data { - buffer.Get(k) - } - } -} - -func benchIterator(b *testing.B, buffer *MemDB) { - for k := 0; k < opCnt; k++ { - buffer.Set(encodeInt(k), encodeInt(k)) - } - b.ResetTimer() - for i := 0; i < b.N; i++ { - iter, err := buffer.Iter(nil, nil) - if err != nil { - b.Error(err) - } - for iter.Valid() { - iter.Next() - } - iter.Close() - } -} diff --git a/store/tikv/unionstore/memdb_iterator.go b/store/tikv/unionstore/memdb_iterator.go deleted file mode 100644 index 5593ecd0e1f5d..0000000000000 --- a/store/tikv/unionstore/memdb_iterator.go +++ /dev/null @@ -1,229 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "bytes" - - "github.com/pingcap/tidb/store/tikv/kv" -) - -// MemdbIterator is an Iterator with KeyFlags related functions. -type MemdbIterator struct { - db *MemDB - curr memdbNodeAddr - start []byte - end []byte - reverse bool - includeFlags bool -} - -// Iter creates an Iterator positioned on the first entry that k <= entry's key. -// If such entry is not found, it returns an invalid Iterator with no error. -// It yields only keys that < upperBound. If upperBound is nil, it means the upperBound is unbounded. -// The Iterator must be Closed after use. -func (db *MemDB) Iter(k []byte, upperBound []byte) (Iterator, error) { - i := &MemdbIterator{ - db: db, - start: k, - end: upperBound, - } - i.init() - return i, nil -} - -// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. -// The returned iterator will iterate from greater key to smaller key. -// If k is nil, the returned iterator will be positioned at the last key. -// TODO: Add lower bound limit -func (db *MemDB) IterReverse(k []byte) (Iterator, error) { - i := &MemdbIterator{ - db: db, - end: k, - reverse: true, - } - i.init() - return i, nil -} - -// IterWithFlags returns a MemdbIterator. -func (db *MemDB) IterWithFlags(k []byte, upperBound []byte) *MemdbIterator { - i := &MemdbIterator{ - db: db, - start: k, - end: upperBound, - includeFlags: true, - } - i.init() - return i -} - -// IterReverseWithFlags returns a reversed MemdbIterator. -func (db *MemDB) IterReverseWithFlags(k []byte) *MemdbIterator { - i := &MemdbIterator{ - db: db, - end: k, - reverse: true, - includeFlags: true, - } - i.init() - return i -} - -func (i *MemdbIterator) init() { - if i.reverse { - if len(i.end) == 0 { - i.seekToLast() - } else { - i.seek(i.end) - } - } else { - if len(i.start) == 0 { - i.seekToFirst() - } else { - i.seek(i.start) - } - } - - if i.isFlagsOnly() && !i.includeFlags { - err := i.Next() - _ = err // memdbIterator will never fail - } -} - -// Valid returns true if the current iterator is valid. -func (i *MemdbIterator) Valid() bool { - if !i.reverse { - return !i.curr.isNull() && (i.end == nil || bytes.Compare(i.Key(), i.end) < 0) - } - return !i.curr.isNull() -} - -// Flags returns flags belong to current iterator. -func (i *MemdbIterator) Flags() kv.KeyFlags { - return i.curr.getKeyFlags() -} - -// UpdateFlags updates and apply with flagsOp. -func (i *MemdbIterator) UpdateFlags(ops ...kv.FlagsOp) { - origin := i.curr.getKeyFlags() - n := kv.ApplyFlagsOps(origin, ops...) - i.curr.setKeyFlags(n) -} - -// HasValue returns false if it is flags only. -func (i *MemdbIterator) HasValue() bool { - return !i.isFlagsOnly() -} - -// Key returns current key. -func (i *MemdbIterator) Key() []byte { - return i.curr.getKey() -} - -// Handle returns MemKeyHandle with the current position. -func (i *MemdbIterator) Handle() MemKeyHandle { - return MemKeyHandle{ - idx: uint16(i.curr.addr.idx), - off: i.curr.addr.off, - } -} - -// Value returns the value. -func (i *MemdbIterator) Value() []byte { - return i.db.vlog.getValue(i.curr.vptr) -} - -// Next goes the next position. -func (i *MemdbIterator) Next() error { - for { - if i.reverse { - i.curr = i.db.predecessor(i.curr) - } else { - i.curr = i.db.successor(i.curr) - } - - // We need to skip persistent flags only nodes. - if i.includeFlags || !i.isFlagsOnly() { - break - } - } - return nil -} - -// Close closes the current iterator. -func (i *MemdbIterator) Close() {} - -func (i *MemdbIterator) seekToFirst() { - y := memdbNodeAddr{nil, nullAddr} - x := i.db.getNode(i.db.root) - - for !x.isNull() { - y = x - x = y.getLeft(i.db) - } - - i.curr = y -} - -func (i *MemdbIterator) seekToLast() { - y := memdbNodeAddr{nil, nullAddr} - x := i.db.getNode(i.db.root) - - for !x.isNull() { - y = x - x = y.getRight(i.db) - } - - i.curr = y -} - -func (i *MemdbIterator) seek(key []byte) { - y := memdbNodeAddr{nil, nullAddr} - x := i.db.getNode(i.db.root) - - var cmp int - for !x.isNull() { - y = x - cmp = bytes.Compare(key, y.getKey()) - - if cmp < 0 { - x = y.getLeft(i.db) - } else if cmp > 0 { - x = y.getRight(i.db) - } else { - break - } - } - - if !i.reverse { - if cmp > 0 { - // Move to next - i.curr = i.db.successor(y) - return - } - i.curr = y - return - } - - if cmp <= 0 && !y.isNull() { - i.curr = i.db.predecessor(y) - return - } - i.curr = y -} - -func (i *MemdbIterator) isFlagsOnly() bool { - return !i.curr.isNull() && i.curr.vptr.isNull() -} diff --git a/store/tikv/unionstore/memdb_norace_test.go b/store/tikv/unionstore/memdb_norace_test.go deleted file mode 100644 index 865be38a15947..0000000000000 --- a/store/tikv/unionstore/memdb_norace_test.go +++ /dev/null @@ -1,139 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -// +build !race - -package unionstore - -import ( - "encoding/binary" - "math/rand" - - . "github.com/pingcap/check" - "github.com/pingcap/goleveldb/leveldb/comparer" - leveldb "github.com/pingcap/goleveldb/leveldb/memdb" -) - -// The test takes too long under the race detector. -func (s testMemDBSuite) TestRandom(c *C) { - c.Parallel() - const cnt = 500000 - keys := make([][]byte, cnt) - for i := range keys { - keys[i] = make([]byte, rand.Intn(19)+1) - rand.Read(keys[i]) - } - - p1 := newMemDB() - p2 := leveldb.New(comparer.DefaultComparer, 4*1024) - for _, k := range keys { - p1.Set(k, k) - _ = p2.Put(k, k) - } - - c.Check(p1.Len(), Equals, p2.Len()) - c.Check(p1.Size(), Equals, p2.Size()) - - rand.Shuffle(cnt, func(i, j int) { keys[i], keys[j] = keys[j], keys[i] }) - - for _, k := range keys { - op := rand.Float64() - if op < 0.35 { - p1.DeleteKey(k) - p2.Delete(k) - } else { - newValue := make([]byte, rand.Intn(19)+1) - rand.Read(newValue) - p1.Set(k, newValue) - _ = p2.Put(k, newValue) - } - } - s.checkConsist(c, p1, p2) -} - -// The test takes too long under the race detector. -func (s testMemDBSuite) TestRandomDerive(c *C) { - c.Parallel() - db := newMemDB() - golden := leveldb.New(comparer.DefaultComparer, 4*1024) - s.testRandomDeriveRecur(c, db, golden, 0) -} - -func (s testMemDBSuite) testRandomDeriveRecur(c *C, db *MemDB, golden *leveldb.DB, depth int) [][2][]byte { - var keys [][]byte - if op := rand.Float64(); op < 0.33 { - start, end := rand.Intn(512), rand.Intn(512)+512 - cnt := end - start - keys = make([][]byte, cnt) - for i := range keys { - keys[i] = make([]byte, 8) - binary.BigEndian.PutUint64(keys[i], uint64(start+i)) - } - } else if op < 0.66 { - keys = make([][]byte, rand.Intn(512)+512) - for i := range keys { - keys[i] = make([]byte, rand.Intn(19)+1) - rand.Read(keys[i]) - } - } else { - keys = make([][]byte, 512) - for i := range keys { - keys[i] = make([]byte, 8) - binary.BigEndian.PutUint64(keys[i], uint64(i)) - } - } - - vals := make([][]byte, len(keys)) - for i := range vals { - vals[i] = make([]byte, rand.Intn(255)+1) - rand.Read(vals[i]) - } - - h := db.Staging() - opLog := make([][2][]byte, 0, len(keys)) - for i := range keys { - db.Set(keys[i], vals[i]) - old, err := golden.Get(keys[i]) - if err != nil { - opLog = append(opLog, [2][]byte{keys[i], nil}) - } else { - opLog = append(opLog, [2][]byte{keys[i], old}) - } - golden.Put(keys[i], vals[i]) - } - - if depth < 2000 { - childOps := s.testRandomDeriveRecur(c, db, golden, depth+1) - opLog = append(opLog, childOps...) - } - - if rand.Float64() < 0.3 && depth > 0 { - db.Cleanup(h) - for i := len(opLog) - 1; i >= 0; i-- { - if opLog[i][1] == nil { - golden.Delete(opLog[i][0]) - } else { - golden.Put(opLog[i][0], opLog[i][1]) - } - } - opLog = nil - } else { - db.Release(h) - } - - if depth%200 == 0 { - s.checkConsist(c, db, golden) - } - - return opLog -} diff --git a/store/tikv/unionstore/memdb_snapshot.go b/store/tikv/unionstore/memdb_snapshot.go deleted file mode 100644 index ad23a1efb0b51..0000000000000 --- a/store/tikv/unionstore/memdb_snapshot.go +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - tikverr "github.com/pingcap/tidb/store/tikv/error" -) - -// SnapshotGetter returns a Getter for a snapshot of MemBuffer. -func (db *MemDB) SnapshotGetter() Getter { - return &memdbSnapGetter{ - db: db, - cp: db.getSnapshot(), - } -} - -// SnapshotIter returns a Iterator for a snapshot of MemBuffer. -func (db *MemDB) SnapshotIter(start, end []byte) Iterator { - it := &memdbSnapIter{ - MemdbIterator: &MemdbIterator{ - db: db, - start: start, - end: end, - }, - cp: db.getSnapshot(), - } - it.init() - return it -} - -func (db *MemDB) getSnapshot() memdbCheckpoint { - if len(db.stages) > 0 { - return db.stages[0] - } - return db.vlog.checkpoint() -} - -type memdbSnapGetter struct { - db *MemDB - cp memdbCheckpoint -} - -func (snap *memdbSnapGetter) Get(key []byte) ([]byte, error) { - x := snap.db.traverse(key, false) - if x.isNull() { - return nil, tikverr.ErrNotExist - } - if x.vptr.isNull() { - // A flag only key, act as value not exists - return nil, tikverr.ErrNotExist - } - v, ok := snap.db.vlog.getSnapshotValue(x.vptr, &snap.cp) - if !ok { - return nil, tikverr.ErrNotExist - } - return v, nil -} - -type memdbSnapIter struct { - *MemdbIterator - value []byte - cp memdbCheckpoint -} - -func (i *memdbSnapIter) Value() []byte { - return i.value -} - -func (i *memdbSnapIter) Next() error { - i.value = nil - for i.Valid() { - if err := i.MemdbIterator.Next(); err != nil { - return err - } - if i.setValue() { - return nil - } - } - return nil -} - -func (i *memdbSnapIter) setValue() bool { - if !i.Valid() { - return false - } - if v, ok := i.db.vlog.getSnapshotValue(i.curr.vptr, &i.cp); ok { - i.value = v - return true - } - return false -} - -func (i *memdbSnapIter) init() { - if len(i.start) == 0 { - i.seekToFirst() - } else { - i.seek(i.start) - } - - if !i.setValue() { - err := i.Next() - _ = err // memdbIterator will never fail - } -} diff --git a/store/tikv/unionstore/memdb_test.go b/store/tikv/unionstore/memdb_test.go deleted file mode 100644 index 31097e522fffc..0000000000000 --- a/store/tikv/unionstore/memdb_test.go +++ /dev/null @@ -1,826 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Copyright 2015 Wenbin Xiao -// -// 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "encoding/binary" - "fmt" - "testing" - - . "github.com/pingcap/check" - leveldb "github.com/pingcap/goleveldb/leveldb/memdb" - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/util/testleak" -) - -type KeyFlags = kv.KeyFlags - -func init() { - testMode = true -} - -func TestT(t *testing.T) { - CustomVerboseFlag = true - TestingT(t) -} - -var ( - _ = Suite(&testKVSuite{}) - _ = Suite(&testMemDBSuite{}) -) - -type testMemDBSuite struct{} - -// DeleteKey is used in test to verify the `deleteNode` used in `vlog.revertToCheckpoint`. -func (db *MemDB) DeleteKey(key []byte) { - x := db.traverse(key, false) - if x.isNull() { - return - } - db.size -= len(db.vlog.getValue(x.vptr)) - db.deleteNode(x) -} - -func (s *testMemDBSuite) TestGetSet(c *C) { - const cnt = 10000 - p := s.fillDB(cnt) - - var buf [4]byte - for i := 0; i < cnt; i++ { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - v, err := p.Get(buf[:]) - c.Assert(err, IsNil) - c.Assert(v, BytesEquals, buf[:]) - } -} - -func (s *testMemDBSuite) TestBigKV(c *C) { - db := newMemDB() - db.Set([]byte{1}, make([]byte, 80<<20)) - c.Assert(db.vlog.blockSize, Equals, maxBlockSize) - c.Assert(len(db.vlog.blocks), Equals, 1) - h := db.Staging() - db.Set([]byte{2}, make([]byte, 127<<20)) - db.Release(h) - c.Assert(db.vlog.blockSize, Equals, maxBlockSize) - c.Assert(len(db.vlog.blocks), Equals, 2) - c.Assert(func() { db.Set([]byte{3}, make([]byte, maxBlockSize+1)) }, Panics, "alloc size is larger than max block size") -} - -func (s *testMemDBSuite) TestIterator(c *C) { - const cnt = 10000 - db := s.fillDB(cnt) - - var buf [4]byte - var i int - - for it, _ := db.Iter(nil, nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - c.Assert(it.Key(), BytesEquals, buf[:]) - c.Assert(it.Value(), BytesEquals, buf[:]) - i++ - } - c.Assert(i, Equals, cnt) - - i-- - for it, _ := db.IterReverse(nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - c.Assert(it.Key(), BytesEquals, buf[:]) - c.Assert(it.Value(), BytesEquals, buf[:]) - i-- - } - c.Assert(i, Equals, -1) -} - -func (s *testMemDBSuite) TestDiscard(c *C) { - const cnt = 10000 - db := newMemDB() - base := s.deriveAndFill(0, cnt, 0, db) - sz := db.Size() - - db.Cleanup(s.deriveAndFill(0, cnt, 1, db)) - c.Assert(db.Len(), Equals, cnt) - c.Assert(db.Size(), Equals, sz) - - var buf [4]byte - - for i := 0; i < cnt; i++ { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - v, err := db.Get(buf[:]) - c.Assert(err, IsNil) - c.Assert(v, BytesEquals, buf[:]) - } - - var i int - for it, _ := db.Iter(nil, nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - c.Assert(it.Key(), BytesEquals, buf[:]) - c.Assert(it.Value(), BytesEquals, buf[:]) - i++ - } - c.Assert(i, Equals, cnt) - - i-- - for it, _ := db.IterReverse(nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - c.Assert(it.Key(), BytesEquals, buf[:]) - c.Assert(it.Value(), BytesEquals, buf[:]) - i-- - } - c.Assert(i, Equals, -1) - - db.Cleanup(base) - for i := 0; i < cnt; i++ { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - _, err := db.Get(buf[:]) - c.Assert(err, NotNil) - } - it1, _ := db.Iter(nil, nil) - it := it1.(*MemdbIterator) - it.seekToFirst() - c.Assert(it.Valid(), IsFalse) - it.seekToLast() - c.Assert(it.Valid(), IsFalse) - it.seek([]byte{0xff}) - c.Assert(it.Valid(), IsFalse) -} - -func (s *testMemDBSuite) TestFlushOverwrite(c *C) { - const cnt = 10000 - db := newMemDB() - db.Release(s.deriveAndFill(0, cnt, 0, db)) - sz := db.Size() - - db.Release(s.deriveAndFill(0, cnt, 1, db)) - - c.Assert(db.Len(), Equals, cnt) - c.Assert(db.Size(), Equals, sz) - - var kbuf, vbuf [4]byte - - for i := 0; i < cnt; i++ { - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i+1)) - v, err := db.Get(kbuf[:]) - c.Assert(err, IsNil) - c.Assert(v, DeepEquals, vbuf[:]) - } - - var i int - for it, _ := db.Iter(nil, nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i+1)) - c.Assert(it.Key(), BytesEquals, kbuf[:]) - c.Assert(it.Value(), BytesEquals, vbuf[:]) - i++ - } - c.Assert(i, Equals, cnt) - - i-- - for it, _ := db.IterReverse(nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i+1)) - c.Assert(it.Key(), BytesEquals, kbuf[:]) - c.Assert(it.Value(), BytesEquals, vbuf[:]) - i-- - } - c.Assert(i, Equals, -1) -} - -func (s *testMemDBSuite) TestComplexUpdate(c *C) { - const ( - keep = 3000 - overwrite = 6000 - insert = 9000 - ) - - db := newMemDB() - db.Release(s.deriveAndFill(0, overwrite, 0, db)) - c.Assert(db.Len(), Equals, overwrite) - db.Release(s.deriveAndFill(keep, insert, 1, db)) - c.Assert(db.Len(), Equals, insert) - - var kbuf, vbuf [4]byte - - for i := 0; i < insert; i++ { - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i)) - if i >= keep { - binary.BigEndian.PutUint32(vbuf[:], uint32(i+1)) - } - v, err := db.Get(kbuf[:]) - c.Assert(err, IsNil) - c.Assert(v, BytesEquals, vbuf[:]) - } -} - -func (s *testMemDBSuite) TestNestedSandbox(c *C) { - db := newMemDB() - h0 := s.deriveAndFill(0, 200, 0, db) - h1 := s.deriveAndFill(0, 100, 1, db) - h2 := s.deriveAndFill(50, 150, 2, db) - h3 := s.deriveAndFill(100, 120, 3, db) - h4 := s.deriveAndFill(0, 150, 4, db) - db.Cleanup(h4) // Discard (0..150 -> 4) - db.Release(h3) // Flush (100..120 -> 3) - db.Cleanup(h2) // Discard (100..120 -> 3) & (50..150 -> 2) - db.Release(h1) // Flush (0..100 -> 1) - db.Release(h0) // Flush (0..100 -> 1) & (0..200 -> 0) - // The final result should be (0..100 -> 1) & (101..200 -> 0) - - var kbuf, vbuf [4]byte - - for i := 0; i < 200; i++ { - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i)) - if i < 100 { - binary.BigEndian.PutUint32(vbuf[:], uint32(i+1)) - } - v, err := db.Get(kbuf[:]) - c.Assert(err, IsNil) - c.Assert(v, BytesEquals, vbuf[:]) - } - - var i int - - for it, _ := db.Iter(nil, nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i)) - if i < 100 { - binary.BigEndian.PutUint32(vbuf[:], uint32(i+1)) - } - c.Assert(it.Key(), BytesEquals, kbuf[:]) - c.Assert(it.Value(), BytesEquals, vbuf[:]) - i++ - } - c.Assert(i, Equals, 200) - - i-- - for it, _ := db.IterReverse(nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i)) - if i < 100 { - binary.BigEndian.PutUint32(vbuf[:], uint32(i+1)) - } - c.Assert(it.Key(), BytesEquals, kbuf[:]) - c.Assert(it.Value(), BytesEquals, vbuf[:]) - i-- - } - c.Assert(i, Equals, -1) -} - -func (s *testMemDBSuite) TestOverwrite(c *C) { - const cnt = 10000 - db := s.fillDB(cnt) - var buf [4]byte - - sz := db.Size() - for i := 0; i < cnt; i += 3 { - var newBuf [4]byte - binary.BigEndian.PutUint32(buf[:], uint32(i)) - binary.BigEndian.PutUint32(newBuf[:], uint32(i*10)) - db.Set(buf[:], newBuf[:]) - } - c.Assert(db.Len(), Equals, cnt) - c.Assert(db.Size(), Equals, sz) - - for i := 0; i < cnt; i++ { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - val, _ := db.Get(buf[:]) - v := binary.BigEndian.Uint32(val) - if i%3 == 0 { - c.Assert(v, Equals, uint32(i*10)) - } else { - c.Assert(v, Equals, uint32(i)) - } - } - - var i int - - for it, _ := db.Iter(nil, nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - c.Assert(it.Key(), BytesEquals, buf[:]) - v := binary.BigEndian.Uint32(it.Value()) - if i%3 == 0 { - c.Assert(v, Equals, uint32(i*10)) - } else { - c.Assert(v, Equals, uint32(i)) - } - i++ - } - c.Assert(i, Equals, cnt) - - i-- - for it, _ := db.IterReverse(nil); it.Valid(); it.Next() { - binary.BigEndian.PutUint32(buf[:], uint32(i)) - c.Assert(it.Key(), BytesEquals, buf[:]) - v := binary.BigEndian.Uint32(it.Value()) - if i%3 == 0 { - c.Assert(v, Equals, uint32(i*10)) - } else { - c.Assert(v, Equals, uint32(i)) - } - i-- - } - c.Assert(i, Equals, -1) -} - -func (s *testMemDBSuite) TestKVLargeThanBlock(c *C) { - db := newMemDB() - db.Set([]byte{1}, make([]byte, 1)) - db.Set([]byte{2}, make([]byte, 4096)) - c.Assert(len(db.vlog.blocks), Equals, 2) - db.Set([]byte{3}, make([]byte, 3000)) - c.Assert(len(db.vlog.blocks), Equals, 2) - val, err := db.Get([]byte{3}) - c.Assert(err, IsNil) - c.Assert(len(val), Equals, 3000) -} - -func (s *testMemDBSuite) TestEmptyDB(c *C) { - db := newMemDB() - _, err := db.Get([]byte{0}) - c.Assert(err, NotNil) - it1, _ := db.Iter(nil, nil) - it := it1.(*MemdbIterator) - it.seekToFirst() - c.Assert(it.Valid(), IsFalse) - it.seekToLast() - c.Assert(it.Valid(), IsFalse) - it.seek([]byte{0xff}) - c.Assert(it.Valid(), IsFalse) -} - -func (s *testMemDBSuite) TestReset(c *C) { - db := s.fillDB(1000) - db.Reset() - _, err := db.Get([]byte{0, 0, 0, 0}) - c.Assert(err, NotNil) - it1, _ := db.Iter(nil, nil) - it := it1.(*MemdbIterator) - it.seekToFirst() - c.Assert(it.Valid(), IsFalse) - it.seekToLast() - c.Assert(it.Valid(), IsFalse) - it.seek([]byte{0xff}) - c.Assert(it.Valid(), IsFalse) -} - -func (s *testMemDBSuite) TestInspectStage(c *C) { - db := newMemDB() - h1 := s.deriveAndFill(0, 1000, 0, db) - h2 := s.deriveAndFill(500, 1000, 1, db) - for i := 500; i < 1500; i++ { - var kbuf [4]byte - // don't update in place - var vbuf [5]byte - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i+2)) - db.Set(kbuf[:], vbuf[:]) - } - h3 := s.deriveAndFill(1000, 2000, 3, db) - - db.InspectStage(h3, func(key []byte, _ KeyFlags, val []byte) { - k := int(binary.BigEndian.Uint32(key)) - v := int(binary.BigEndian.Uint32(val)) - - c.Assert(k >= 1000 && k < 2000, IsTrue) - c.Assert(v-k, DeepEquals, 3) - }) - - db.InspectStage(h2, func(key []byte, _ KeyFlags, val []byte) { - k := int(binary.BigEndian.Uint32(key)) - v := int(binary.BigEndian.Uint32(val)) - - c.Assert(k >= 500 && k < 2000, IsTrue) - if k < 1000 { - c.Assert(v-k, Equals, 2) - } else { - c.Assert(v-k, Equals, 3) - } - }) - - db.Cleanup(h3) - db.Release(h2) - - db.InspectStage(h1, func(key []byte, _ KeyFlags, val []byte) { - k := int(binary.BigEndian.Uint32(key)) - v := int(binary.BigEndian.Uint32(val)) - - c.Assert(k >= 0 && k < 1500, IsTrue) - if k < 500 { - c.Assert(v-k, Equals, 0) - } else { - c.Assert(v-k, Equals, 2) - } - }) - - db.Release(h1) -} - -func (s *testMemDBSuite) TestDirty(c *C) { - db := newMemDB() - db.Set([]byte{1}, []byte{1}) - c.Assert(db.Dirty(), IsTrue) - - db = newMemDB() - h := db.Staging() - db.Set([]byte{1}, []byte{1}) - db.Cleanup(h) - c.Assert(db.Dirty(), IsFalse) - - h = db.Staging() - db.Set([]byte{1}, []byte{1}) - db.Release(h) - c.Assert(db.Dirty(), IsTrue) - - // persistent flags will make memdb dirty. - db = newMemDB() - h = db.Staging() - db.SetWithFlags([]byte{1}, []byte{1}, kv.SetKeyLocked) - db.Cleanup(h) - c.Assert(db.Dirty(), IsTrue) - - // non-persistent flags will not make memdb dirty. - db = newMemDB() - h = db.Staging() - db.SetWithFlags([]byte{1}, []byte{1}, kv.SetPresumeKeyNotExists) - db.Cleanup(h) - c.Assert(db.Dirty(), IsFalse) -} - -func (s *testMemDBSuite) TestFlags(c *C) { - const cnt = 10000 - db := newMemDB() - h := db.Staging() - for i := uint32(0); i < cnt; i++ { - var buf [4]byte - binary.BigEndian.PutUint32(buf[:], i) - if i%2 == 0 { - db.SetWithFlags(buf[:], buf[:], kv.SetPresumeKeyNotExists, kv.SetKeyLocked) - } else { - db.SetWithFlags(buf[:], buf[:], kv.SetPresumeKeyNotExists) - } - } - db.Cleanup(h) - - for i := uint32(0); i < cnt; i++ { - var buf [4]byte - binary.BigEndian.PutUint32(buf[:], i) - _, err := db.Get(buf[:]) - c.Assert(err, NotNil) - flags, err := db.GetFlags(buf[:]) - if i%2 == 0 { - c.Assert(err, IsNil) - c.Assert(flags.HasLocked(), IsTrue) - c.Assert(flags.HasPresumeKeyNotExists(), IsFalse) - } else { - c.Assert(err, NotNil) - } - } - - c.Assert(db.Len(), Equals, 5000) - c.Assert(db.Size(), Equals, 20000) - - it1, _ := db.Iter(nil, nil) - it := it1.(*MemdbIterator) - c.Assert(it.Valid(), IsFalse) - - it.includeFlags = true - it.init() - - for ; it.Valid(); it.Next() { - k := binary.BigEndian.Uint32(it.Key()) - c.Assert(k%2 == 0, IsTrue) - } - - for i := uint32(0); i < cnt; i++ { - var buf [4]byte - binary.BigEndian.PutUint32(buf[:], i) - db.UpdateFlags(buf[:], kv.DelKeyLocked) - } - for i := uint32(0); i < cnt; i++ { - var buf [4]byte - binary.BigEndian.PutUint32(buf[:], i) - _, err := db.Get(buf[:]) - c.Assert(err, NotNil) - - // UpdateFlags will create missing node. - flags, err := db.GetFlags(buf[:]) - c.Assert(err, IsNil) - c.Assert(flags.HasLocked(), IsFalse) - } -} - -func (s *testMemDBSuite) checkConsist(c *C, p1 *MemDB, p2 *leveldb.DB) { - c.Assert(p1.Len(), Equals, p2.Len()) - c.Assert(p1.Size(), Equals, p2.Size()) - - it1, _ := p1.Iter(nil, nil) - it2 := p2.NewIterator(nil) - - var prevKey, prevVal []byte - for it2.First(); it2.Valid(); it2.Next() { - v, err := p1.Get(it2.Key()) - c.Assert(err, IsNil) - c.Assert(v, BytesEquals, it2.Value()) - - c.Assert(it1.Key(), BytesEquals, it2.Key()) - c.Assert(it1.Value(), BytesEquals, it2.Value()) - - it, _ := p1.Iter(it2.Key(), nil) - c.Assert(it.Key(), BytesEquals, it2.Key()) - c.Assert(it.Value(), BytesEquals, it2.Value()) - - if prevKey != nil { - it, _ = p1.IterReverse(it2.Key()) - c.Assert(it.Key(), BytesEquals, prevKey) - c.Assert(it.Value(), BytesEquals, prevVal) - } - - it1.Next() - prevKey = it2.Key() - prevVal = it2.Value() - } - - it1, _ = p1.IterReverse(nil) - for it2.Last(); it2.Valid(); it2.Prev() { - c.Assert(it1.Key(), BytesEquals, it2.Key()) - c.Assert(it1.Value(), BytesEquals, it2.Value()) - it1.Next() - } -} - -func (s *testMemDBSuite) fillDB(cnt int) *MemDB { - db := newMemDB() - h := s.deriveAndFill(0, cnt, 0, db) - db.Release(h) - return db -} - -func (s *testMemDBSuite) deriveAndFill(start, end, valueBase int, db *MemDB) int { - h := db.Staging() - var kbuf, vbuf [4]byte - for i := start; i < end; i++ { - binary.BigEndian.PutUint32(kbuf[:], uint32(i)) - binary.BigEndian.PutUint32(vbuf[:], uint32(i+valueBase)) - db.Set(kbuf[:], vbuf[:]) - } - return h -} - -const ( - startIndex = 0 - testCount = 2 - indexStep = 2 -) - -type testKVSuite struct { - bs []*MemDB -} - -func (s *testKVSuite) SetUpSuite(c *C) { - s.bs = make([]*MemDB, 1) - s.bs[0] = newMemDB() -} - -func (s *testKVSuite) ResetMembuffers() { - s.bs[0] = newMemDB() -} - -func insertData(c *C, buffer *MemDB) { - for i := startIndex; i < testCount; i++ { - val := encodeInt(i * indexStep) - err := buffer.Set(val, val) - c.Assert(err, IsNil) - } -} - -func encodeInt(n int) []byte { - return []byte(fmt.Sprintf("%010d", n)) -} - -func decodeInt(s []byte) int { - var n int - fmt.Sscanf(string(s), "%010d", &n) - return n -} - -func valToStr(c *C, iter Iterator) string { - val := iter.Value() - return string(val) -} - -func checkNewIterator(c *C, buffer *MemDB) { - for i := startIndex; i < testCount; i++ { - val := encodeInt(i * indexStep) - iter, err := buffer.Iter(val, nil) - c.Assert(err, IsNil) - c.Assert(iter.Key(), BytesEquals, val) - c.Assert(decodeInt([]byte(valToStr(c, iter))), Equals, i*indexStep) - iter.Close() - } - - // Test iterator Next() - for i := startIndex; i < testCount-1; i++ { - val := encodeInt(i * indexStep) - iter, err := buffer.Iter(val, nil) - c.Assert(err, IsNil) - c.Assert(iter.Key(), BytesEquals, val) - c.Assert(valToStr(c, iter), Equals, string(val)) - - err = iter.Next() - c.Assert(err, IsNil) - c.Assert(iter.Valid(), IsTrue) - - val = encodeInt((i + 1) * indexStep) - c.Assert(iter.Key(), BytesEquals, val) - c.Assert(valToStr(c, iter), Equals, string(val)) - iter.Close() - } - - // Non exist and beyond maximum seek test - iter, err := buffer.Iter(encodeInt(testCount*indexStep), nil) - c.Assert(err, IsNil) - c.Assert(iter.Valid(), IsFalse) - - // Non exist but between existing keys seek test, - // it returns the smallest key that larger than the one we are seeking - inBetween := encodeInt((testCount-1)*indexStep - 1) - last := encodeInt((testCount - 1) * indexStep) - iter, err = buffer.Iter(inBetween, nil) - c.Assert(err, IsNil) - c.Assert(iter.Valid(), IsTrue) - c.Assert(iter.Key(), Not(BytesEquals), inBetween) - c.Assert(iter.Key(), BytesEquals, last) - iter.Close() -} - -func mustGet(c *C, buffer *MemDB) { - for i := startIndex; i < testCount; i++ { - s := encodeInt(i * indexStep) - val, err := buffer.Get(s) - c.Assert(err, IsNil) - c.Assert(string(val), Equals, string(s)) - } -} - -func (s *testKVSuite) TestGetSet(c *C) { - defer testleak.AfterTest(c)() - for _, buffer := range s.bs { - insertData(c, buffer) - mustGet(c, buffer) - } - s.ResetMembuffers() -} - -func (s *testKVSuite) TestNewIterator(c *C) { - defer testleak.AfterTest(c)() - for _, buffer := range s.bs { - // should be invalid - iter, err := buffer.Iter(nil, nil) - c.Assert(err, IsNil) - c.Assert(iter.Valid(), IsFalse) - - insertData(c, buffer) - checkNewIterator(c, buffer) - } - s.ResetMembuffers() -} - -// FnKeyCmp is the function for iterator the keys -type FnKeyCmp func(key []byte) bool - -// TODO: remove it since it is duplicated with kv.NextUtil -// NextUntil applies FnKeyCmp to each entry of the iterator until meets some condition. -// It will stop when fn returns true, or iterator is invalid or an error occurs. -func NextUntil(it Iterator, fn FnKeyCmp) error { - var err error - for it.Valid() && !fn(it.Key()) { - err = it.Next() - if err != nil { - return err - } - } - return nil -} - -func (s *testKVSuite) TestIterNextUntil(c *C) { - defer testleak.AfterTest(c)() - buffer := newMemDB() - insertData(c, buffer) - - iter, err := buffer.Iter(nil, nil) - c.Assert(err, IsNil) - - err = NextUntil(iter, func(k []byte) bool { - return false - }) - c.Assert(err, IsNil) - c.Assert(iter.Valid(), IsFalse) -} - -func (s *testKVSuite) TestBasicNewIterator(c *C) { - defer testleak.AfterTest(c)() - for _, buffer := range s.bs { - it, err := buffer.Iter([]byte("2"), nil) - c.Assert(err, IsNil) - c.Assert(it.Valid(), IsFalse) - } -} - -func (s *testKVSuite) TestNewIteratorMin(c *C) { - defer testleak.AfterTest(c)() - kvs := []struct { - key string - value string - }{ - {"DATA_test_main_db_tbl_tbl_test_record__00000000000000000001", "lock-version"}, - {"DATA_test_main_db_tbl_tbl_test_record__00000000000000000001_0002", "1"}, - {"DATA_test_main_db_tbl_tbl_test_record__00000000000000000001_0003", "hello"}, - {"DATA_test_main_db_tbl_tbl_test_record__00000000000000000002", "lock-version"}, - {"DATA_test_main_db_tbl_tbl_test_record__00000000000000000002_0002", "2"}, - {"DATA_test_main_db_tbl_tbl_test_record__00000000000000000002_0003", "hello"}, - } - for _, buffer := range s.bs { - for _, kv := range kvs { - err := buffer.Set([]byte(kv.key), []byte(kv.value)) - c.Assert(err, IsNil) - } - - cnt := 0 - it, err := buffer.Iter(nil, nil) - c.Assert(err, IsNil) - for it.Valid() { - cnt++ - err := it.Next() - c.Assert(err, IsNil) - } - c.Assert(cnt, Equals, 6) - - it, err = buffer.Iter([]byte("DATA_test_main_db_tbl_tbl_test_record__00000000000000000000"), nil) - c.Assert(err, IsNil) - c.Assert(string(it.Key()), Equals, "DATA_test_main_db_tbl_tbl_test_record__00000000000000000001") - } - s.ResetMembuffers() -} - -func (s *testKVSuite) TestMemDBStaging(c *C) { - buffer := newMemDB() - err := buffer.Set([]byte("x"), make([]byte, 2)) - c.Assert(err, IsNil) - - h1 := buffer.Staging() - err = buffer.Set([]byte("x"), make([]byte, 3)) - c.Assert(err, IsNil) - - h2 := buffer.Staging() - err = buffer.Set([]byte("yz"), make([]byte, 1)) - c.Assert(err, IsNil) - - v, _ := buffer.Get([]byte("x")) - c.Assert(len(v), Equals, 3) - - buffer.Release(h2) - - v, _ = buffer.Get([]byte("yz")) - c.Assert(len(v), Equals, 1) - - buffer.Cleanup(h1) - - v, _ = buffer.Get([]byte("x")) - c.Assert(len(v), Equals, 2) -} - -func (s *testKVSuite) TestBufferLimit(c *C) { - buffer := newMemDB() - buffer.bufferSizeLimit = 1000 - buffer.entrySizeLimit = 500 - - err := buffer.Set([]byte("x"), make([]byte, 500)) - c.Assert(err, NotNil) // entry size limit - - err = buffer.Set([]byte("x"), make([]byte, 499)) - c.Assert(err, IsNil) - err = buffer.Set([]byte("yz"), make([]byte, 499)) - c.Assert(err, NotNil) // buffer size limit - - err = buffer.Delete(make([]byte, 499)) - c.Assert(err, IsNil) - - err = buffer.Delete(make([]byte, 500)) - c.Assert(err, NotNil) -} diff --git a/store/tikv/unionstore/mock.go b/store/tikv/unionstore/mock.go deleted file mode 100644 index e62b1a4108147..0000000000000 --- a/store/tikv/unionstore/mock.go +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "context" - - tikverr "github.com/pingcap/tidb/store/tikv/error" -) - -type mockSnapshot struct { - store *MemDB -} - -func (s *mockSnapshot) Get(_ context.Context, k []byte) ([]byte, error) { - return s.store.Get(k) -} - -func (s *mockSnapshot) SetPriority(priority int) { - -} - -func (s *mockSnapshot) BatchGet(_ context.Context, keys [][]byte) (map[string][]byte, error) { - m := make(map[string][]byte, len(keys)) - for _, k := range keys { - v, err := s.store.Get(k) - if tikverr.IsErrNotFound(err) { - continue - } - if err != nil { - return nil, err - } - m[string(k)] = v - } - return m, nil -} - -func (s *mockSnapshot) Iter(k []byte, upperBound []byte) (Iterator, error) { - return s.store.Iter(k, upperBound) -} - -func (s *mockSnapshot) IterReverse(k []byte) (Iterator, error) { - return s.store.IterReverse(k) -} - -func (s *mockSnapshot) SetOption(opt int, val interface{}) {} diff --git a/store/tikv/unionstore/union_iter.go b/store/tikv/unionstore/union_iter.go deleted file mode 100644 index 00b7eec0eefd5..0000000000000 --- a/store/tikv/unionstore/union_iter.go +++ /dev/null @@ -1,187 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "go.uber.org/zap" -) - -// UnionIter is the iterator on an UnionStore. -type UnionIter struct { - dirtyIt Iterator - snapshotIt Iterator - - dirtyValid bool - snapshotValid bool - - curIsDirty bool - isValid bool - reverse bool -} - -// NewUnionIter returns a union iterator for BufferStore. -func NewUnionIter(dirtyIt Iterator, snapshotIt Iterator, reverse bool) (*UnionIter, error) { - it := &UnionIter{ - dirtyIt: dirtyIt, - snapshotIt: snapshotIt, - dirtyValid: dirtyIt.Valid(), - snapshotValid: snapshotIt.Valid(), - reverse: reverse, - } - err := it.updateCur() - if err != nil { - return nil, err - } - return it, nil -} - -// dirtyNext makes iter.dirtyIt go and update valid status. -func (iter *UnionIter) dirtyNext() error { - err := iter.dirtyIt.Next() - iter.dirtyValid = iter.dirtyIt.Valid() - return err -} - -// snapshotNext makes iter.snapshotIt go and update valid status. -func (iter *UnionIter) snapshotNext() error { - err := iter.snapshotIt.Next() - iter.snapshotValid = iter.snapshotIt.Valid() - return err -} - -func (iter *UnionIter) updateCur() error { - iter.isValid = true - for { - if !iter.dirtyValid && !iter.snapshotValid { - iter.isValid = false - break - } - - if !iter.dirtyValid { - iter.curIsDirty = false - break - } - - if !iter.snapshotValid { - iter.curIsDirty = true - // if delete it - if len(iter.dirtyIt.Value()) == 0 { - if err := iter.dirtyNext(); err != nil { - return err - } - continue - } - break - } - - // both valid - if iter.snapshotValid && iter.dirtyValid { - snapshotKey := iter.snapshotIt.Key() - dirtyKey := iter.dirtyIt.Key() - cmp := kv.CmpKey(dirtyKey, snapshotKey) - if iter.reverse { - cmp = -cmp - } - // if equal, means both have value - if cmp == 0 { - if len(iter.dirtyIt.Value()) == 0 { - // snapshot has a record, but txn says we have deleted it - // just go next - if err := iter.dirtyNext(); err != nil { - return err - } - if err := iter.snapshotNext(); err != nil { - return err - } - continue - } - // both go next - if err := iter.snapshotNext(); err != nil { - return err - } - iter.curIsDirty = true - break - } else if cmp > 0 { - // record from snapshot comes first - iter.curIsDirty = false - break - } else { - // record from dirty comes first - if len(iter.dirtyIt.Value()) == 0 { - logutil.BgLogger().Warn("delete a record not exists?", - zap.String("key", kv.StrKey(iter.dirtyIt.Key()))) - // jump over this deletion - if err := iter.dirtyNext(); err != nil { - return err - } - continue - } - iter.curIsDirty = true - break - } - } - } - return nil -} - -// Next implements the Iterator Next interface. -func (iter *UnionIter) Next() error { - var err error - if !iter.curIsDirty { - err = iter.snapshotNext() - } else { - err = iter.dirtyNext() - } - if err != nil { - return err - } - err = iter.updateCur() - return err -} - -// Value implements the Iterator Value interface. -// Multi columns -func (iter *UnionIter) Value() []byte { - if !iter.curIsDirty { - return iter.snapshotIt.Value() - } - return iter.dirtyIt.Value() -} - -// Key implements the Iterator Key interface. -func (iter *UnionIter) Key() []byte { - if !iter.curIsDirty { - return iter.snapshotIt.Key() - } - return iter.dirtyIt.Key() -} - -// Valid implements the Iterator Valid interface. -func (iter *UnionIter) Valid() bool { - return iter.isValid -} - -// Close implements the Iterator Close interface. -func (iter *UnionIter) Close() { - if iter.snapshotIt != nil { - iter.snapshotIt.Close() - iter.snapshotIt = nil - } - if iter.dirtyIt != nil { - iter.dirtyIt.Close() - iter.dirtyIt = nil - } -} diff --git a/store/tikv/unionstore/union_store.go b/store/tikv/unionstore/union_store.go deleted file mode 100644 index 08354975e38c5..0000000000000 --- a/store/tikv/unionstore/union_store.go +++ /dev/null @@ -1,136 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "context" - - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" -) - -// Iterator is the interface for a iterator on KV store. -type Iterator interface { - Valid() bool - Key() []byte - Value() []byte - Next() error - Close() -} - -// Getter is the interface for the Get method. -type Getter interface { - // Get gets the value for key k from kv store. - // If corresponding kv pair does not exist, it returns nil and ErrNotExist. - Get(k []byte) ([]byte, error) -} - -// uSnapshot defines the interface for the snapshot fetched from KV store. -type uSnapshot interface { - // Get gets the value for key k from kv store. - // If corresponding kv pair does not exist, it returns nil and ErrNotExist. - Get(ctx context.Context, k []byte) ([]byte, error) - // Iter creates an Iterator positioned on the first entry that k <= entry's key. - // If such entry is not found, it returns an invalid Iterator with no error. - // It yields only keys that < upperBound. If upperBound is nil, it means the upperBound is unbounded. - // The Iterator must be Closed after use. - Iter(k []byte, upperBound []byte) (Iterator, error) - - // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. - // The returned iterator will iterate from greater key to smaller key. - // If k is nil, the returned iterator will be positioned at the last key. - // TODO: Add lower bound limit - IterReverse(k []byte) (Iterator, error) -} - -// KVUnionStore is an in-memory Store which contains a buffer for write and a -// snapshot for read. -type KVUnionStore struct { - memBuffer *MemDB - snapshot uSnapshot -} - -// NewUnionStore builds a new unionStore. -func NewUnionStore(snapshot uSnapshot) *KVUnionStore { - return &KVUnionStore{ - snapshot: snapshot, - memBuffer: newMemDB(), - } -} - -// GetMemBuffer return the MemBuffer binding to this unionStore. -func (us *KVUnionStore) GetMemBuffer() *MemDB { - return us.memBuffer -} - -// Get implements the Retriever interface. -func (us *KVUnionStore) Get(ctx context.Context, k []byte) ([]byte, error) { - v, err := us.memBuffer.Get(k) - if tikverr.IsErrNotFound(err) { - v, err = us.snapshot.Get(ctx, k) - } - if err != nil { - return v, err - } - if len(v) == 0 { - return nil, tikverr.ErrNotExist - } - return v, nil -} - -// Iter implements the Retriever interface. -func (us *KVUnionStore) Iter(k, upperBound []byte) (Iterator, error) { - bufferIt, err := us.memBuffer.Iter(k, upperBound) - if err != nil { - return nil, err - } - retrieverIt, err := us.snapshot.Iter(k, upperBound) - if err != nil { - return nil, err - } - return NewUnionIter(bufferIt, retrieverIt, false) -} - -// IterReverse implements the Retriever interface. -func (us *KVUnionStore) IterReverse(k []byte) (Iterator, error) { - bufferIt, err := us.memBuffer.IterReverse(k) - if err != nil { - return nil, err - } - retrieverIt, err := us.snapshot.IterReverse(k) - if err != nil { - return nil, err - } - return NewUnionIter(bufferIt, retrieverIt, true) -} - -// HasPresumeKeyNotExists gets the key exist error info for the lazy check. -func (us *KVUnionStore) HasPresumeKeyNotExists(k []byte) bool { - flags, err := us.memBuffer.GetFlags(k) - if err != nil { - return false - } - return flags.HasPresumeKeyNotExists() -} - -// UnmarkPresumeKeyNotExists deletes the key exist error info for the lazy check. -func (us *KVUnionStore) UnmarkPresumeKeyNotExists(k []byte) { - us.memBuffer.UpdateFlags(k, kv.DelPresumeKeyNotExists) -} - -// SetEntrySizeLimit sets the size limit for each entry and total buffer. -func (us *KVUnionStore) SetEntrySizeLimit(entryLimit, bufferLimit uint64) { - us.memBuffer.entrySizeLimit = entryLimit - us.memBuffer.bufferSizeLimit = bufferLimit -} diff --git a/store/tikv/unionstore/union_store_test.go b/store/tikv/unionstore/union_store_test.go deleted file mode 100644 index e45500254ef2c..0000000000000 --- a/store/tikv/unionstore/union_store_test.go +++ /dev/null @@ -1,139 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package unionstore - -import ( - "context" - - . "github.com/pingcap/check" - tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/util/testleak" -) - -var _ = Suite(&testUnionStoreSuite{}) - -type testUnionStoreSuite struct { - store *MemDB - us *KVUnionStore -} - -func (s *testUnionStoreSuite) SetUpTest(c *C) { - s.store = newMemDB() - s.us = NewUnionStore(&mockSnapshot{s.store}) -} - -func (s *testUnionStoreSuite) TestGetSet(c *C) { - defer testleak.AfterTest(c)() - err := s.store.Set([]byte("1"), []byte("1")) - c.Assert(err, IsNil) - v, err := s.us.Get(context.TODO(), []byte("1")) - c.Assert(err, IsNil) - c.Assert(v, BytesEquals, []byte("1")) - err = s.us.GetMemBuffer().Set([]byte("1"), []byte("2")) - c.Assert(err, IsNil) - v, err = s.us.Get(context.TODO(), []byte("1")) - c.Assert(err, IsNil) - c.Assert(v, BytesEquals, []byte("2")) - c.Assert(s.us.GetMemBuffer().Size(), Equals, 2) - c.Assert(s.us.GetMemBuffer().Len(), Equals, 1) -} - -func (s *testUnionStoreSuite) TestDelete(c *C) { - defer testleak.AfterTest(c)() - err := s.store.Set([]byte("1"), []byte("1")) - c.Assert(err, IsNil) - err = s.us.GetMemBuffer().Delete([]byte("1")) - c.Assert(err, IsNil) - _, err = s.us.Get(context.TODO(), []byte("1")) - c.Assert(tikverr.IsErrNotFound(err), IsTrue) - - err = s.us.GetMemBuffer().Set([]byte("1"), []byte("2")) - c.Assert(err, IsNil) - v, err := s.us.Get(context.TODO(), []byte("1")) - c.Assert(err, IsNil) - c.Assert(v, BytesEquals, []byte("2")) -} - -func (s *testUnionStoreSuite) TestSeek(c *C) { - defer testleak.AfterTest(c)() - err := s.store.Set([]byte("1"), []byte("1")) - c.Assert(err, IsNil) - err = s.store.Set([]byte("2"), []byte("2")) - c.Assert(err, IsNil) - err = s.store.Set([]byte("3"), []byte("3")) - c.Assert(err, IsNil) - - iter, err := s.us.Iter(nil, nil) - c.Assert(err, IsNil) - checkIterator(c, iter, [][]byte{[]byte("1"), []byte("2"), []byte("3")}, [][]byte{[]byte("1"), []byte("2"), []byte("3")}) - - iter, err = s.us.Iter([]byte("2"), nil) - c.Assert(err, IsNil) - checkIterator(c, iter, [][]byte{[]byte("2"), []byte("3")}, [][]byte{[]byte("2"), []byte("3")}) - - err = s.us.GetMemBuffer().Set([]byte("4"), []byte("4")) - c.Assert(err, IsNil) - iter, err = s.us.Iter([]byte("2"), nil) - c.Assert(err, IsNil) - checkIterator(c, iter, [][]byte{[]byte("2"), []byte("3"), []byte("4")}, [][]byte{[]byte("2"), []byte("3"), []byte("4")}) - - err = s.us.GetMemBuffer().Delete([]byte("3")) - c.Assert(err, IsNil) - iter, err = s.us.Iter([]byte("2"), nil) - c.Assert(err, IsNil) - checkIterator(c, iter, [][]byte{[]byte("2"), []byte("4")}, [][]byte{[]byte("2"), []byte("4")}) -} - -func (s *testUnionStoreSuite) TestIterReverse(c *C) { - defer testleak.AfterTest(c)() - err := s.store.Set([]byte("1"), []byte("1")) - c.Assert(err, IsNil) - err = s.store.Set([]byte("2"), []byte("2")) - c.Assert(err, IsNil) - err = s.store.Set([]byte("3"), []byte("3")) - c.Assert(err, IsNil) - - iter, err := s.us.IterReverse(nil) - c.Assert(err, IsNil) - checkIterator(c, iter, [][]byte{[]byte("3"), []byte("2"), []byte("1")}, [][]byte{[]byte("3"), []byte("2"), []byte("1")}) - - iter, err = s.us.IterReverse([]byte("3")) - c.Assert(err, IsNil) - checkIterator(c, iter, [][]byte{[]byte("2"), []byte("1")}, [][]byte{[]byte("2"), []byte("1")}) - - err = s.us.GetMemBuffer().Set([]byte("0"), []byte("0")) - c.Assert(err, IsNil) - iter, err = s.us.IterReverse([]byte("3")) - c.Assert(err, IsNil) - checkIterator(c, iter, [][]byte{[]byte("2"), []byte("1"), []byte("0")}, [][]byte{[]byte("2"), []byte("1"), []byte("0")}) - - err = s.us.GetMemBuffer().Delete([]byte("1")) - c.Assert(err, IsNil) - iter, err = s.us.IterReverse([]byte("3")) - c.Assert(err, IsNil) - checkIterator(c, iter, [][]byte{[]byte("2"), []byte("0")}, [][]byte{[]byte("2"), []byte("0")}) -} - -func checkIterator(c *C, iter Iterator, keys [][]byte, values [][]byte) { - defer iter.Close() - c.Assert(len(keys), Equals, len(values)) - for i, k := range keys { - v := values[i] - c.Assert(iter.Valid(), IsTrue) - c.Assert(iter.Key(), BytesEquals, k) - c.Assert(iter.Value(), BytesEquals, v) - c.Assert(iter.Next(), IsNil) - } - c.Assert(iter.Valid(), IsFalse) -} diff --git a/store/tikv/util/codec/bytes.go b/store/tikv/util/codec/bytes.go deleted file mode 100644 index 58c9890d724d9..0000000000000 --- a/store/tikv/util/codec/bytes.go +++ /dev/null @@ -1,170 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package codec - -import ( - "runtime" - "unsafe" - - "github.com/pingcap/errors" -) - -const ( - encGroupSize = 8 - encMarker = byte(0xFF) - encPad = byte(0x0) -) - -var ( - pads = make([]byte, encGroupSize) -) - -// EncodeBytes guarantees the encoded value is in ascending order for comparison, -// encoding with the following rule: -// [group1][marker1]...[groupN][markerN] -// group is 8 bytes slice which is padding with 0. -// marker is `0xFF - padding 0 count` -// For example: -// [] -> [0, 0, 0, 0, 0, 0, 0, 0, 247] -// [1, 2, 3] -> [1, 2, 3, 0, 0, 0, 0, 0, 250] -// [1, 2, 3, 0] -> [1, 2, 3, 0, 0, 0, 0, 0, 251] -// [1, 2, 3, 4, 5, 6, 7, 8] -> [1, 2, 3, 4, 5, 6, 7, 8, 255, 0, 0, 0, 0, 0, 0, 0, 0, 247] -// Refer: https://github.com/facebook/mysql-5.6/wiki/MyRocks-record-format#memcomparable-format -func EncodeBytes(b []byte, data []byte) []byte { - // Allocate more space to avoid unnecessary slice growing. - // Assume that the byte slice size is about `(len(data) / encGroupSize + 1) * (encGroupSize + 1)` bytes, - // that is `(len(data) / 8 + 1) * 9` in our implement. - dLen := len(data) - reallocSize := (dLen/encGroupSize + 1) * (encGroupSize + 1) - result := reallocBytes(b, reallocSize) - for idx := 0; idx <= dLen; idx += encGroupSize { - remain := dLen - idx - padCount := 0 - if remain >= encGroupSize { - result = append(result, data[idx:idx+encGroupSize]...) - } else { - padCount = encGroupSize - remain - result = append(result, data[idx:]...) - result = append(result, pads[:padCount]...) - } - - marker := encMarker - byte(padCount) - result = append(result, marker) - } - - return result -} - -func decodeBytes(b []byte, buf []byte, reverse bool) ([]byte, []byte, error) { - if buf == nil { - buf = make([]byte, 0, len(b)) - } - buf = buf[:0] - for { - if len(b) < encGroupSize+1 { - return nil, nil, errors.New("insufficient bytes to decode value") - } - - groupBytes := b[:encGroupSize+1] - - group := groupBytes[:encGroupSize] - marker := groupBytes[encGroupSize] - - var padCount byte - if reverse { - padCount = marker - } else { - padCount = encMarker - marker - } - if padCount > encGroupSize { - return nil, nil, errors.Errorf("invalid marker byte, group bytes %q", groupBytes) - } - - realGroupSize := encGroupSize - padCount - buf = append(buf, group[:realGroupSize]...) - b = b[encGroupSize+1:] - - if padCount != 0 { - var padByte = encPad - if reverse { - padByte = encMarker - } - // Check validity of padding bytes. - for _, v := range group[realGroupSize:] { - if v != padByte { - return nil, nil, errors.Errorf("invalid padding byte, group bytes %q", groupBytes) - } - } - break - } - } - if reverse { - reverseBytes(buf) - } - return b, buf, nil -} - -// DecodeBytes decodes bytes which is encoded by EncodeBytes before, -// returns the leftover bytes and decoded value if no error. -// `buf` is used to buffer data to avoid the cost of makeslice in decodeBytes when DecodeBytes is called by Decoder.DecodeOne. -func DecodeBytes(b []byte, buf []byte) ([]byte, []byte, error) { - return decodeBytes(b, buf, false) -} - -// See https://golang.org/src/crypto/cipher/xor.go -const wordSize = int(unsafe.Sizeof(uintptr(0))) -const supportsUnaligned = runtime.GOARCH == "386" || runtime.GOARCH == "amd64" - -func fastReverseBytes(b []byte) { - n := len(b) - w := n / wordSize - if w > 0 { - bw := *(*[]uintptr)(unsafe.Pointer(&b)) - for i := 0; i < w; i++ { - bw[i] = ^bw[i] - } - } - - for i := w * wordSize; i < n; i++ { - b[i] = ^b[i] - } -} - -func safeReverseBytes(b []byte) { - for i := range b { - b[i] = ^b[i] - } -} - -func reverseBytes(b []byte) { - if supportsUnaligned { - fastReverseBytes(b) - return - } - - safeReverseBytes(b) -} - -// reallocBytes is like realloc. -func reallocBytes(b []byte, n int) []byte { - newSize := len(b) + n - if cap(b) < newSize { - bs := make([]byte, len(b), newSize) - copy(bs, b) - return bs - } - - // slice b has capability to store n bytes - return b -} diff --git a/store/tikv/util/codec/number.go b/store/tikv/util/codec/number.go deleted file mode 100644 index 47912e813b2fc..0000000000000 --- a/store/tikv/util/codec/number.go +++ /dev/null @@ -1,284 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package codec - -import ( - "encoding/binary" - "math" - - "github.com/pingcap/errors" -) - -const signMask uint64 = 0x8000000000000000 - -// EncodeIntToCmpUint make int v to comparable uint type -func EncodeIntToCmpUint(v int64) uint64 { - return uint64(v) ^ signMask -} - -// DecodeCmpUintToInt decodes the u that encoded by EncodeIntToCmpUint -func DecodeCmpUintToInt(u uint64) int64 { - return int64(u ^ signMask) -} - -// EncodeInt appends the encoded value to slice b and returns the appended slice. -// EncodeInt guarantees that the encoded value is in ascending order for comparison. -func EncodeInt(b []byte, v int64) []byte { - var data [8]byte - u := EncodeIntToCmpUint(v) - binary.BigEndian.PutUint64(data[:], u) - return append(b, data[:]...) -} - -// EncodeIntDesc appends the encoded value to slice b and returns the appended slice. -// EncodeIntDesc guarantees that the encoded value is in descending order for comparison. -func EncodeIntDesc(b []byte, v int64) []byte { - var data [8]byte - u := EncodeIntToCmpUint(v) - binary.BigEndian.PutUint64(data[:], ^u) - return append(b, data[:]...) -} - -// DecodeInt decodes value encoded by EncodeInt before. -// It returns the leftover un-decoded slice, decoded value if no error. -func DecodeInt(b []byte) ([]byte, int64, error) { - if len(b) < 8 { - return nil, 0, errors.New("insufficient bytes to decode value") - } - - u := binary.BigEndian.Uint64(b[:8]) - v := DecodeCmpUintToInt(u) - b = b[8:] - return b, v, nil -} - -// DecodeIntDesc decodes value encoded by EncodeInt before. -// It returns the leftover un-decoded slice, decoded value if no error. -func DecodeIntDesc(b []byte) ([]byte, int64, error) { - if len(b) < 8 { - return nil, 0, errors.New("insufficient bytes to decode value") - } - - u := binary.BigEndian.Uint64(b[:8]) - v := DecodeCmpUintToInt(^u) - b = b[8:] - return b, v, nil -} - -// EncodeUint appends the encoded value to slice b and returns the appended slice. -// EncodeUint guarantees that the encoded value is in ascending order for comparison. -func EncodeUint(b []byte, v uint64) []byte { - var data [8]byte - binary.BigEndian.PutUint64(data[:], v) - return append(b, data[:]...) -} - -// EncodeUintDesc appends the encoded value to slice b and returns the appended slice. -// EncodeUintDesc guarantees that the encoded value is in descending order for comparison. -func EncodeUintDesc(b []byte, v uint64) []byte { - var data [8]byte - binary.BigEndian.PutUint64(data[:], ^v) - return append(b, data[:]...) -} - -// DecodeUint decodes value encoded by EncodeUint before. -// It returns the leftover un-decoded slice, decoded value if no error. -func DecodeUint(b []byte) ([]byte, uint64, error) { - if len(b) < 8 { - return nil, 0, errors.New("insufficient bytes to decode value") - } - - v := binary.BigEndian.Uint64(b[:8]) - b = b[8:] - return b, v, nil -} - -// DecodeUintDesc decodes value encoded by EncodeInt before. -// It returns the leftover un-decoded slice, decoded value if no error. -func DecodeUintDesc(b []byte) ([]byte, uint64, error) { - if len(b) < 8 { - return nil, 0, errors.New("insufficient bytes to decode value") - } - - data := b[:8] - v := binary.BigEndian.Uint64(data) - b = b[8:] - return b, ^v, nil -} - -// EncodeVarint appends the encoded value to slice b and returns the appended slice. -// Note that the encoded result is not memcomparable. -func EncodeVarint(b []byte, v int64) []byte { - var data [binary.MaxVarintLen64]byte - n := binary.PutVarint(data[:], v) - return append(b, data[:n]...) -} - -// DecodeVarint decodes value encoded by EncodeVarint before. -// It returns the leftover un-decoded slice, decoded value if no error. -func DecodeVarint(b []byte) ([]byte, int64, error) { - v, n := binary.Varint(b) - if n > 0 { - return b[n:], v, nil - } - if n < 0 { - return nil, 0, errors.New("value larger than 64 bits") - } - return nil, 0, errors.New("insufficient bytes to decode value") -} - -// EncodeUvarint appends the encoded value to slice b and returns the appended slice. -// Note that the encoded result is not memcomparable. -func EncodeUvarint(b []byte, v uint64) []byte { - var data [binary.MaxVarintLen64]byte - n := binary.PutUvarint(data[:], v) - return append(b, data[:n]...) -} - -// DecodeUvarint decodes value encoded by EncodeUvarint before. -// It returns the leftover un-decoded slice, decoded value if no error. -func DecodeUvarint(b []byte) ([]byte, uint64, error) { - v, n := binary.Uvarint(b) - if n > 0 { - return b[n:], v, nil - } - if n < 0 { - return nil, 0, errors.New("value larger than 64 bits") - } - return nil, 0, errors.New("insufficient bytes to decode value") -} - -const ( - negativeTagEnd = 8 // negative tag is (negativeTagEnd - length). - positiveTagStart = 0xff - 8 // Positive tag is (positiveTagStart + length). -) - -// EncodeComparableVarint encodes an int64 to a mem-comparable bytes. -func EncodeComparableVarint(b []byte, v int64) []byte { - if v < 0 { - // All negative value has a tag byte prefix (negativeTagEnd - length). - // Smaller negative value encodes to more bytes, has smaller tag. - if v >= -0xff { - return append(b, negativeTagEnd-1, byte(v)) - } else if v >= -0xffff { - return append(b, negativeTagEnd-2, byte(v>>8), byte(v)) - } else if v >= -0xffffff { - return append(b, negativeTagEnd-3, byte(v>>16), byte(v>>8), byte(v)) - } else if v >= -0xffffffff { - return append(b, negativeTagEnd-4, byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) - } else if v >= -0xffffffffff { - return append(b, negativeTagEnd-5, byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) - } else if v >= -0xffffffffffff { - return append(b, negativeTagEnd-6, byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8), - byte(v)) - } else if v >= -0xffffffffffffff { - return append(b, negativeTagEnd-7, byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16), - byte(v>>8), byte(v)) - } - return append(b, negativeTagEnd-8, byte(v>>56), byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24), - byte(v>>16), byte(v>>8), byte(v)) - } - return EncodeComparableUvarint(b, uint64(v)) -} - -// EncodeComparableUvarint encodes uint64 into mem-comparable bytes. -func EncodeComparableUvarint(b []byte, v uint64) []byte { - // The first byte has 256 values, [0, 7] is reserved for negative tags, - // [248, 255] is reserved for larger positive tags, - // So we can store value [0, 239] in a single byte. - // Values cannot be stored in single byte has a tag byte prefix (positiveTagStart+length). - // Larger value encodes to more bytes, has larger tag. - if v <= positiveTagStart-negativeTagEnd { - return append(b, byte(v)+negativeTagEnd) - } else if v <= 0xff { - return append(b, positiveTagStart+1, byte(v)) - } else if v <= 0xffff { - return append(b, positiveTagStart+2, byte(v>>8), byte(v)) - } else if v <= 0xffffff { - return append(b, positiveTagStart+3, byte(v>>16), byte(v>>8), byte(v)) - } else if v <= 0xffffffff { - return append(b, positiveTagStart+4, byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) - } else if v <= 0xffffffffff { - return append(b, positiveTagStart+5, byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) - } else if v <= 0xffffffffffff { - return append(b, positiveTagStart+6, byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8), - byte(v)) - } else if v <= 0xffffffffffffff { - return append(b, positiveTagStart+7, byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16), - byte(v>>8), byte(v)) - } - return append(b, positiveTagStart+8, byte(v>>56), byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24), - byte(v>>16), byte(v>>8), byte(v)) -} - -var ( - errDecodeInsufficient = errors.New("insufficient bytes to decode value") - errDecodeInvalid = errors.New("invalid bytes to decode value") -) - -// DecodeComparableUvarint decodes mem-comparable uvarint. -func DecodeComparableUvarint(b []byte) ([]byte, uint64, error) { - if len(b) == 0 { - return nil, 0, errDecodeInsufficient - } - first := b[0] - b = b[1:] - if first < negativeTagEnd { - return nil, 0, errors.Trace(errDecodeInvalid) - } - if first <= positiveTagStart { - return b, uint64(first) - negativeTagEnd, nil - } - length := int(first) - positiveTagStart - if len(b) < length { - return nil, 0, errors.Trace(errDecodeInsufficient) - } - var v uint64 - for _, c := range b[:length] { - v = (v << 8) | uint64(c) - } - return b[length:], v, nil -} - -// DecodeComparableVarint decodes mem-comparable varint. -func DecodeComparableVarint(b []byte) ([]byte, int64, error) { - if len(b) == 0 { - return nil, 0, errors.Trace(errDecodeInsufficient) - } - first := b[0] - if first >= negativeTagEnd && first <= positiveTagStart { - return b, int64(first) - negativeTagEnd, nil - } - b = b[1:] - var length int - var v uint64 - if first < negativeTagEnd { - length = negativeTagEnd - int(first) - v = math.MaxUint64 // negative value has all bits on by default. - } else { - length = int(first) - positiveTagStart - } - if len(b) < length { - return nil, 0, errors.Trace(errDecodeInsufficient) - } - for _, c := range b[:length] { - v = (v << 8) | uint64(c) - } - if first > positiveTagStart && v > math.MaxInt64 { - return nil, 0, errors.Trace(errDecodeInvalid) - } else if first < negativeTagEnd && v <= math.MaxInt64 { - return nil, 0, errors.Trace(errDecodeInvalid) - } - return b[length:], int64(v), nil -} diff --git a/store/tikv/util/execdetails.go b/store/tikv/util/execdetails.go deleted file mode 100644 index 0f2dab372ff16..0000000000000 --- a/store/tikv/util/execdetails.go +++ /dev/null @@ -1,304 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "bytes" - "math" - "strconv" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/kvproto/pkg/kvrpcpb" -) - -type commitDetailCtxKeyType struct{} -type lockKeysDetailCtxKeyType struct{} -type execDetailsCtxKeyType struct{} - -var ( - // CommitDetailCtxKey presents CommitDetail info key in context. - CommitDetailCtxKey = commitDetailCtxKeyType{} - - // LockKeysDetailCtxKey presents LockKeysDetail info key in context. - LockKeysDetailCtxKey = lockKeysDetailCtxKeyType{} - - // ExecDetailsKey presents ExecDetail info key in context. - ExecDetailsKey = execDetailsCtxKeyType{} -) - -// CommitDetails contains commit detail information. -type CommitDetails struct { - GetCommitTsTime time.Duration - PrewriteTime time.Duration - WaitPrewriteBinlogTime time.Duration - CommitTime time.Duration - LocalLatchTime time.Duration - Mu struct { - sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string - } - ResolveLockTime int64 - WriteKeys int - WriteSize int - PrewriteRegionNum int32 - TxnRetry int -} - -// Merge merges commit details into itself. -func (cd *CommitDetails) Merge(other *CommitDetails) { - cd.GetCommitTsTime += other.GetCommitTsTime - cd.PrewriteTime += other.PrewriteTime - cd.WaitPrewriteBinlogTime += other.WaitPrewriteBinlogTime - cd.CommitTime += other.CommitTime - cd.LocalLatchTime += other.LocalLatchTime - cd.ResolveLockTime += other.ResolveLockTime - cd.WriteKeys += other.WriteKeys - cd.WriteSize += other.WriteSize - cd.PrewriteRegionNum += other.PrewriteRegionNum - cd.TxnRetry += other.TxnRetry - cd.Mu.CommitBackoffTime += other.Mu.CommitBackoffTime - cd.Mu.BackoffTypes = append(cd.Mu.BackoffTypes, other.Mu.BackoffTypes...) -} - -// Clone returns a deep copy of itself. -func (cd *CommitDetails) Clone() *CommitDetails { - commit := &CommitDetails{ - GetCommitTsTime: cd.GetCommitTsTime, - PrewriteTime: cd.PrewriteTime, - WaitPrewriteBinlogTime: cd.WaitPrewriteBinlogTime, - CommitTime: cd.CommitTime, - LocalLatchTime: cd.LocalLatchTime, - ResolveLockTime: cd.ResolveLockTime, - WriteKeys: cd.WriteKeys, - WriteSize: cd.WriteSize, - PrewriteRegionNum: cd.PrewriteRegionNum, - TxnRetry: cd.TxnRetry, - } - commit.Mu.BackoffTypes = append([]string{}, cd.Mu.BackoffTypes...) - commit.Mu.CommitBackoffTime = cd.Mu.CommitBackoffTime - return commit -} - -// LockKeysDetails contains pessimistic lock keys detail information. -type LockKeysDetails struct { - TotalTime time.Duration - RegionNum int32 - LockKeys int32 - ResolveLockTime int64 - BackoffTime int64 - Mu struct { - sync.Mutex - BackoffTypes []string - } - LockRPCTime int64 - LockRPCCount int64 - RetryCount int -} - -// Merge merges lock keys execution details into self. -func (ld *LockKeysDetails) Merge(lockKey *LockKeysDetails) { - ld.TotalTime += lockKey.TotalTime - ld.RegionNum += lockKey.RegionNum - ld.LockKeys += lockKey.LockKeys - ld.ResolveLockTime += lockKey.ResolveLockTime - ld.BackoffTime += lockKey.BackoffTime - ld.LockRPCTime += lockKey.LockRPCTime - ld.LockRPCCount += ld.LockRPCCount - ld.Mu.BackoffTypes = append(ld.Mu.BackoffTypes, lockKey.Mu.BackoffTypes...) - ld.RetryCount++ -} - -// Clone returns a deep copy of itself. -func (ld *LockKeysDetails) Clone() *LockKeysDetails { - lock := &LockKeysDetails{ - TotalTime: ld.TotalTime, - RegionNum: ld.RegionNum, - LockKeys: ld.LockKeys, - ResolveLockTime: ld.ResolveLockTime, - BackoffTime: ld.BackoffTime, - LockRPCTime: ld.LockRPCTime, - LockRPCCount: ld.LockRPCCount, - RetryCount: ld.RetryCount, - } - lock.Mu.BackoffTypes = append([]string{}, ld.Mu.BackoffTypes...) - return lock -} - -// ExecDetails contains execution detail info. -type ExecDetails struct { - BackoffCount int64 - BackoffDuration int64 - WaitKVRespDuration int64 - WaitPDRespDuration int64 -} - -// FormatDuration uses to format duration, this function will prune precision before format duration. -// Pruning precision is for human readability. The prune rule is: -// 1. if the duration was less than 1us, return the original string. -// 2. readable value >=10, keep 1 decimal, otherwise, keep 2 decimal. such as: -// 9.412345ms -> 9.41ms -// 10.412345ms -> 10.4ms -// 5.999s -> 6s -// 100.45µs -> 100.5µs -func FormatDuration(d time.Duration) string { - if d <= time.Microsecond { - return d.String() - } - unit := getUnit(d) - if unit == time.Nanosecond { - return d.String() - } - integer := (d / unit) * unit - decimal := float64(d%unit) / float64(unit) - if d < 10*unit { - decimal = math.Round(decimal*100) / 100 - } else { - decimal = math.Round(decimal*10) / 10 - } - d = integer + time.Duration(decimal*float64(unit)) - return d.String() -} - -func getUnit(d time.Duration) time.Duration { - if d >= time.Second { - return time.Second - } else if d >= time.Millisecond { - return time.Millisecond - } else if d >= time.Microsecond { - return time.Microsecond - } - return time.Nanosecond -} - -// ScanDetail contains coprocessor scan detail information. -type ScanDetail struct { - // TotalKeys is the approximate number of MVCC keys meet during scanning. It includes - // deleted versions, but does not include RocksDB tombstone keys. - TotalKeys int64 - // ProcessedKeys is the number of user keys scanned from the storage. - // It does not include deleted version or RocksDB tombstone keys. - // For Coprocessor requests, it includes keys that has been filtered out by Selection. - ProcessedKeys int64 - // RocksdbDeleteSkippedCount is the total number of deletes and single deletes skipped over during - // iteration, i.e. how many RocksDB tombstones are skipped. - RocksdbDeleteSkippedCount uint64 - // RocksdbKeySkippedCount it the total number of internal keys skipped over during iteration. - RocksdbKeySkippedCount uint64 - // RocksdbBlockCacheHitCount is the total number of RocksDB block cache hits. - RocksdbBlockCacheHitCount uint64 - // RocksdbBlockReadCount is the total number of block reads (with IO). - RocksdbBlockReadCount uint64 - // RocksdbBlockReadByte is the total number of bytes from block reads. - RocksdbBlockReadByte uint64 -} - -// Merge merges scan detail execution details into self. -func (sd *ScanDetail) Merge(scanDetail *ScanDetail) { - atomic.AddInt64(&sd.TotalKeys, scanDetail.TotalKeys) - atomic.AddInt64(&sd.ProcessedKeys, scanDetail.ProcessedKeys) - atomic.AddUint64(&sd.RocksdbDeleteSkippedCount, scanDetail.RocksdbDeleteSkippedCount) - atomic.AddUint64(&sd.RocksdbKeySkippedCount, scanDetail.RocksdbKeySkippedCount) - atomic.AddUint64(&sd.RocksdbBlockCacheHitCount, scanDetail.RocksdbBlockCacheHitCount) - atomic.AddUint64(&sd.RocksdbBlockReadCount, scanDetail.RocksdbBlockReadCount) - atomic.AddUint64(&sd.RocksdbBlockReadByte, scanDetail.RocksdbBlockReadByte) -} - -var zeroScanDetail = ScanDetail{} - -// String implements the fmt.Stringer interface. -func (sd *ScanDetail) String() string { - if sd == nil || *sd == zeroScanDetail { - return "" - } - buf := bytes.NewBuffer(make([]byte, 0, 16)) - buf.WriteString("scan_detail: {") - buf.WriteString("total_process_keys: ") - buf.WriteString(strconv.FormatInt(sd.ProcessedKeys, 10)) - buf.WriteString(", total_keys: ") - buf.WriteString(strconv.FormatInt(sd.TotalKeys, 10)) - buf.WriteString(", rocksdb: {") - buf.WriteString("delete_skipped_count: ") - buf.WriteString(strconv.FormatUint(sd.RocksdbDeleteSkippedCount, 10)) - buf.WriteString(", key_skipped_count: ") - buf.WriteString(strconv.FormatUint(sd.RocksdbKeySkippedCount, 10)) - buf.WriteString(", block: {") - buf.WriteString("cache_hit_count: ") - buf.WriteString(strconv.FormatUint(sd.RocksdbBlockCacheHitCount, 10)) - buf.WriteString(", read_count: ") - buf.WriteString(strconv.FormatUint(sd.RocksdbBlockReadCount, 10)) - buf.WriteString(", read_byte: ") - buf.WriteString(FormatBytes(int64(sd.RocksdbBlockReadByte))) - buf.WriteString("}}}") - return buf.String() -} - -// MergeFromScanDetailV2 merges scan detail from pb into itself. -func (sd *ScanDetail) MergeFromScanDetailV2(scanDetail *kvrpcpb.ScanDetailV2) { - if scanDetail != nil { - sd.TotalKeys += int64(scanDetail.TotalVersions) - sd.ProcessedKeys += int64(scanDetail.ProcessedVersions) - sd.RocksdbDeleteSkippedCount += scanDetail.RocksdbDeleteSkippedCount - sd.RocksdbKeySkippedCount += scanDetail.RocksdbKeySkippedCount - sd.RocksdbBlockCacheHitCount += scanDetail.RocksdbBlockCacheHitCount - sd.RocksdbBlockReadCount += scanDetail.RocksdbBlockReadCount - sd.RocksdbBlockReadByte += scanDetail.RocksdbBlockReadByte - } -} - -// TimeDetail contains coprocessor time detail information. -type TimeDetail struct { - // WaitWallTimeMs is the off-cpu wall time which is elapsed in TiKV side. Usually this includes queue waiting time and - // other kind of waitings in series. - ProcessTime time.Duration - // Off-cpu and on-cpu wall time elapsed to actually process the request payload. It does not - // include `wait_wall_time`. - // This field is very close to the CPU time in most cases. Some wait time spend in RocksDB - // cannot be excluded for now, like Mutex wait time, which is included in this field, so that - // this field is called wall time instead of CPU time. - WaitTime time.Duration - // KvReadWallTimeMs is the time used in KV Scan/Get. - KvReadWallTimeMs time.Duration -} - -// String implements the fmt.Stringer interface. -func (td *TimeDetail) String() string { - if td == nil { - return "" - } - buf := bytes.NewBuffer(make([]byte, 0, 16)) - if td.ProcessTime > 0 { - buf.WriteString("total_process_time: ") - buf.WriteString(FormatDuration(td.ProcessTime)) - } - if td.WaitTime > 0 { - if buf.Len() > 0 { - buf.WriteString(", ") - } - buf.WriteString("total_wait_time: ") - buf.WriteString(FormatDuration(td.WaitTime)) - } - return buf.String() -} - -// MergeFromTimeDetail merges time detail from pb into itself. -func (td *TimeDetail) MergeFromTimeDetail(timeDetail *kvrpcpb.TimeDetail) { - if timeDetail != nil { - td.WaitTime += time.Duration(timeDetail.WaitWallTimeMs) * time.Millisecond - td.ProcessTime += time.Duration(timeDetail.ProcessWallTimeMs) * time.Millisecond - td.KvReadWallTimeMs += time.Duration(timeDetail.KvReadWallTimeMs) * time.Millisecond - } -} diff --git a/store/tikv/util/failpoint.go b/store/tikv/util/failpoint.go deleted file mode 100644 index fb097e9c0b400..0000000000000 --- a/store/tikv/util/failpoint.go +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "github.com/pingcap/failpoint" -) - -const failpointPrefix = "tikvclient/" - -// EvalFailpoint injects code for testing. It is used to replace `failpoint.Inject` -// to make it possible to be used in a library. -func EvalFailpoint(name string) (interface{}, error) { - return failpoint.Eval(failpointPrefix + name) -} diff --git a/store/tikv/util/misc.go b/store/tikv/util/misc.go deleted file mode 100644 index 6b56b974d65fb..0000000000000 --- a/store/tikv/util/misc.go +++ /dev/null @@ -1,135 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "context" - "fmt" - "strconv" - "strings" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv/logutil" - "go.uber.org/zap" -) - -// GCTimeFormat is the format that gc_worker used to store times. -const GCTimeFormat = "20060102-15:04:05 -0700" - -// CompatibleParseGCTime parses a string with `GCTimeFormat` and returns a time.Time. If `value` can't be parsed as that -// format, truncate to last space and try again. This function is only useful when loading times that saved by -// gc_worker. We have changed the format that gc_worker saves time (removed the last field), but when loading times it -// should be compatible with the old format. -func CompatibleParseGCTime(value string) (time.Time, error) { - t, err := time.Parse(GCTimeFormat, value) - - if err != nil { - // Remove the last field that separated by space - parts := strings.Split(value, " ") - prefix := strings.Join(parts[:len(parts)-1], " ") - t, err = time.Parse(GCTimeFormat, prefix) - } - - if err != nil { - err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\"", value, GCTimeFormat) - } - return t, err -} - -// WithRecovery wraps goroutine startup call with force recovery. -// it will dump current goroutine stack into log if catch any recover result. -// exec: execute logic function. -// recoverFn: handler will be called after recover and before dump stack, passing `nil` means noop. -func WithRecovery(exec func(), recoverFn func(r interface{})) { - defer func() { - r := recover() - if recoverFn != nil { - recoverFn(r) - } - if r != nil { - logutil.BgLogger().Error("panic in the recoverable goroutine", - zap.Reflect("r", r), - zap.Stack("stack trace")) - } - }() - exec() -} - -type sessionIDCtxKey struct{} - -// SessionID is the context key type to mark a session. -var SessionID = sessionIDCtxKey{} - -// SetSessionID sets session id into context -func SetSessionID(ctx context.Context, sessionID uint64) context.Context { - return context.WithValue(ctx, SessionID, sessionID) -} - -const ( - byteSizeGB = int64(1 << 30) - byteSizeMB = int64(1 << 20) - byteSizeKB = int64(1 << 10) - byteSizeBB = int64(1) -) - -// FormatBytes uses to format bytes, this function will prune precision before format bytes. -func FormatBytes(numBytes int64) string { - if numBytes <= byteSizeKB { - return BytesToString(numBytes) - } - unit, unitStr := getByteUnit(numBytes) - if unit == byteSizeBB { - return BytesToString(numBytes) - } - v := float64(numBytes) / float64(unit) - decimal := 1 - if numBytes%unit == 0 { - decimal = 0 - } else if v < 10 { - decimal = 2 - } - return fmt.Sprintf("%v %s", strconv.FormatFloat(v, 'f', decimal, 64), unitStr) -} - -func getByteUnit(b int64) (int64, string) { - if b > byteSizeGB { - return byteSizeGB, "GB" - } else if b > byteSizeMB { - return byteSizeMB, "MB" - } else if b > byteSizeKB { - return byteSizeKB, "KB" - } - return byteSizeBB, "Bytes" -} - -// BytesToString converts the memory consumption to a readable string. -func BytesToString(numBytes int64) string { - GB := float64(numBytes) / float64(byteSizeGB) - if GB > 1 { - return fmt.Sprintf("%v GB", GB) - } - - MB := float64(numBytes) / float64(byteSizeMB) - if MB > 1 { - return fmt.Sprintf("%v MB", MB) - } - - KB := float64(numBytes) / float64(byteSizeKB) - if KB > 1 { - return fmt.Sprintf("%v KB", KB) - } - - return fmt.Sprintf("%v Bytes", numBytes) -} diff --git a/store/tikv/util/misc_test.go b/store/tikv/util/misc_test.go deleted file mode 100644 index 2f0eeb63de767..0000000000000 --- a/store/tikv/util/misc_test.go +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "testing" - "time" - - . "github.com/pingcap/check" -) - -func TestT(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&testMiscSuite{}) - -type testMiscSuite struct { -} - -func (s *testMiscSuite) TestCompatibleParseGCTime(c *C) { - values := []string{ - "20181218-19:53:37 +0800 CST", - "20181218-19:53:37 +0800 MST", - "20181218-19:53:37 +0800 FOO", - "20181218-19:53:37 +0800 +08", - "20181218-19:53:37 +0800", - "20181218-19:53:37 +0800 ", - "20181218-11:53:37 +0000", - } - - invalidValues := []string{ - "", - " ", - "foo", - "20181218-11:53:37", - "20181218-19:53:37 +0800CST", - "20181218-19:53:37 +0800 FOO BAR", - "20181218-19:53:37 +0800FOOOOOOO BAR", - "20181218-19:53:37 ", - } - - expectedTime := time.Date(2018, 12, 18, 11, 53, 37, 0, time.UTC) - expectedTimeFormatted := "20181218-19:53:37 +0800" - - beijing, err := time.LoadLocation("Asia/Shanghai") - c.Assert(err, IsNil) - - for _, value := range values { - t, err := CompatibleParseGCTime(value) - c.Assert(err, IsNil) - c.Assert(t.Equal(expectedTime), Equals, true) - - formatted := t.In(beijing).Format(GCTimeFormat) - c.Assert(formatted, Equals, expectedTimeFormatted) - } - - for _, value := range invalidValues { - _, err := CompatibleParseGCTime(value) - c.Assert(err, NotNil) - } -} diff --git a/store/tikv/util/pd_interceptor.go b/store/tikv/util/pd_interceptor.go deleted file mode 100644 index 7f2731fed4bc6..0000000000000 --- a/store/tikv/util/pd_interceptor.go +++ /dev/null @@ -1,114 +0,0 @@ -// Copyright 2020 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "context" - "sync/atomic" - "time" - - "github.com/pingcap/kvproto/pkg/metapb" - pd "github.com/tikv/pd/client" -) - -var ( - _ pd.Client = &InterceptedPDClient{} - _ pd.TSFuture = &interceptedTsFuture{} -) - -func recordPDWaitTime(ctx context.Context, start time.Time) { - stmtExec := ctx.Value(ExecDetailsKey) - if stmtExec != nil { - detail := stmtExec.(*ExecDetails) - atomic.AddInt64(&detail.WaitPDRespDuration, int64(time.Since(start))) - } -} - -// InterceptedPDClient is a PD's wrapper client to record stmt detail. -type InterceptedPDClient struct { - pd.Client -} - -// interceptedTsFuture is a PD's wrapper future to record stmt detail. -type interceptedTsFuture struct { - pd.TSFuture - ctx context.Context -} - -// Wait implements pd.Client#Wait. -func (m interceptedTsFuture) Wait() (int64, int64, error) { - start := time.Now() - physical, logical, err := m.TSFuture.Wait() - recordPDWaitTime(m.ctx, start) - return physical, logical, err -} - -// GetTS implements pd.Client#GetTS. -func (m InterceptedPDClient) GetTS(ctx context.Context) (int64, int64, error) { - start := time.Now() - physical, logical, err := m.Client.GetTS(ctx) - recordPDWaitTime(ctx, start) - return physical, logical, err -} - -// GetTSAsync implements pd.Client#GetTSAsync. -func (m InterceptedPDClient) GetTSAsync(ctx context.Context) pd.TSFuture { - start := time.Now() - f := m.Client.GetTSAsync(ctx) - recordPDWaitTime(ctx, start) - return interceptedTsFuture{ - ctx: ctx, - TSFuture: f, - } -} - -// GetRegion implements pd.Client#GetRegion. -func (m InterceptedPDClient) GetRegion(ctx context.Context, key []byte) (*pd.Region, error) { - start := time.Now() - r, err := m.Client.GetRegion(ctx, key) - recordPDWaitTime(ctx, start) - return r, err -} - -// GetPrevRegion implements pd.Client#GetPrevRegion. -func (m InterceptedPDClient) GetPrevRegion(ctx context.Context, key []byte) (*pd.Region, error) { - start := time.Now() - r, err := m.Client.GetPrevRegion(ctx, key) - recordPDWaitTime(ctx, start) - return r, err -} - -// GetRegionByID implements pd.Client#GetRegionByID. -func (m InterceptedPDClient) GetRegionByID(ctx context.Context, regionID uint64) (*pd.Region, error) { - start := time.Now() - r, err := m.Client.GetRegionByID(ctx, regionID) - recordPDWaitTime(ctx, start) - return r, err -} - -// ScanRegions implements pd.Client#ScanRegions. -func (m InterceptedPDClient) ScanRegions(ctx context.Context, key, endKey []byte, limit int) ([]*pd.Region, error) { - start := time.Now() - r, err := m.Client.ScanRegions(ctx, key, endKey, limit) - recordPDWaitTime(ctx, start) - return r, err -} - -// GetStore implements pd.Client#GetStore. -func (m InterceptedPDClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) { - start := time.Now() - s, err := m.Client.GetStore(ctx, storeID) - recordPDWaitTime(ctx, start) - return s, err -} diff --git a/store/tikv/util/rate_limit.go b/store/tikv/util/rate_limit.go deleted file mode 100644 index 04aa1f3b89e47..0000000000000 --- a/store/tikv/util/rate_limit.go +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -// RateLimit wraps a fix sized channel to control concurrency. -type RateLimit struct { - capacity int - token chan struct{} -} - -// NewRateLimit creates a limit controller with capacity n. -func NewRateLimit(n int) *RateLimit { - return &RateLimit{ - capacity: n, - token: make(chan struct{}, n), - } -} - -// GetToken acquires a token. -func (r *RateLimit) GetToken(done <-chan struct{}) (exit bool) { - select { - case <-done: - return true - case r.token <- struct{}{}: - return false - } -} - -// PutToken puts a token back. -func (r *RateLimit) PutToken() { - select { - case <-r.token: - default: - panic("put a redundant token") - } -} - -// GetCapacity returns the token capacity. -func (r *RateLimit) GetCapacity() int { - return r.capacity -} diff --git a/store/tikv/util/rate_limit_test.go b/store/tikv/util/rate_limit_test.go deleted file mode 100644 index 29f4a72b75e0f..0000000000000 --- a/store/tikv/util/rate_limit_test.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import ( - "time" - - . "github.com/pingcap/check" -) - -func (s *testMiscSuite) TestRateLimit(c *C) { - done := make(chan struct{}, 1) - rl := NewRateLimit(1) - c.Assert(rl.PutToken, PanicMatches, "put a redundant token") - exit := rl.GetToken(done) - c.Assert(exit, Equals, false) - rl.PutToken() - c.Assert(rl.PutToken, PanicMatches, "put a redundant token") - - exit = rl.GetToken(done) - c.Assert(exit, Equals, false) - done <- struct{}{} - exit = rl.GetToken(done) // blocked but exit - c.Assert(exit, Equals, true) - - sig := make(chan int, 1) - go func() { - exit = rl.GetToken(done) // blocked - c.Assert(exit, Equals, false) - close(sig) - }() - time.Sleep(200 * time.Millisecond) - rl.PutToken() - <-sig -} diff --git a/store/tikv/util/testleak/add-leaktest.sh b/store/tikv/util/testleak/add-leaktest.sh deleted file mode 100755 index 18b23d6da4f58..0000000000000 --- a/store/tikv/util/testleak/add-leaktest.sh +++ /dev/null @@ -1,33 +0,0 @@ -#!/bin/sh -# Copyright 2019 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, -# See the License for the specific language governing permissions and -# limitations under the License. -# -# Usage: add-leaktest.sh pkg/*_test.go - -set -eu - -sed -i'~' -e ' - /^func (s \*test.*Suite) Test.*(c \*C) {/ { - n - /testleak.AfterTest/! i\ - defer testleak.AfterTest(c)() - } -' $@ - -for i in $@; do - if ! cmp -s $i $i~ ; then - goimports -w $i - fi -echo $i - rm -f $i~ -done diff --git a/store/tikv/util/testleak/check-leaktest.sh b/store/tikv/util/testleak/check-leaktest.sh deleted file mode 100755 index 62083b9014363..0000000000000 --- a/store/tikv/util/testleak/check-leaktest.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/bin/sh -# Copyright 2019 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, -# See the License for the specific language governing permissions and -# limitations under the License. -# -# Usage: check-leaktest.sh -# It needs to run under the github.com/pingcap/tidb directory. - -set -e - -pkgs=$(git grep 'Suite' |grep -vE "Godeps|tags" |awk -F: '{print $1}' | xargs -n1 dirname | sort |uniq) -echo $pkgs -for pkg in ${pkgs}; do - if [ -z "$(ls ${pkg}/*_test.go 2>/dev/null)" ]; then - continue - fi - awk -F'[(]' ' -/func \(s .*Suite\) Test.*C\) {/ { - test = $1"("$2 - next -} - -/defer testleak.AfterTest/ { - test = 0 - next -} - -{ - if (test && (FILENAME != "./tidb_test.go")) { - printf "%s: %s: missing defer testleak.AfterTest\n", FILENAME, test - test = 0 - code = 1 - } -} - -END { - exit code -} - -' ${pkg}/*_test.go -done diff --git a/store/tikv/util/testleak/fake.go b/store/tikv/util/testleak/fake.go deleted file mode 100644 index 5d4ad573f69d6..0000000000000 --- a/store/tikv/util/testleak/fake.go +++ /dev/null @@ -1,37 +0,0 @@ -// Copyright 2017 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, -// See the License for the specific language governing permissions and -// limitations under the License. -// +build !leak - -package testleak - -import ( - "testing" - - "github.com/pingcap/check" -) - -// BeforeTest is a dummy implementation when build tag 'leak' is not set. -func BeforeTest() { -} - -// AfterTest is a dummy implementation when build tag 'leak' is not set. -func AfterTest(c *check.C) func() { - return func() { - } -} - -// AfterTestT is used after all the test cases is finished. -func AfterTestT(t *testing.T) func() { - return func() { - } -} diff --git a/store/tikv/util/testleak/leaktest.go b/store/tikv/util/testleak/leaktest.go deleted file mode 100644 index 035eedc34bd4a..0000000000000 --- a/store/tikv/util/testleak/leaktest.go +++ /dev/null @@ -1,173 +0,0 @@ -// Copyright 2013 The Go Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -// Copyright 2016 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, -// See the License for the specific language governing permissions and -// limitations under the License. -// +build leak - -package testleak - -import ( - "runtime" - "sort" - "strings" - "testing" - "time" - - "github.com/pingcap/check" -) - -func interestingGoroutines() (gs []string) { - buf := make([]byte, 2<<20) - buf = buf[:runtime.Stack(buf, true)] - ignoreList := []string{ - "testing.RunTests", - "check.(*resultTracker).start", - "check.(*suiteRunner).runFunc", - "check.(*suiteRunner).parallelRun", - "localstore.(*dbStore).scheduler", - "testing.(*T).Run", - "testing.Main(", - "runtime.goexit", - "created by runtime.gc", - "interestingGoroutines", - "runtime.MHeap_Scavenger", - "created by os/signal.init", - "gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun", - // these go routines are async terminated, so they may still alive after test end, thus cause - // false positive leak failures - "google.golang.org/grpc.(*addrConn).resetTransport", - "google.golang.org/grpc.(*ccBalancerWrapper).watcher", - "github.com/pingcap/goleveldb/leveldb/util.(*BufferPool).drain", - "github.com/pingcap/goleveldb/leveldb.(*DB).compactionError", - "github.com/pingcap/goleveldb/leveldb.(*DB).mpoolDrain", - "go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop", - "go.etcd.io/etcd/v3/pkg/logutil.(*MergeLogger).outputLoop", - "oracles.(*pdOracle).updateTS", - "tikv.(*KVStore).runSafePointChecker", - "tikv.(*RegionCache).asyncCheckAndResolveLoop", - "github.com/pingcap/badger", - "github.com/ngaut/unistore/tikv.(*MVCCStore).runUpdateSafePointLoop", - } - shouldIgnore := func(stack string) bool { - if stack == "" { - return true - } - for _, ident := range ignoreList { - if strings.Contains(stack, ident) { - return true - } - } - return false - } - for _, g := range strings.Split(string(buf), "\n\n") { - sl := strings.SplitN(g, "\n", 2) - if len(sl) != 2 { - continue - } - stack := strings.TrimSpace(sl[1]) - if shouldIgnore(stack) { - continue - } - gs = append(gs, stack) - } - sort.Strings(gs) - return -} - -var beforeTestGoroutines = map[string]bool{} -var testGoroutinesInited bool - -// BeforeTest gets the current goroutines. -// It's used for check.Suite.SetUpSuite() function. -// Now it's only used in the tidb_test.go. -// Note: it's not accurate, consider the following function: -// func loop() { -// for { -// select { -// case <-ticker.C: -// DoSomething() -// } -// } -// } -// If this loop step into DoSomething() during BeforeTest(), the stack for this goroutine will contain DoSomething(). -// Then if this loop jumps out of DoSomething during AfterTest(), the stack for this goroutine will not contain DoSomething(). -// Resulting in false-positive leak reports. -func BeforeTest() { - for _, g := range interestingGoroutines() { - beforeTestGoroutines[g] = true - } - testGoroutinesInited = true -} - -const defaultCheckCnt = 50 - -func checkLeakAfterTest(errorFunc func(cnt int, g string)) func() { - // After `BeforeTest`, `beforeTestGoroutines` may still be empty, in this case, - // we shouldn't init it again. - if !testGoroutinesInited && len(beforeTestGoroutines) == 0 { - for _, g := range interestingGoroutines() { - beforeTestGoroutines[g] = true - } - } - - cnt := defaultCheckCnt - return func() { - defer func() { - beforeTestGoroutines = map[string]bool{} - testGoroutinesInited = false - }() - - var leaked []string - for i := 0; i < cnt; i++ { - leaked = leaked[:0] - for _, g := range interestingGoroutines() { - if !beforeTestGoroutines[g] { - leaked = append(leaked, g) - } - } - // Bad stuff found, but goroutines might just still be - // shutting down, so give it some time. - if len(leaked) != 0 { - time.Sleep(50 * time.Millisecond) - continue - } - - return - } - for _, g := range leaked { - errorFunc(cnt, g) - } - } -} - -// AfterTest gets the current goroutines and runs the returned function to -// get the goroutines at that time to contrast whether any goroutines leaked. -// Usage: defer testleak.AfterTest(c)() -// It can call with BeforeTest() at the beginning of check.Suite.TearDownSuite() or -// call alone at the beginning of each test. -func AfterTest(c *check.C) func() { - errorFunc := func(cnt int, g string) { - c.Errorf("Test %s check-count %d appears to have leaked: %v", c.TestName(), cnt, g) - } - return checkLeakAfterTest(errorFunc) -} - -// AfterTestT is used after all the test cases is finished. -func AfterTestT(t *testing.T) func() { - errorFunc := func(cnt int, g string) { - t.Errorf("Test %s check-count %d appears to have leaked: %v", t.Name(), cnt, g) - } - return checkLeakAfterTest(errorFunc) -} diff --git a/store/tikv/util/ts_set.go b/store/tikv/util/ts_set.go deleted file mode 100644 index 798772dd84c54..0000000000000 --- a/store/tikv/util/ts_set.go +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright 2021 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package util - -import "sync" - -// TSSet is a set of timestamps. -type TSSet struct { - sync.RWMutex - m map[uint64]struct{} -} - -// NewTSSet creates a set to store timestamps. -func NewTSSet(capacity int) *TSSet { - return &TSSet{ - m: make(map[uint64]struct{}, capacity), - } -} - -// Put puts timestamps into the map. -func (s *TSSet) Put(tss ...uint64) { - s.Lock() - defer s.Unlock() - for _, ts := range tss { - s.m[ts] = struct{}{} - } -} - -// GetAll returns all timestamps in the set. -func (s *TSSet) GetAll() []uint64 { - s.RLock() - defer s.RUnlock() - if len(s.m) == 0 { - return nil - } - ret := make([]uint64, 0, len(s.m)) - for ts := range s.m { - ret = append(ret, ts) - } - return ret -} diff --git a/tools/check/errcheck_excludes.txt b/tools/check/errcheck_excludes.txt index 04da5feb27fd0..d12c2b8e861a1 100644 --- a/tools/check/errcheck_excludes.txt +++ b/tools/check/errcheck_excludes.txt @@ -1,4 +1,3 @@ fmt.Fprintf fmt.Fprint -fmt.Sscanf -github.com/pingcap/tidb/store/tikv/util.EvalFailpoint \ No newline at end of file +fmt.Sscanf \ No newline at end of file