mirror of https://github.com/tikv/client-go.git
Merge branch 'master' into bot/update-owners-1723456948376
This commit is contained in:
commit
eb941f71c3
|
|
@ -48,6 +48,18 @@ const (
|
||||||
DefGrpcInitialWindowSize = 1 << 27 // 128MiB
|
DefGrpcInitialWindowSize = 1 << 27 // 128MiB
|
||||||
DefGrpcInitialConnWindowSize = 1 << 27 // 128MiB
|
DefGrpcInitialConnWindowSize = 1 << 27 // 128MiB
|
||||||
DefMaxConcurrencyRequestLimit = math.MaxInt64
|
DefMaxConcurrencyRequestLimit = math.MaxInt64
|
||||||
|
DefBatchPolicy = BatchPolicyStandard
|
||||||
|
)
|
||||||
|
|
||||||
|
const (
|
||||||
|
// BatchPolicyBasic is the basic batch policy whose behavior is consistent with versions before v8.3.0.
|
||||||
|
BatchPolicyBasic = "basic"
|
||||||
|
// BatchPolicyStandard dynamically batches requests based the arrival time intervals of recent requests.
|
||||||
|
BatchPolicyStandard = "standard"
|
||||||
|
// BatchPolicyPositive always performs additional batching.
|
||||||
|
BatchPolicyPositive = "positive"
|
||||||
|
// BatchPolicyCustom allows users to customize the internal batch options.
|
||||||
|
BatchPolicyCustom = "custom"
|
||||||
)
|
)
|
||||||
|
|
||||||
// TiKVClient is the config for tikv client.
|
// TiKVClient is the config for tikv client.
|
||||||
|
|
@ -72,6 +84,9 @@ type TiKVClient struct {
|
||||||
// CommitTimeout is the max time which command 'commit' will wait.
|
// CommitTimeout is the max time which command 'commit' will wait.
|
||||||
CommitTimeout string `toml:"commit-timeout" json:"commit-timeout"`
|
CommitTimeout string `toml:"commit-timeout" json:"commit-timeout"`
|
||||||
AsyncCommit AsyncCommit `toml:"async-commit" json:"async-commit"`
|
AsyncCommit AsyncCommit `toml:"async-commit" json:"async-commit"`
|
||||||
|
|
||||||
|
// BatchPolicy is the policy for batching requests.
|
||||||
|
BatchPolicy string `toml:"batch-policy" json:"batch-policy"`
|
||||||
// MaxBatchSize is the max batch size when calling batch commands API.
|
// MaxBatchSize is the max batch size when calling batch commands API.
|
||||||
MaxBatchSize uint `toml:"max-batch-size" json:"max-batch-size"`
|
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.
|
// If TiKV load is greater than this, TiDB will wait for a while to avoid little batch.
|
||||||
|
|
@ -153,6 +168,7 @@ func DefaultTiKVClient() TiKVClient {
|
||||||
AllowedClockDrift: 500 * time.Millisecond,
|
AllowedClockDrift: 500 * time.Millisecond,
|
||||||
},
|
},
|
||||||
|
|
||||||
|
BatchPolicy: DefBatchPolicy,
|
||||||
MaxBatchSize: 128,
|
MaxBatchSize: 128,
|
||||||
OverloadThreshold: 200,
|
OverloadThreshold: 200,
|
||||||
MaxBatchWaitTime: 0,
|
MaxBatchWaitTime: 0,
|
||||||
|
|
|
||||||
|
|
@ -57,7 +57,7 @@ func TestBackoffErrorType(t *testing.T) {
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
// 6ms sleep at most in total
|
// 6ms sleep at most in total
|
||||||
for i := 0; i < 2; i++ {
|
for i := 0; i < 2; i++ {
|
||||||
err = b.Backoff(BoMaxDataNotReady, errors.New("data not ready"))
|
err = b.Backoff(BoMaxRegionNotInitialized, errors.New("region not initialized"))
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
}
|
}
|
||||||
// 100ms sleep at most in total
|
// 100ms sleep at most in total
|
||||||
|
|
@ -88,7 +88,7 @@ func TestBackoffDeepCopy(t *testing.T) {
|
||||||
b := NewBackofferWithVars(context.TODO(), 4, nil)
|
b := NewBackofferWithVars(context.TODO(), 4, nil)
|
||||||
// 700 ms sleep in total and the backoffer will return an error next time.
|
// 700 ms sleep in total and the backoffer will return an error next time.
|
||||||
for i := 0; i < 3; i++ {
|
for i := 0; i < 3; i++ {
|
||||||
err = b.Backoff(BoMaxDataNotReady, errors.New("data not ready"))
|
err = b.Backoff(BoMaxRegionNotInitialized, errors.New("region not initialized"))
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
}
|
}
|
||||||
bForked, cancel := b.Fork()
|
bForked, cancel := b.Fork()
|
||||||
|
|
@ -96,7 +96,7 @@ func TestBackoffDeepCopy(t *testing.T) {
|
||||||
bCloned := b.Clone()
|
bCloned := b.Clone()
|
||||||
for _, b := range []*Backoffer{bForked, bCloned} {
|
for _, b := range []*Backoffer{bForked, bCloned} {
|
||||||
err = b.Backoff(BoTiKVRPC, errors.New("tikv rpc"))
|
err = b.Backoff(BoTiKVRPC, errors.New("tikv rpc"))
|
||||||
assert.ErrorIs(t, err, BoMaxDataNotReady.err)
|
assert.ErrorIs(t, err, BoMaxRegionNotInitialized.err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -133,7 +133,6 @@ var (
|
||||||
BoTxnNotFound = NewConfig("txnNotFound", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrResolveLockTimeout)
|
BoTxnNotFound = NewConfig("txnNotFound", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrResolveLockTimeout)
|
||||||
BoStaleCmd = NewConfig("staleCommand", &metrics.BackoffHistogramStaleCmd, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrTiKVStaleCommand)
|
BoStaleCmd = NewConfig("staleCommand", &metrics.BackoffHistogramStaleCmd, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrTiKVStaleCommand)
|
||||||
BoMaxTsNotSynced = NewConfig("maxTsNotSynced", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrTiKVMaxTimestampNotSynced)
|
BoMaxTsNotSynced = NewConfig("maxTsNotSynced", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrTiKVMaxTimestampNotSynced)
|
||||||
BoMaxDataNotReady = NewConfig("dataNotReady", &metrics.BackoffHistogramDataNotReady, NewBackoffFnCfg(2, 2000, NoJitter), tikverr.ErrRegionDataNotReady)
|
|
||||||
BoMaxRegionNotInitialized = NewConfig("regionNotInitialized", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrRegionNotInitialized)
|
BoMaxRegionNotInitialized = NewConfig("regionNotInitialized", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrRegionNotInitialized)
|
||||||
BoIsWitness = NewConfig("isWitness", &metrics.BackoffHistogramIsWitness, NewBackoffFnCfg(1000, 10000, EqualJitter), tikverr.ErrIsWitness)
|
BoIsWitness = NewConfig("isWitness", &metrics.BackoffHistogramIsWitness, NewBackoffFnCfg(1000, 10000, EqualJitter), tikverr.ErrIsWitness)
|
||||||
// TxnLockFast's `base` load from vars.BackoffLockFast when create BackoffFn.
|
// TxnLockFast's `base` load from vars.BackoffLockFast when create BackoffFn.
|
||||||
|
|
|
||||||
|
|
@ -300,8 +300,7 @@ func (a *connArray) Init(addr string, security config.Security, idleNotify *uint
|
||||||
allowBatch := (cfg.TiKVClient.MaxBatchSize > 0) && enableBatch
|
allowBatch := (cfg.TiKVClient.MaxBatchSize > 0) && enableBatch
|
||||||
if allowBatch {
|
if allowBatch {
|
||||||
a.batchConn = newBatchConn(uint(len(a.v)), cfg.TiKVClient.MaxBatchSize, idleNotify)
|
a.batchConn = newBatchConn(uint(len(a.v)), cfg.TiKVClient.MaxBatchSize, idleNotify)
|
||||||
a.pendingRequests = metrics.TiKVBatchPendingRequests.WithLabelValues(a.target)
|
a.batchConn.initMetrics(a.target)
|
||||||
a.batchSize = metrics.TiKVBatchRequests.WithLabelValues(a.target)
|
|
||||||
}
|
}
|
||||||
keepAlive := cfg.TiKVClient.GrpcKeepAliveTime
|
keepAlive := cfg.TiKVClient.GrpcKeepAliveTime
|
||||||
keepAliveTimeout := cfg.TiKVClient.GrpcKeepAliveTimeout
|
keepAliveTimeout := cfg.TiKVClient.GrpcKeepAliveTimeout
|
||||||
|
|
@ -365,6 +364,7 @@ func (a *connArray) Init(addr string, security config.Security, idleNotify *uint
|
||||||
dialTimeout: a.dialTimeout,
|
dialTimeout: a.dialTimeout,
|
||||||
tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false},
|
tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false},
|
||||||
eventListener: eventListener,
|
eventListener: eventListener,
|
||||||
|
metrics: &a.metrics,
|
||||||
}
|
}
|
||||||
batchClient.maxConcurrencyRequestLimit.Store(cfg.TiKVClient.MaxConcurrencyRequestLimit)
|
batchClient.maxConcurrencyRequestLimit.Store(cfg.TiKVClient.MaxConcurrencyRequestLimit)
|
||||||
a.batchCommandsClients = append(a.batchCommandsClients, batchClient)
|
a.batchCommandsClients = append(a.batchCommandsClients, batchClient)
|
||||||
|
|
|
||||||
|
|
@ -37,9 +37,12 @@ package client
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"encoding/json"
|
||||||
"fmt"
|
"fmt"
|
||||||
"math"
|
"math"
|
||||||
|
"runtime"
|
||||||
"runtime/trace"
|
"runtime/trace"
|
||||||
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"time"
|
"time"
|
||||||
|
|
@ -72,6 +75,11 @@ type batchCommandsEntry struct {
|
||||||
canceled int32
|
canceled int32
|
||||||
err error
|
err error
|
||||||
pri uint64
|
pri uint64
|
||||||
|
|
||||||
|
// start indicates when the batch commands entry is generated and sent to the batch conn channel.
|
||||||
|
start time.Time
|
||||||
|
sendLat int64
|
||||||
|
recvLat int64
|
||||||
}
|
}
|
||||||
|
|
||||||
func (b *batchCommandsEntry) isCanceled() bool {
|
func (b *batchCommandsEntry) isCanceled() bool {
|
||||||
|
|
@ -98,6 +106,8 @@ type batchCommandsBuilder struct {
|
||||||
requestIDs []uint64
|
requestIDs []uint64
|
||||||
// In most cases, there isn't any forwardingReq.
|
// In most cases, there isn't any forwardingReq.
|
||||||
forwardingReqs map[string]*tikvpb.BatchCommandsRequest
|
forwardingReqs map[string]*tikvpb.BatchCommandsRequest
|
||||||
|
|
||||||
|
latestReqStartTime time.Time
|
||||||
}
|
}
|
||||||
|
|
||||||
func (b *batchCommandsBuilder) len() int {
|
func (b *batchCommandsBuilder) len() int {
|
||||||
|
|
@ -106,6 +116,9 @@ func (b *batchCommandsBuilder) len() int {
|
||||||
|
|
||||||
func (b *batchCommandsBuilder) push(entry *batchCommandsEntry) {
|
func (b *batchCommandsBuilder) push(entry *batchCommandsEntry) {
|
||||||
b.entries.Push(entry)
|
b.entries.Push(entry)
|
||||||
|
if entry.start.After(b.latestReqStartTime) {
|
||||||
|
b.latestReqStartTime = entry.start
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
const highTaskPriority = 10
|
const highTaskPriority = 10
|
||||||
|
|
@ -208,6 +221,23 @@ func newBatchCommandsBuilder(maxBatchSize uint) *batchCommandsBuilder {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type batchConnMetrics struct {
|
||||||
|
pendingRequests prometheus.Observer
|
||||||
|
batchSize prometheus.Observer
|
||||||
|
|
||||||
|
sendLoopWaitHeadDur prometheus.Observer
|
||||||
|
sendLoopWaitMoreDur prometheus.Observer
|
||||||
|
sendLoopSendDur prometheus.Observer
|
||||||
|
|
||||||
|
recvLoopRecvDur prometheus.Observer
|
||||||
|
recvLoopProcessDur prometheus.Observer
|
||||||
|
|
||||||
|
headArrivalInterval prometheus.Observer
|
||||||
|
batchMoreRequests prometheus.Observer
|
||||||
|
|
||||||
|
bestBatchSize prometheus.Observer
|
||||||
|
}
|
||||||
|
|
||||||
type batchConn struct {
|
type batchConn struct {
|
||||||
// An atomic flag indicates whether the batch is idle or not.
|
// An atomic flag indicates whether the batch is idle or not.
|
||||||
// 0 for busy, others for idle.
|
// 0 for busy, others for idle.
|
||||||
|
|
@ -225,10 +255,11 @@ type batchConn struct {
|
||||||
idleNotify *uint32
|
idleNotify *uint32
|
||||||
idleDetect *time.Timer
|
idleDetect *time.Timer
|
||||||
|
|
||||||
pendingRequests prometheus.Observer
|
fetchMoreTimer *time.Timer
|
||||||
batchSize prometheus.Observer
|
|
||||||
|
|
||||||
index uint32
|
index uint32
|
||||||
|
|
||||||
|
metrics batchConnMetrics
|
||||||
}
|
}
|
||||||
|
|
||||||
func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn {
|
func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn {
|
||||||
|
|
@ -243,15 +274,27 @@ func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (a *batchConn) initMetrics(target string) {
|
||||||
|
a.metrics.pendingRequests = metrics.TiKVBatchPendingRequests.WithLabelValues(target)
|
||||||
|
a.metrics.batchSize = metrics.TiKVBatchRequests.WithLabelValues(target)
|
||||||
|
a.metrics.sendLoopWaitHeadDur = metrics.TiKVBatchSendLoopDuration.WithLabelValues(target, "wait-head")
|
||||||
|
a.metrics.sendLoopWaitMoreDur = metrics.TiKVBatchSendLoopDuration.WithLabelValues(target, "wait-more")
|
||||||
|
a.metrics.sendLoopSendDur = metrics.TiKVBatchSendLoopDuration.WithLabelValues(target, "send")
|
||||||
|
a.metrics.recvLoopRecvDur = metrics.TiKVBatchRecvLoopDuration.WithLabelValues(target, "recv")
|
||||||
|
a.metrics.recvLoopProcessDur = metrics.TiKVBatchRecvLoopDuration.WithLabelValues(target, "process")
|
||||||
|
a.metrics.headArrivalInterval = metrics.TiKVBatchHeadArrivalInterval.WithLabelValues(target)
|
||||||
|
a.metrics.batchMoreRequests = metrics.TiKVBatchMoreRequests.WithLabelValues(target)
|
||||||
|
a.metrics.bestBatchSize = metrics.TiKVBatchBestSize.WithLabelValues(target)
|
||||||
|
}
|
||||||
|
|
||||||
func (a *batchConn) isIdle() bool {
|
func (a *batchConn) isIdle() bool {
|
||||||
return atomic.LoadUint32(&a.idle) != 0
|
return atomic.LoadUint32(&a.idle) != 0
|
||||||
}
|
}
|
||||||
|
|
||||||
// fetchAllPendingRequests fetches all pending requests from the channel.
|
// fetchAllPendingRequests fetches all pending requests from the channel.
|
||||||
func (a *batchConn) fetchAllPendingRequests(
|
func (a *batchConn) fetchAllPendingRequests(maxBatchSize int) (headRecvTime time.Time, headArrivalInterval time.Duration) {
|
||||||
maxBatchSize int,
|
|
||||||
) time.Time {
|
|
||||||
// Block on the first element.
|
// Block on the first element.
|
||||||
|
latestReqStartTime := a.reqBuilder.latestReqStartTime
|
||||||
var headEntry *batchCommandsEntry
|
var headEntry *batchCommandsEntry
|
||||||
select {
|
select {
|
||||||
case headEntry = <-a.batchCommandsCh:
|
case headEntry = <-a.batchCommandsCh:
|
||||||
|
|
@ -264,14 +307,17 @@ func (a *batchConn) fetchAllPendingRequests(
|
||||||
atomic.AddUint32(&a.idle, 1)
|
atomic.AddUint32(&a.idle, 1)
|
||||||
atomic.CompareAndSwapUint32(a.idleNotify, 0, 1)
|
atomic.CompareAndSwapUint32(a.idleNotify, 0, 1)
|
||||||
// This batchConn to be recycled
|
// This batchConn to be recycled
|
||||||
return time.Now()
|
return time.Now(), 0
|
||||||
case <-a.closed:
|
case <-a.closed:
|
||||||
return time.Now()
|
return time.Now(), 0
|
||||||
}
|
}
|
||||||
if headEntry == nil {
|
if headEntry == nil {
|
||||||
return time.Now()
|
return time.Now(), 0
|
||||||
|
}
|
||||||
|
headRecvTime = time.Now()
|
||||||
|
if headEntry.start.After(latestReqStartTime) && !latestReqStartTime.IsZero() {
|
||||||
|
headArrivalInterval = headEntry.start.Sub(latestReqStartTime)
|
||||||
}
|
}
|
||||||
ts := time.Now()
|
|
||||||
a.reqBuilder.push(headEntry)
|
a.reqBuilder.push(headEntry)
|
||||||
|
|
||||||
// This loop is for trying best to collect more requests.
|
// This loop is for trying best to collect more requests.
|
||||||
|
|
@ -279,14 +325,14 @@ func (a *batchConn) fetchAllPendingRequests(
|
||||||
select {
|
select {
|
||||||
case entry := <-a.batchCommandsCh:
|
case entry := <-a.batchCommandsCh:
|
||||||
if entry == nil {
|
if entry == nil {
|
||||||
return ts
|
return
|
||||||
}
|
}
|
||||||
a.reqBuilder.push(entry)
|
a.reqBuilder.push(entry)
|
||||||
default:
|
default:
|
||||||
return ts
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return ts
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
// fetchMorePendingRequests fetches more pending requests from the channel.
|
// fetchMorePendingRequests fetches more pending requests from the channel.
|
||||||
|
|
@ -296,23 +342,33 @@ func (a *batchConn) fetchMorePendingRequests(
|
||||||
maxWaitTime time.Duration,
|
maxWaitTime time.Duration,
|
||||||
) {
|
) {
|
||||||
// Try to collect `batchWaitSize` requests, or wait `maxWaitTime`.
|
// Try to collect `batchWaitSize` requests, or wait `maxWaitTime`.
|
||||||
after := time.NewTimer(maxWaitTime)
|
if a.fetchMoreTimer == nil {
|
||||||
|
a.fetchMoreTimer = time.NewTimer(maxWaitTime)
|
||||||
|
} else {
|
||||||
|
a.fetchMoreTimer.Reset(maxWaitTime)
|
||||||
|
}
|
||||||
for a.reqBuilder.len() < batchWaitSize {
|
for a.reqBuilder.len() < batchWaitSize {
|
||||||
select {
|
select {
|
||||||
case entry := <-a.batchCommandsCh:
|
case entry := <-a.batchCommandsCh:
|
||||||
if entry == nil {
|
if entry == nil {
|
||||||
|
if !a.fetchMoreTimer.Stop() {
|
||||||
|
<-a.fetchMoreTimer.C
|
||||||
|
}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
a.reqBuilder.push(entry)
|
a.reqBuilder.push(entry)
|
||||||
case <-after.C:
|
case <-a.fetchMoreTimer.C:
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
after.Stop()
|
if !a.fetchMoreTimer.Stop() {
|
||||||
|
<-a.fetchMoreTimer.C
|
||||||
|
}
|
||||||
|
|
||||||
// Do an additional non-block try. Here we test the length with `maxBatchSize` instead
|
// Do an additional non-block try. Here we test the length with `maxBatchSize` instead
|
||||||
// of `batchWaitSize` because trying best to fetch more requests is necessary so that
|
// of `batchWaitSize` because trying best to fetch more requests is necessary so that
|
||||||
// we can adjust the `batchWaitSize` dynamically.
|
// we can adjust the `batchWaitSize` dynamically.
|
||||||
|
yielded := false
|
||||||
for a.reqBuilder.len() < maxBatchSize {
|
for a.reqBuilder.len() < maxBatchSize {
|
||||||
select {
|
select {
|
||||||
case entry := <-a.batchCommandsCh:
|
case entry := <-a.batchCommandsCh:
|
||||||
|
|
@ -321,16 +377,140 @@ func (a *batchConn) fetchMorePendingRequests(
|
||||||
}
|
}
|
||||||
a.reqBuilder.push(entry)
|
a.reqBuilder.push(entry)
|
||||||
default:
|
default:
|
||||||
|
if yielded {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
// yield once to batch more requests.
|
||||||
|
runtime.Gosched()
|
||||||
|
yielded = true
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
const idleTimeout = 3 * time.Minute
|
const idleTimeout = 3 * time.Minute
|
||||||
|
|
||||||
|
var (
|
||||||
|
// presetBatchPolicies defines a set of [turboBatchOptions] as batch policies.
|
||||||
|
presetBatchPolicies = map[string]turboBatchOptions{
|
||||||
|
config.BatchPolicyBasic: {},
|
||||||
|
config.BatchPolicyStandard: {V: turboBatchTimeBased, T: 0.0001, N: 5, W: 0.2, P: 0.8, Q: 0.8},
|
||||||
|
config.BatchPolicyPositive: {V: turboBatchAlways, T: 0.0001},
|
||||||
|
}
|
||||||
|
)
|
||||||
|
|
||||||
|
const (
|
||||||
|
turboBatchAlways = iota
|
||||||
|
turboBatchTimeBased
|
||||||
|
turboBatchProbBased
|
||||||
|
)
|
||||||
|
|
||||||
|
// turboBatchOptions defines internal options for the [turboBatchTrigger].
|
||||||
|
type turboBatchOptions struct {
|
||||||
|
// V determines the batch strategy: always(v=0), time-based(v=1), prob-based(v=2).
|
||||||
|
V int `json:"v"`
|
||||||
|
// N currently is used to determine the max arrival interval (n * t).
|
||||||
|
N int `json:"n,omitempty"`
|
||||||
|
// T is the max wait time for the batch.
|
||||||
|
T float64 `json:"t,omitempty"`
|
||||||
|
// W is used to adjust the `estArrivalInterval` or `estFetchMoreProb` dynamically.
|
||||||
|
// - time-based(v=1): estArrivalInterval = w*reqArrivalInterval + (1-w)*estArrivalInterval
|
||||||
|
// - prob-based(v=2): estFetchMoreProb = w*thisProb + (1-w)*estFetchMoreProb
|
||||||
|
W float64 `json:"w,omitempty"`
|
||||||
|
// P is used to determine whether to fetch more requests:
|
||||||
|
// - time-based(v=1): estArrivalInterval < p * t
|
||||||
|
// - prob-based(v=2): estFetchMoreProb > p
|
||||||
|
P float64 `json:"p,omitempty"`
|
||||||
|
// Q is used to adjust the `batchWaitSize` dynamically.
|
||||||
|
Q float64 `json:"q,omitempty"`
|
||||||
|
}
|
||||||
|
|
||||||
|
// turboBatchTrigger is used to trigger the `fetchMorePendingRequests` dynamically according to the request arrival
|
||||||
|
// intervals. The option `v` indicates the strategy of triggering:
|
||||||
|
//
|
||||||
|
// - turboBatchAlways: always fetch more requests.
|
||||||
|
//
|
||||||
|
// - turboBatchTimeBased: fetch more requests if estArrivalInterval < p * t
|
||||||
|
// where estArrivalInterval = w*reqArrivalInterval + (1-w)*estArrivalInterval
|
||||||
|
// and reqArrivalInterval = min(reqArrivalInterval, n * t)
|
||||||
|
//
|
||||||
|
// - turboBatchProbBased: fetch more requests if estFetchMoreProb > p
|
||||||
|
// where estFetchMoreProb = w*thisProb + (1-w)*estFetchMoreProb
|
||||||
|
// and thisProb = reqArrivalInterval < t ? 1 : 0
|
||||||
|
//
|
||||||
|
// The option `q` is used to adjust the `batchWaitSize` dynamically. If the fractional part of the `avgBatchWaitSize` is
|
||||||
|
// greater or equal to `q`, the `batchWaitSize` will be increased by 1.
|
||||||
|
type turboBatchTrigger struct {
|
||||||
|
opts turboBatchOptions
|
||||||
|
|
||||||
|
estFetchMoreProb float64
|
||||||
|
estArrivalInterval float64
|
||||||
|
maxArrivalInterval float64
|
||||||
|
}
|
||||||
|
|
||||||
|
func newTurboBatchTriggerFromPolicy(policy string) (trigger turboBatchTrigger, ok bool) {
|
||||||
|
if opts, found := presetBatchPolicies[policy]; found {
|
||||||
|
return turboBatchTrigger{opts: opts}, true
|
||||||
|
}
|
||||||
|
rawOpts, _ := strings.CutPrefix(policy, config.BatchPolicyCustom)
|
||||||
|
if err := json.Unmarshal([]byte(strings.TrimSpace(rawOpts)), &trigger.opts); err != nil {
|
||||||
|
return turboBatchTrigger{opts: presetBatchPolicies[config.DefBatchPolicy]}, false
|
||||||
|
}
|
||||||
|
ok = true
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *turboBatchTrigger) turboWaitSeconds() float64 {
|
||||||
|
return t.opts.T
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *turboBatchTrigger) turboWaitTime() time.Duration {
|
||||||
|
return time.Duration(t.opts.T * float64(time.Second))
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *turboBatchTrigger) needFetchMore(reqArrivalInterval time.Duration) bool {
|
||||||
|
if t.opts.V == turboBatchTimeBased {
|
||||||
|
thisArrivalInterval := reqArrivalInterval.Seconds()
|
||||||
|
if t.maxArrivalInterval == 0 {
|
||||||
|
t.maxArrivalInterval = t.turboWaitSeconds() * float64(t.opts.N)
|
||||||
|
}
|
||||||
|
if thisArrivalInterval > t.maxArrivalInterval {
|
||||||
|
thisArrivalInterval = t.maxArrivalInterval
|
||||||
|
}
|
||||||
|
if t.estArrivalInterval == 0 {
|
||||||
|
t.estArrivalInterval = thisArrivalInterval
|
||||||
|
} else {
|
||||||
|
t.estArrivalInterval = t.opts.W*thisArrivalInterval + (1-t.opts.W)*t.estArrivalInterval
|
||||||
|
}
|
||||||
|
return t.estArrivalInterval < t.turboWaitSeconds()*t.opts.P
|
||||||
|
} else if t.opts.V == turboBatchProbBased {
|
||||||
|
thisProb := .0
|
||||||
|
if reqArrivalInterval.Seconds() < t.turboWaitSeconds() {
|
||||||
|
thisProb = 1
|
||||||
|
}
|
||||||
|
t.estFetchMoreProb = t.opts.W*thisProb + (1-t.opts.W)*t.estFetchMoreProb
|
||||||
|
return t.estFetchMoreProb > t.opts.P
|
||||||
|
} else {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *turboBatchTrigger) preferredBatchWaitSize(avgBatchWaitSize float64, defBatchWaitSize int) int {
|
||||||
|
if t.opts.V == turboBatchAlways {
|
||||||
|
return defBatchWaitSize
|
||||||
|
}
|
||||||
|
n, m := math.Modf(avgBatchWaitSize)
|
||||||
|
batchWaitSize := int(n)
|
||||||
|
if m >= t.opts.Q {
|
||||||
|
batchWaitSize++
|
||||||
|
}
|
||||||
|
return batchWaitSize
|
||||||
|
}
|
||||||
|
|
||||||
// BatchSendLoopPanicCounter is only used for testing.
|
// BatchSendLoopPanicCounter is only used for testing.
|
||||||
var BatchSendLoopPanicCounter int64 = 0
|
var BatchSendLoopPanicCounter int64 = 0
|
||||||
|
|
||||||
|
var initBatchPolicyWarn sync.Once
|
||||||
|
|
||||||
func (a *batchConn) batchSendLoop(cfg config.TiKVClient) {
|
func (a *batchConn) batchSendLoop(cfg config.TiKVClient) {
|
||||||
defer func() {
|
defer func() {
|
||||||
if r := recover(); r != nil {
|
if r := recover(); r != nil {
|
||||||
|
|
@ -344,11 +524,20 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) {
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
bestBatchWaitSize := cfg.BatchWaitSize
|
trigger, ok := newTurboBatchTriggerFromPolicy(cfg.BatchPolicy)
|
||||||
|
if !ok {
|
||||||
|
initBatchPolicyWarn.Do(func() {
|
||||||
|
logutil.BgLogger().Warn("fallback to default batch policy due to invalid value", zap.String("value", cfg.BatchPolicy))
|
||||||
|
})
|
||||||
|
}
|
||||||
|
turboBatchWaitTime := trigger.turboWaitTime()
|
||||||
|
|
||||||
|
avgBatchWaitSize := float64(cfg.BatchWaitSize)
|
||||||
for {
|
for {
|
||||||
|
sendLoopStartTime := time.Now()
|
||||||
a.reqBuilder.reset()
|
a.reqBuilder.reset()
|
||||||
|
|
||||||
start := a.fetchAllPendingRequests(int(cfg.MaxBatchSize))
|
headRecvTime, headArrivalInterval := a.fetchAllPendingRequests(int(cfg.MaxBatchSize))
|
||||||
|
|
||||||
// curl -X PUT -d 'return(true)' http://0.0.0.0:10080/fail/tikvclient/mockBlockOnBatchClient
|
// curl -X PUT -d 'return(true)' http://0.0.0.0:10080/fail/tikvclient/mockBlockOnBatchClient
|
||||||
if val, err := util.EvalFailpoint("mockBlockOnBatchClient"); err == nil {
|
if val, err := util.EvalFailpoint("mockBlockOnBatchClient"); err == nil {
|
||||||
|
|
@ -357,27 +546,37 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if a.reqBuilder.len() < int(cfg.MaxBatchSize) && cfg.MaxBatchWaitTime > 0 {
|
if batchSize := a.reqBuilder.len(); batchSize < int(cfg.MaxBatchSize) {
|
||||||
|
if cfg.MaxBatchWaitTime > 0 && atomic.LoadUint64(&a.tikvTransportLayerLoad) > uint64(cfg.OverloadThreshold) {
|
||||||
// If the target TiKV is overload, wait a while to collect more requests.
|
// If the target TiKV is overload, wait a while to collect more requests.
|
||||||
if atomic.LoadUint64(&a.tikvTransportLayerLoad) >= uint64(cfg.OverloadThreshold) {
|
|
||||||
metrics.TiKVBatchWaitOverLoad.Inc()
|
metrics.TiKVBatchWaitOverLoad.Inc()
|
||||||
a.fetchMorePendingRequests(int(cfg.MaxBatchSize), int(bestBatchWaitSize), cfg.MaxBatchWaitTime)
|
a.fetchMorePendingRequests(int(cfg.MaxBatchSize), int(cfg.BatchWaitSize), cfg.MaxBatchWaitTime)
|
||||||
|
} else if turboBatchWaitTime > 0 && headArrivalInterval > 0 && trigger.needFetchMore(headArrivalInterval) {
|
||||||
|
batchWaitSize := trigger.preferredBatchWaitSize(avgBatchWaitSize, int(cfg.BatchWaitSize))
|
||||||
|
a.fetchMorePendingRequests(int(cfg.MaxBatchSize), batchWaitSize, turboBatchWaitTime)
|
||||||
|
a.metrics.batchMoreRequests.Observe(float64(a.reqBuilder.len() - batchSize))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
a.pendingRequests.Observe(float64(len(a.batchCommandsCh) + a.reqBuilder.len()))
|
|
||||||
length := a.reqBuilder.len()
|
length := a.reqBuilder.len()
|
||||||
|
a.metrics.pendingRequests.Observe(float64(len(a.batchCommandsCh) + length))
|
||||||
if uint(length) == 0 {
|
if uint(length) == 0 {
|
||||||
// The batch command channel is closed.
|
// The batch command channel is closed.
|
||||||
return
|
return
|
||||||
} else if uint(length) < bestBatchWaitSize && bestBatchWaitSize > 1 {
|
} else {
|
||||||
// Waits too long to collect requests, reduce the target batch size.
|
avgBatchWaitSize = 0.2*float64(length) + 0.8*avgBatchWaitSize
|
||||||
bestBatchWaitSize--
|
|
||||||
} else if uint(length) > bestBatchWaitSize+4 && bestBatchWaitSize < cfg.MaxBatchSize {
|
|
||||||
bestBatchWaitSize++
|
|
||||||
}
|
}
|
||||||
|
a.metrics.bestBatchSize.Observe(avgBatchWaitSize)
|
||||||
|
a.metrics.headArrivalInterval.Observe(headArrivalInterval.Seconds())
|
||||||
|
a.metrics.sendLoopWaitHeadDur.Observe(headRecvTime.Sub(sendLoopStartTime).Seconds())
|
||||||
|
a.metrics.sendLoopWaitMoreDur.Observe(time.Since(sendLoopStartTime).Seconds())
|
||||||
|
|
||||||
a.getClientAndSend()
|
a.getClientAndSend()
|
||||||
metrics.TiKVBatchSendLatency.Observe(float64(time.Since(start)))
|
|
||||||
|
sendLoopEndTime := time.Now()
|
||||||
|
a.metrics.sendLoopSendDur.Observe(sendLoopEndTime.Sub(sendLoopStartTime).Seconds())
|
||||||
|
if dur := sendLoopEndTime.Sub(headRecvTime); dur > 5*time.Millisecond {
|
||||||
|
metrics.TiKVBatchSendTailLatency.Observe(dur.Seconds())
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -429,10 +628,12 @@ func (a *batchConn) getClientAndSend() {
|
||||||
}
|
}
|
||||||
defer cli.unlockForSend()
|
defer cli.unlockForSend()
|
||||||
available := cli.available()
|
available := cli.available()
|
||||||
|
reqSendTime := time.Now()
|
||||||
batch := 0
|
batch := 0
|
||||||
req, forwardingReqs := a.reqBuilder.buildWithLimit(available, func(id uint64, e *batchCommandsEntry) {
|
req, forwardingReqs := a.reqBuilder.buildWithLimit(available, func(id uint64, e *batchCommandsEntry) {
|
||||||
cli.batched.Store(id, e)
|
cli.batched.Store(id, e)
|
||||||
cli.sent.Add(1)
|
cli.sent.Add(1)
|
||||||
|
atomic.StoreInt64(&e.sendLat, int64(reqSendTime.Sub(e.start)))
|
||||||
if trace.IsEnabled() {
|
if trace.IsEnabled() {
|
||||||
trace.Log(e.ctx, "rpc", "send")
|
trace.Log(e.ctx, "rpc", "send")
|
||||||
}
|
}
|
||||||
|
|
@ -446,7 +647,7 @@ func (a *batchConn) getClientAndSend() {
|
||||||
cli.send(forwardedHost, req)
|
cli.send(forwardedHost, req)
|
||||||
}
|
}
|
||||||
if batch > 0 {
|
if batch > 0 {
|
||||||
a.batchSize.Observe(float64(batch))
|
a.metrics.batchSize.Observe(float64(batch))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -490,7 +691,6 @@ type batchCommandsStream struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *batchCommandsStream) recv() (resp *tikvpb.BatchCommandsResponse, err error) {
|
func (s *batchCommandsStream) recv() (resp *tikvpb.BatchCommandsResponse, err error) {
|
||||||
now := time.Now()
|
|
||||||
defer func() {
|
defer func() {
|
||||||
if r := recover(); r != nil {
|
if r := recover(); r != nil {
|
||||||
metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
|
metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
|
||||||
|
|
@ -499,11 +699,6 @@ func (s *batchCommandsStream) recv() (resp *tikvpb.BatchCommandsResponse, err er
|
||||||
zap.Stack("stack"))
|
zap.Stack("stack"))
|
||||||
err = errors.New("batch conn recv paniced")
|
err = errors.New("batch conn recv paniced")
|
||||||
}
|
}
|
||||||
if err == nil {
|
|
||||||
metrics.BatchRecvHistogramOK.Observe(float64(time.Since(now)))
|
|
||||||
} else {
|
|
||||||
metrics.BatchRecvHistogramError.Observe(float64(time.Since(now)))
|
|
||||||
}
|
|
||||||
}()
|
}()
|
||||||
if _, err := util.EvalFailpoint("gotErrorInRecvLoop"); err == nil {
|
if _, err := util.EvalFailpoint("gotErrorInRecvLoop"); err == nil {
|
||||||
return nil, errors.New("injected error in batchRecvLoop")
|
return nil, errors.New("injected error in batchRecvLoop")
|
||||||
|
|
@ -567,6 +762,8 @@ type batchCommandsClient struct {
|
||||||
// eventListener is the listener set by external code to observe some events in the client. It's stored in a atomic
|
// eventListener is the listener set by external code to observe some events in the client. It's stored in a atomic
|
||||||
// pointer to make setting thread-safe.
|
// pointer to make setting thread-safe.
|
||||||
eventListener *atomic.Pointer[ClientEventListener]
|
eventListener *atomic.Pointer[ClientEventListener]
|
||||||
|
|
||||||
|
metrics *batchConnMetrics
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *batchCommandsClient) isStopped() bool {
|
func (c *batchCommandsClient) isStopped() bool {
|
||||||
|
|
@ -708,7 +905,7 @@ func (c *batchCommandsClient) recreateStreamingClientOnce(streamClient *batchCom
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransportLayerLoad *uint64, streamClient *batchCommandsStream) {
|
func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransportLayerLoad *uint64, connMetrics *batchConnMetrics, streamClient *batchCommandsStream) {
|
||||||
defer func() {
|
defer func() {
|
||||||
if r := recover(); r != nil {
|
if r := recover(); r != nil {
|
||||||
metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
|
metrics.TiKVPanicCounter.WithLabelValues(metrics.LabelBatchRecvLoop).Inc()
|
||||||
|
|
@ -716,13 +913,16 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport
|
||||||
zap.Any("r", r),
|
zap.Any("r", r),
|
||||||
zap.Stack("stack"))
|
zap.Stack("stack"))
|
||||||
logutil.BgLogger().Info("restart batchRecvLoop")
|
logutil.BgLogger().Info("restart batchRecvLoop")
|
||||||
go c.batchRecvLoop(cfg, tikvTransportLayerLoad, streamClient)
|
go c.batchRecvLoop(cfg, tikvTransportLayerLoad, connMetrics, streamClient)
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
epoch := atomic.LoadUint64(&c.epoch)
|
epoch := atomic.LoadUint64(&c.epoch)
|
||||||
for {
|
for {
|
||||||
|
recvLoopStartTime := time.Now()
|
||||||
resp, err := streamClient.recv()
|
resp, err := streamClient.recv()
|
||||||
|
respRecvTime := time.Now()
|
||||||
|
connMetrics.recvLoopRecvDur.Observe(respRecvTime.Sub(recvLoopStartTime).Seconds())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if c.isStopped() {
|
if c.isStopped() {
|
||||||
return
|
return
|
||||||
|
|
@ -764,6 +964,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport
|
||||||
}
|
}
|
||||||
entry := value.(*batchCommandsEntry)
|
entry := value.(*batchCommandsEntry)
|
||||||
|
|
||||||
|
atomic.StoreInt64(&entry.recvLat, int64(respRecvTime.Sub(entry.start)))
|
||||||
if trace.IsEnabled() {
|
if trace.IsEnabled() {
|
||||||
trace.Log(entry.ctx, "rpc", "received")
|
trace.Log(entry.ctx, "rpc", "received")
|
||||||
}
|
}
|
||||||
|
|
@ -781,6 +982,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport
|
||||||
// We need to consider TiKV load only if batch-wait strategy is enabled.
|
// We need to consider TiKV load only if batch-wait strategy is enabled.
|
||||||
atomic.StoreUint64(tikvTransportLayerLoad, transportLayerLoad)
|
atomic.StoreUint64(tikvTransportLayerLoad, transportLayerLoad)
|
||||||
}
|
}
|
||||||
|
connMetrics.recvLoopProcessDur.Observe(time.Since(recvLoopStartTime).Seconds())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -875,7 +1077,7 @@ func (c *batchCommandsClient) initBatchClient(forwardedHost string) error {
|
||||||
} else {
|
} else {
|
||||||
c.forwardedClients[forwardedHost] = streamClient
|
c.forwardedClients[forwardedHost] = streamClient
|
||||||
}
|
}
|
||||||
go c.batchRecvLoop(c.tikvClientCfg, c.tikvLoad, streamClient)
|
go c.batchRecvLoop(c.tikvClientCfg, c.tikvLoad, c.metrics, streamClient)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -908,11 +1110,20 @@ func sendBatchRequest(
|
||||||
canceled: 0,
|
canceled: 0,
|
||||||
err: nil,
|
err: nil,
|
||||||
pri: priority,
|
pri: priority,
|
||||||
|
start: time.Now(),
|
||||||
}
|
}
|
||||||
timer := time.NewTimer(timeout)
|
timer := time.NewTimer(timeout)
|
||||||
defer timer.Stop()
|
defer func() {
|
||||||
|
timer.Stop()
|
||||||
|
if sendLat := atomic.LoadInt64(&entry.sendLat); sendLat > 0 {
|
||||||
|
metrics.BatchRequestDurationSend.Observe(time.Duration(sendLat).Seconds())
|
||||||
|
}
|
||||||
|
if recvLat := atomic.LoadInt64(&entry.recvLat); recvLat > 0 {
|
||||||
|
metrics.BatchRequestDurationRecv.Observe(time.Duration(recvLat).Seconds())
|
||||||
|
}
|
||||||
|
metrics.BatchRequestDurationDone.Observe(time.Since(entry.start).Seconds())
|
||||||
|
}()
|
||||||
|
|
||||||
start := time.Now()
|
|
||||||
select {
|
select {
|
||||||
case batchConn.batchCommandsCh <- entry:
|
case batchConn.batchCommandsCh <- entry:
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
|
|
@ -925,8 +1136,6 @@ func sendBatchRequest(
|
||||||
case <-timer.C:
|
case <-timer.C:
|
||||||
return nil, errors.WithMessage(context.DeadlineExceeded, "wait sendLoop")
|
return nil, errors.WithMessage(context.DeadlineExceeded, "wait sendLoop")
|
||||||
}
|
}
|
||||||
waitSendDuration := time.Since(start)
|
|
||||||
metrics.TiKVBatchWaitDuration.Observe(float64(waitSendDuration))
|
|
||||||
|
|
||||||
select {
|
select {
|
||||||
case res, ok := <-entry.res:
|
case res, ok := <-entry.res:
|
||||||
|
|
@ -945,8 +1154,13 @@ func sendBatchRequest(
|
||||||
return nil, errors.New("batchConn closed")
|
return nil, errors.New("batchConn closed")
|
||||||
case <-timer.C:
|
case <-timer.C:
|
||||||
atomic.StoreInt32(&entry.canceled, 1)
|
atomic.StoreInt32(&entry.canceled, 1)
|
||||||
reason := fmt.Sprintf("wait recvLoop timeout, timeout:%s, wait_send_duration:%s, wait_recv_duration:%s",
|
reason := fmt.Sprintf("wait recvLoop timeout, timeout:%s", timeout)
|
||||||
timeout, util.FormatDuration(waitSendDuration), util.FormatDuration(time.Since(start)-waitSendDuration))
|
if sendLat := atomic.LoadInt64(&entry.sendLat); sendLat > 0 {
|
||||||
|
reason += fmt.Sprintf(", send:%s", util.FormatDuration(time.Duration(sendLat)))
|
||||||
|
if recvLat := atomic.LoadInt64(&entry.recvLat); recvLat > 0 {
|
||||||
|
reason += fmt.Sprintf(", recv:%s", util.FormatDuration(time.Duration(recvLat-sendLat)))
|
||||||
|
}
|
||||||
|
}
|
||||||
return nil, errors.WithMessage(context.DeadlineExceeded, reason)
|
return nil, errors.WithMessage(context.DeadlineExceeded, reason)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -148,7 +148,7 @@ func TestSendWhenReconnect(t *testing.T) {
|
||||||
|
|
||||||
req := tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{})
|
req := tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{})
|
||||||
_, err = rpcClient.SendRequest(context.Background(), addr, req, 5*time.Second)
|
_, err = rpcClient.SendRequest(context.Background(), addr, req, 5*time.Second)
|
||||||
require.Regexp(t, "wait recvLoop timeout, timeout:5s, wait_send_duration:.*, wait_recv_duration:.*: context deadline exceeded", err.Error())
|
require.Regexp(t, "wait recvLoop timeout, timeout:5s: context deadline exceeded", err.Error())
|
||||||
server.Stop()
|
server.Stop()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1067,3 +1067,76 @@ func TestConcurrentCloseConnPanic(t *testing.T) {
|
||||||
}()
|
}()
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestBatchPolicy(t *testing.T) {
|
||||||
|
t.Run(config.BatchPolicyBasic, func(t *testing.T) {
|
||||||
|
trigger, ok := newTurboBatchTriggerFromPolicy(config.BatchPolicyBasic)
|
||||||
|
require.True(t, ok)
|
||||||
|
require.False(t, trigger.turboWaitTime() > 0)
|
||||||
|
})
|
||||||
|
t.Run(config.BatchPolicyPositive, func(t *testing.T) {
|
||||||
|
trigger, ok := newTurboBatchTriggerFromPolicy(config.BatchPolicyPositive)
|
||||||
|
require.True(t, ok)
|
||||||
|
require.Equal(t, trigger.turboWaitTime(), 100*time.Microsecond)
|
||||||
|
require.True(t, trigger.needFetchMore(time.Hour))
|
||||||
|
require.True(t, trigger.needFetchMore(time.Millisecond))
|
||||||
|
require.Equal(t, 8, trigger.preferredBatchWaitSize(1, 8))
|
||||||
|
require.Equal(t, 8, trigger.preferredBatchWaitSize(1.2, 8))
|
||||||
|
require.Equal(t, 8, trigger.preferredBatchWaitSize(1.8, 8))
|
||||||
|
})
|
||||||
|
t.Run(config.BatchPolicyStandard, func(t *testing.T) {
|
||||||
|
trigger, ok := newTurboBatchTriggerFromPolicy(config.BatchPolicyStandard)
|
||||||
|
require.True(t, ok)
|
||||||
|
require.Equal(t, 1, trigger.preferredBatchWaitSize(1, 8))
|
||||||
|
require.Equal(t, 1, trigger.preferredBatchWaitSize(1.2, 8))
|
||||||
|
require.Equal(t, 2, trigger.preferredBatchWaitSize(1.8, 8))
|
||||||
|
require.Equal(t, trigger.turboWaitTime(), 100*time.Microsecond)
|
||||||
|
require.False(t, trigger.needFetchMore(100*time.Microsecond))
|
||||||
|
require.False(t, trigger.needFetchMore(80*time.Microsecond))
|
||||||
|
require.True(t, trigger.needFetchMore(10*time.Microsecond))
|
||||||
|
require.True(t, trigger.needFetchMore(80*time.Microsecond))
|
||||||
|
require.False(t, trigger.needFetchMore(90*time.Microsecond))
|
||||||
|
|
||||||
|
for i := 0; i < 50; i++ {
|
||||||
|
trigger.needFetchMore(time.Hour)
|
||||||
|
}
|
||||||
|
require.Less(t, trigger.estArrivalInterval, trigger.maxArrivalInterval)
|
||||||
|
for i := 0; i < 8; i++ {
|
||||||
|
require.False(t, trigger.needFetchMore(10*time.Microsecond))
|
||||||
|
}
|
||||||
|
require.True(t, trigger.needFetchMore(10*time.Microsecond))
|
||||||
|
})
|
||||||
|
t.Run(config.BatchPolicyCustom, func(t *testing.T) {
|
||||||
|
trigger, ok := newTurboBatchTriggerFromPolicy(config.BatchPolicyCustom + " {} ")
|
||||||
|
require.True(t, ok)
|
||||||
|
require.Equal(t, trigger.opts, presetBatchPolicies[config.BatchPolicyBasic])
|
||||||
|
|
||||||
|
trigger, ok = newTurboBatchTriggerFromPolicy(`{"t":0.0001}`)
|
||||||
|
require.True(t, ok)
|
||||||
|
require.Equal(t, trigger.opts, presetBatchPolicies[config.BatchPolicyPositive])
|
||||||
|
|
||||||
|
trigger, ok = newTurboBatchTriggerFromPolicy(`{"v":1,"t":0.0001,"n":5,"w":0.2,"p":0.8,"q":0.8}`)
|
||||||
|
require.True(t, ok)
|
||||||
|
require.Equal(t, trigger.opts, presetBatchPolicies[config.BatchPolicyStandard])
|
||||||
|
|
||||||
|
trigger, ok = newTurboBatchTriggerFromPolicy(`{"v":2,"t":0.001,"w":0.2,"p":0.5}`)
|
||||||
|
require.True(t, ok)
|
||||||
|
require.Equal(t, 2, trigger.preferredBatchWaitSize(1, 8))
|
||||||
|
require.Equal(t, 2, trigger.preferredBatchWaitSize(1.2, 8))
|
||||||
|
require.Equal(t, trigger.turboWaitTime(), time.Millisecond)
|
||||||
|
require.False(t, trigger.needFetchMore(time.Millisecond-time.Microsecond))
|
||||||
|
require.False(t, trigger.needFetchMore(time.Millisecond-time.Microsecond))
|
||||||
|
require.False(t, trigger.needFetchMore(time.Millisecond-time.Microsecond))
|
||||||
|
require.True(t, trigger.needFetchMore(time.Millisecond-time.Microsecond))
|
||||||
|
require.False(t, trigger.needFetchMore(time.Millisecond))
|
||||||
|
})
|
||||||
|
t.Run("invalid", func(t *testing.T) {
|
||||||
|
for _, val := range []string{
|
||||||
|
"", "invalid", "custom", "custom {x:1}",
|
||||||
|
} {
|
||||||
|
trigger, ok := newTurboBatchTriggerFromPolicy(val)
|
||||||
|
require.False(t, ok)
|
||||||
|
require.Equal(t, trigger.opts, presetBatchPolicies[config.DefBatchPolicy])
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -1715,13 +1715,7 @@ func (s *RegionRequestSender) onRegionError(
|
||||||
if s.replicaSelector != nil {
|
if s.replicaSelector != nil {
|
||||||
s.replicaSelector.onDataIsNotReady()
|
s.replicaSelector.onDataIsNotReady()
|
||||||
}
|
}
|
||||||
if !req.IsGlobalStaleRead() {
|
// do not backoff data-is-not-ready as we always retry with normal snapshot read.
|
||||||
// only backoff local stale reads as global should retry immediately against the leader as a normal read
|
|
||||||
err = bo.Backoff(retry.BoMaxDataNotReady, errors.New("data is not ready"))
|
|
||||||
if err != nil {
|
|
||||||
return false, err
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return true, nil
|
return true, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -64,15 +64,19 @@ var (
|
||||||
TiKVLocalLatchWaitTimeHistogram prometheus.Histogram
|
TiKVLocalLatchWaitTimeHistogram prometheus.Histogram
|
||||||
TiKVStatusDuration *prometheus.HistogramVec
|
TiKVStatusDuration *prometheus.HistogramVec
|
||||||
TiKVStatusCounter *prometheus.CounterVec
|
TiKVStatusCounter *prometheus.CounterVec
|
||||||
TiKVBatchWaitDuration prometheus.Histogram
|
TiKVBatchSendTailLatency prometheus.Histogram
|
||||||
TiKVBatchSendLatency prometheus.Histogram
|
TiKVBatchSendLoopDuration *prometheus.SummaryVec
|
||||||
|
TiKVBatchRecvLoopDuration *prometheus.SummaryVec
|
||||||
|
TiKVBatchHeadArrivalInterval *prometheus.SummaryVec
|
||||||
|
TiKVBatchBestSize *prometheus.SummaryVec
|
||||||
|
TiKVBatchMoreRequests *prometheus.SummaryVec
|
||||||
TiKVBatchWaitOverLoad prometheus.Counter
|
TiKVBatchWaitOverLoad prometheus.Counter
|
||||||
TiKVBatchPendingRequests *prometheus.HistogramVec
|
TiKVBatchPendingRequests *prometheus.HistogramVec
|
||||||
TiKVBatchRequests *prometheus.HistogramVec
|
TiKVBatchRequests *prometheus.HistogramVec
|
||||||
|
TiKVBatchRequestDuration *prometheus.SummaryVec
|
||||||
TiKVBatchClientUnavailable prometheus.Histogram
|
TiKVBatchClientUnavailable prometheus.Histogram
|
||||||
TiKVBatchClientWaitEstablish prometheus.Histogram
|
TiKVBatchClientWaitEstablish prometheus.Histogram
|
||||||
TiKVBatchClientRecycle prometheus.Histogram
|
TiKVBatchClientRecycle prometheus.Histogram
|
||||||
TiKVBatchRecvLatency *prometheus.HistogramVec
|
|
||||||
TiKVRangeTaskStats *prometheus.GaugeVec
|
TiKVRangeTaskStats *prometheus.GaugeVec
|
||||||
TiKVRangeTaskPushDuration *prometheus.HistogramVec
|
TiKVRangeTaskPushDuration *prometheus.HistogramVec
|
||||||
TiKVTokenWaitDuration prometheus.Histogram
|
TiKVTokenWaitDuration prometheus.Histogram
|
||||||
|
|
@ -358,35 +362,60 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) {
|
||||||
ConstLabels: constLabels,
|
ConstLabels: constLabels,
|
||||||
}, []string{LblResult})
|
}, []string{LblResult})
|
||||||
|
|
||||||
TiKVBatchWaitDuration = prometheus.NewHistogram(
|
TiKVBatchSendTailLatency = prometheus.NewHistogram(
|
||||||
prometheus.HistogramOpts{
|
prometheus.HistogramOpts{
|
||||||
Namespace: namespace,
|
Namespace: namespace,
|
||||||
Subsystem: subsystem,
|
Subsystem: subsystem,
|
||||||
Name: "batch_wait_duration",
|
Name: "batch_send_tail_latency_seconds",
|
||||||
Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s
|
Buckets: prometheus.ExponentialBuckets(0.005, 2, 10), // 5ms ~ 2.56s
|
||||||
Help: "batch wait duration",
|
Help: "batch send tail latency",
|
||||||
ConstLabels: constLabels,
|
ConstLabels: constLabels,
|
||||||
})
|
})
|
||||||
|
|
||||||
TiKVBatchSendLatency = prometheus.NewHistogram(
|
TiKVBatchSendLoopDuration = prometheus.NewSummaryVec(
|
||||||
prometheus.HistogramOpts{
|
prometheus.SummaryOpts{
|
||||||
Namespace: namespace,
|
Namespace: namespace,
|
||||||
Subsystem: subsystem,
|
Subsystem: subsystem,
|
||||||
Name: "batch_send_latency",
|
Name: "batch_send_loop_duration_seconds",
|
||||||
Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s
|
Help: "batch send loop duration breakdown by steps",
|
||||||
Help: "batch send latency",
|
|
||||||
ConstLabels: constLabels,
|
ConstLabels: constLabels,
|
||||||
})
|
}, []string{"store", "step"})
|
||||||
|
|
||||||
TiKVBatchRecvLatency = prometheus.NewHistogramVec(
|
TiKVBatchRecvLoopDuration = prometheus.NewSummaryVec(
|
||||||
prometheus.HistogramOpts{
|
prometheus.SummaryOpts{
|
||||||
Namespace: namespace,
|
Namespace: namespace,
|
||||||
Subsystem: subsystem,
|
Subsystem: subsystem,
|
||||||
Name: "batch_recv_latency",
|
Name: "batch_recv_loop_duration_seconds",
|
||||||
Buckets: prometheus.ExponentialBuckets(1000, 2, 34), // 1us ~ 8000s
|
Help: "batch recv loop duration breakdown by steps",
|
||||||
Help: "batch recv latency",
|
|
||||||
ConstLabels: constLabels,
|
ConstLabels: constLabels,
|
||||||
}, []string{LblResult})
|
}, []string{"store", "step"})
|
||||||
|
|
||||||
|
TiKVBatchHeadArrivalInterval = prometheus.NewSummaryVec(
|
||||||
|
prometheus.SummaryOpts{
|
||||||
|
Namespace: namespace,
|
||||||
|
Subsystem: subsystem,
|
||||||
|
Name: "batch_head_arrival_interval_seconds",
|
||||||
|
Help: "arrival interval of the head request in batch",
|
||||||
|
ConstLabels: constLabels,
|
||||||
|
}, []string{"store"})
|
||||||
|
|
||||||
|
TiKVBatchBestSize = prometheus.NewSummaryVec(
|
||||||
|
prometheus.SummaryOpts{
|
||||||
|
Namespace: namespace,
|
||||||
|
Subsystem: subsystem,
|
||||||
|
Name: "batch_best_size",
|
||||||
|
Help: "best batch size estimated by the batch client",
|
||||||
|
ConstLabels: constLabels,
|
||||||
|
}, []string{"store"})
|
||||||
|
|
||||||
|
TiKVBatchMoreRequests = prometheus.NewSummaryVec(
|
||||||
|
prometheus.SummaryOpts{
|
||||||
|
Namespace: namespace,
|
||||||
|
Subsystem: subsystem,
|
||||||
|
Name: "batch_more_requests_total",
|
||||||
|
Help: "number of requests batched by extra fetch",
|
||||||
|
ConstLabels: constLabels,
|
||||||
|
}, []string{"store"})
|
||||||
|
|
||||||
TiKVBatchWaitOverLoad = prometheus.NewCounter(
|
TiKVBatchWaitOverLoad = prometheus.NewCounter(
|
||||||
prometheus.CounterOpts{
|
prometheus.CounterOpts{
|
||||||
|
|
@ -417,6 +446,15 @@ func initMetrics(namespace, subsystem string, constLabels prometheus.Labels) {
|
||||||
ConstLabels: constLabels,
|
ConstLabels: constLabels,
|
||||||
}, []string{"store"})
|
}, []string{"store"})
|
||||||
|
|
||||||
|
TiKVBatchRequestDuration = prometheus.NewSummaryVec(
|
||||||
|
prometheus.SummaryOpts{
|
||||||
|
Namespace: namespace,
|
||||||
|
Subsystem: subsystem,
|
||||||
|
Name: "batch_request_duration_seconds",
|
||||||
|
Help: "batch request duration breakdown by steps",
|
||||||
|
ConstLabels: constLabels,
|
||||||
|
}, []string{"step"})
|
||||||
|
|
||||||
TiKVBatchClientUnavailable = prometheus.NewHistogram(
|
TiKVBatchClientUnavailable = prometheus.NewHistogram(
|
||||||
prometheus.HistogramOpts{
|
prometheus.HistogramOpts{
|
||||||
Namespace: namespace,
|
Namespace: namespace,
|
||||||
|
|
@ -839,12 +877,16 @@ func RegisterMetrics() {
|
||||||
prometheus.MustRegister(TiKVLocalLatchWaitTimeHistogram)
|
prometheus.MustRegister(TiKVLocalLatchWaitTimeHistogram)
|
||||||
prometheus.MustRegister(TiKVStatusDuration)
|
prometheus.MustRegister(TiKVStatusDuration)
|
||||||
prometheus.MustRegister(TiKVStatusCounter)
|
prometheus.MustRegister(TiKVStatusCounter)
|
||||||
prometheus.MustRegister(TiKVBatchWaitDuration)
|
prometheus.MustRegister(TiKVBatchSendTailLatency)
|
||||||
prometheus.MustRegister(TiKVBatchSendLatency)
|
prometheus.MustRegister(TiKVBatchSendLoopDuration)
|
||||||
prometheus.MustRegister(TiKVBatchRecvLatency)
|
prometheus.MustRegister(TiKVBatchRecvLoopDuration)
|
||||||
|
prometheus.MustRegister(TiKVBatchHeadArrivalInterval)
|
||||||
|
prometheus.MustRegister(TiKVBatchBestSize)
|
||||||
|
prometheus.MustRegister(TiKVBatchMoreRequests)
|
||||||
prometheus.MustRegister(TiKVBatchWaitOverLoad)
|
prometheus.MustRegister(TiKVBatchWaitOverLoad)
|
||||||
prometheus.MustRegister(TiKVBatchPendingRequests)
|
prometheus.MustRegister(TiKVBatchPendingRequests)
|
||||||
prometheus.MustRegister(TiKVBatchRequests)
|
prometheus.MustRegister(TiKVBatchRequests)
|
||||||
|
prometheus.MustRegister(TiKVBatchRequestDuration)
|
||||||
prometheus.MustRegister(TiKVBatchClientUnavailable)
|
prometheus.MustRegister(TiKVBatchClientUnavailable)
|
||||||
prometheus.MustRegister(TiKVBatchClientWaitEstablish)
|
prometheus.MustRegister(TiKVBatchClientWaitEstablish)
|
||||||
prometheus.MustRegister(TiKVBatchClientRecycle)
|
prometheus.MustRegister(TiKVBatchClientRecycle)
|
||||||
|
|
|
||||||
|
|
@ -173,6 +173,10 @@ var (
|
||||||
StaleReadLocalOutBytes prometheus.Counter
|
StaleReadLocalOutBytes prometheus.Counter
|
||||||
StaleReadRemoteInBytes prometheus.Counter
|
StaleReadRemoteInBytes prometheus.Counter
|
||||||
StaleReadRemoteOutBytes prometheus.Counter
|
StaleReadRemoteOutBytes prometheus.Counter
|
||||||
|
|
||||||
|
BatchRequestDurationSend prometheus.Observer
|
||||||
|
BatchRequestDurationRecv prometheus.Observer
|
||||||
|
BatchRequestDurationDone prometheus.Observer
|
||||||
)
|
)
|
||||||
|
|
||||||
func initShortcuts() {
|
func initShortcuts() {
|
||||||
|
|
@ -287,8 +291,9 @@ func initShortcuts() {
|
||||||
OnePCTxnCounterError = TiKVOnePCTxnCounter.WithLabelValues("err")
|
OnePCTxnCounterError = TiKVOnePCTxnCounter.WithLabelValues("err")
|
||||||
OnePCTxnCounterFallback = TiKVOnePCTxnCounter.WithLabelValues("fallback")
|
OnePCTxnCounterFallback = TiKVOnePCTxnCounter.WithLabelValues("fallback")
|
||||||
|
|
||||||
BatchRecvHistogramOK = TiKVBatchRecvLatency.WithLabelValues("ok")
|
BatchRequestDurationSend = TiKVBatchRequestDuration.WithLabelValues("send")
|
||||||
BatchRecvHistogramError = TiKVBatchRecvLatency.WithLabelValues("err")
|
BatchRequestDurationRecv = TiKVBatchRequestDuration.WithLabelValues("recv")
|
||||||
|
BatchRequestDurationDone = TiKVBatchRequestDuration.WithLabelValues("done")
|
||||||
|
|
||||||
PrewriteAssertionUsageCounterNone = TiKVPrewriteAssertionUsageCounter.WithLabelValues("none")
|
PrewriteAssertionUsageCounterNone = TiKVPrewriteAssertionUsageCounter.WithLabelValues("none")
|
||||||
PrewriteAssertionUsageCounterExist = TiKVPrewriteAssertionUsageCounter.WithLabelValues("exist")
|
PrewriteAssertionUsageCounterExist = TiKVPrewriteAssertionUsageCounter.WithLabelValues("exist")
|
||||||
|
|
|
||||||
|
|
@ -1216,6 +1216,11 @@ func (rs *SnapshotRuntimeStats) String() string {
|
||||||
return buf.String()
|
return buf.String()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetTimeDetail returns the timeDetail
|
||||||
|
func (rs *SnapshotRuntimeStats) GetTimeDetail() *util.TimeDetail {
|
||||||
|
return rs.timeDetail
|
||||||
|
}
|
||||||
|
|
||||||
// GetCmdRPCCount returns the count of the corresponding kind of rpc requests
|
// GetCmdRPCCount returns the count of the corresponding kind of rpc requests
|
||||||
func (rs *SnapshotRuntimeStats) GetCmdRPCCount(cmd tikvrpc.CmdType) int64 {
|
func (rs *SnapshotRuntimeStats) GetCmdRPCCount(cmd tikvrpc.CmdType) int64 {
|
||||||
if rs.rpcStats == nil || len(rs.rpcStats.RPCStats) == 0 {
|
if rs.rpcStats == nil || len(rs.rpcStats.RPCStats) == 0 {
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue