store/tikv: remove files (#25440)
This commit is contained in:
18
.build.ps1
18
.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
|
||||
|
||||
9
Makefile
9
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
|
||||
|
||||
1781
store/tikv/2pc.go
1781
store/tikv/2pc.go
File diff suppressed because it is too large
Load Diff
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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...)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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()
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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},
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
435
store/tikv/kv.go
435
store/tikv/kv.go
@ -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
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
)
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
)
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@ -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())
|
||||
}
|
||||
}
|
||||
@ -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{}
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
@ -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")
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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()
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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"
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@ -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)
|
||||
}
|
||||
@ -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" }
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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()
|
||||
}
|
||||
@ -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))
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
@ -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() {
|
||||
}
|
||||
@ -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"))
|
||||
}
|
||||
@ -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() {
|
||||
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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))))
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
}
|
||||
@ -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()
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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()
|
||||
}
|
||||
}
|
||||
@ -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()
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
@ -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{}) {}
|
||||
@ -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
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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)
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user