mirror of https://github.com/tikv/client-go.git
adjust parameter for pipelined memdb by failpoint (#1205)
* adjust parameter for pipelined memdb by failpoint Signed-off-by: you06 <you1474600@gmail.com> * update comment Signed-off-by: you06 <you1474600@gmail.com> --------- Signed-off-by: you06 <you1474600@gmail.com> Co-authored-by: cfzjywxk <lsswxrxr@163.com>
This commit is contained in:
parent
83ec5f2f33
commit
c72236a601
|
|
@ -44,6 +44,7 @@ type PipelinedMemDB struct {
|
||||||
len, size int // len and size records the total flushed and onflushing memdb.
|
len, size int // len and size records the total flushed and onflushing memdb.
|
||||||
generation uint64
|
generation uint64
|
||||||
entryLimit, bufferLimit uint64
|
entryLimit, bufferLimit uint64
|
||||||
|
flushOption flushOption
|
||||||
}
|
}
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
|
@ -57,6 +58,30 @@ const (
|
||||||
ForceFlushSizeThreshold = 128 * 1024 * 1024 // 128MB
|
ForceFlushSizeThreshold = 128 * 1024 * 1024 // 128MB
|
||||||
)
|
)
|
||||||
|
|
||||||
|
type flushOption struct {
|
||||||
|
MinFlushKeys int
|
||||||
|
MinFlushSize int
|
||||||
|
ForceFlushSizeThreshold int
|
||||||
|
}
|
||||||
|
|
||||||
|
func newFlushOption() flushOption {
|
||||||
|
opt := flushOption{
|
||||||
|
MinFlushKeys: MinFlushKeys,
|
||||||
|
MinFlushSize: MinFlushSize,
|
||||||
|
ForceFlushSizeThreshold: ForceFlushSizeThreshold,
|
||||||
|
}
|
||||||
|
if val, err := util.EvalFailpoint("pipelinedMemDBMinFlushKeys"); err == nil && val != nil {
|
||||||
|
opt.MinFlushKeys = val.(int)
|
||||||
|
}
|
||||||
|
if val, err := util.EvalFailpoint("pipelinedMemDBMinFlushSize"); err == nil && val != nil {
|
||||||
|
opt.MinFlushSize = val.(int)
|
||||||
|
}
|
||||||
|
if val, err := util.EvalFailpoint("pipelinedMemDBForceFlushSizeThreshold"); err == nil && val != nil {
|
||||||
|
opt.ForceFlushSizeThreshold = val.(int)
|
||||||
|
}
|
||||||
|
return opt
|
||||||
|
}
|
||||||
|
|
||||||
type pipelinedMemDBSkipRemoteBuffer struct{}
|
type pipelinedMemDBSkipRemoteBuffer struct{}
|
||||||
|
|
||||||
// TODO: skip remote buffer by context is too obscure, add a new method to read local buffer.
|
// TODO: skip remote buffer by context is too obscure, add a new method to read local buffer.
|
||||||
|
|
@ -73,6 +98,7 @@ type BufferBatchGetter func(ctx context.Context, keys [][]byte) (map[string][]by
|
||||||
func NewPipelinedMemDB(bufferBatchGetter BufferBatchGetter, flushFunc FlushFunc) *PipelinedMemDB {
|
func NewPipelinedMemDB(bufferBatchGetter BufferBatchGetter, flushFunc FlushFunc) *PipelinedMemDB {
|
||||||
memdb := newMemDB()
|
memdb := newMemDB()
|
||||||
memdb.setSkipMutex(true)
|
memdb.setSkipMutex(true)
|
||||||
|
flushOptoin := newFlushOption()
|
||||||
return &PipelinedMemDB{
|
return &PipelinedMemDB{
|
||||||
memDB: memdb,
|
memDB: memdb,
|
||||||
errCh: make(chan error, 1),
|
errCh: make(chan error, 1),
|
||||||
|
|
@ -82,6 +108,7 @@ func NewPipelinedMemDB(bufferBatchGetter BufferBatchGetter, flushFunc FlushFunc)
|
||||||
// keep entryLimit and bufferLimit same with the memdb's default values.
|
// keep entryLimit and bufferLimit same with the memdb's default values.
|
||||||
entryLimit: memdb.entrySizeLimit,
|
entryLimit: memdb.entrySizeLimit,
|
||||||
bufferLimit: memdb.bufferSizeLimit,
|
bufferLimit: memdb.bufferSizeLimit,
|
||||||
|
flushOption: flushOptoin,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -221,10 +248,10 @@ func (p *PipelinedMemDB) needFlush() bool {
|
||||||
// MinFlushSize <= size < ForceFlushSizeThreshold && keys < MinFlushKeys, do not flush.
|
// MinFlushSize <= size < ForceFlushSizeThreshold && keys < MinFlushKeys, do not flush.
|
||||||
// MinFlushSize <= size < ForceFlushSizeThreshold && keys >= MinFlushKeys, flush.
|
// MinFlushSize <= size < ForceFlushSizeThreshold && keys >= MinFlushKeys, flush.
|
||||||
// size >= ForceFlushSizeThreshold, flush.
|
// size >= ForceFlushSizeThreshold, flush.
|
||||||
if size < MinFlushSize || (p.memDB.Len() < MinFlushKeys && size < ForceFlushSizeThreshold) {
|
if size < p.flushOption.MinFlushSize || (p.memDB.Len() < p.flushOption.MinFlushKeys && size < p.flushOption.ForceFlushSizeThreshold) {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if p.onFlushing.Load() && size < ForceFlushSizeThreshold {
|
if p.onFlushing.Load() && size < p.flushOption.ForceFlushSizeThreshold {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
return true
|
return true
|
||||||
|
|
|
||||||
|
|
@ -20,8 +20,10 @@ import (
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/pingcap/failpoint"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
tikverr "github.com/tikv/client-go/v2/error"
|
tikverr "github.com/tikv/client-go/v2/error"
|
||||||
|
"github.com/tikv/client-go/v2/util"
|
||||||
)
|
)
|
||||||
|
|
||||||
func emptyBufferBatchGetter(ctx context.Context, keys [][]byte) (map[string][]byte, error) {
|
func emptyBufferBatchGetter(ctx context.Context, keys [][]byte) (map[string][]byte, error) {
|
||||||
|
|
@ -297,3 +299,44 @@ func TestErrorIterator(t *testing.T) {
|
||||||
iteratorToErr(memdb.SnapshotIter(nil, nil))
|
iteratorToErr(memdb.SnapshotIter(nil, nil))
|
||||||
iteratorToErr(memdb.SnapshotIterReverse(nil, nil))
|
iteratorToErr(memdb.SnapshotIterReverse(nil, nil))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestPipelinedAdjustFlushCondition(t *testing.T) {
|
||||||
|
util.EnableFailpoints()
|
||||||
|
memdb := NewPipelinedMemDB(emptyBufferBatchGetter, func(_ uint64, db *MemDB) error { return nil })
|
||||||
|
memdb.Set([]byte("key"), []byte("value"))
|
||||||
|
flushed, err := memdb.Flush(false)
|
||||||
|
require.Nil(t, err)
|
||||||
|
require.False(t, flushed)
|
||||||
|
|
||||||
|
// can flush even only 1 key
|
||||||
|
require.Nil(t, failpoint.Enable("tikvclient/pipelinedMemDBMinFlushKeys", `return(1)`))
|
||||||
|
require.Nil(t, failpoint.Enable("tikvclient/pipelinedMemDBMinFlushSize", `return(1)`))
|
||||||
|
memdb = NewPipelinedMemDB(emptyBufferBatchGetter, func(_ uint64, db *MemDB) error { return nil })
|
||||||
|
memdb.Set([]byte("key"), []byte("value"))
|
||||||
|
flushed, err = memdb.Flush(false)
|
||||||
|
require.Nil(t, err)
|
||||||
|
require.True(t, flushed)
|
||||||
|
|
||||||
|
// need 2 keys to flush
|
||||||
|
require.Nil(t, failpoint.Enable("tikvclient/pipelinedMemDBMinFlushKeys", `return(2)`))
|
||||||
|
require.Nil(t, failpoint.Enable("tikvclient/pipelinedMemDBMinFlushSize", `return(1)`))
|
||||||
|
memdb = NewPipelinedMemDB(emptyBufferBatchGetter, func(_ uint64, db *MemDB) error { return nil })
|
||||||
|
memdb.Set([]byte("key"), []byte("value"))
|
||||||
|
flushed, err = memdb.Flush(false)
|
||||||
|
require.Nil(t, err)
|
||||||
|
require.False(t, flushed)
|
||||||
|
|
||||||
|
// need 2 keys to flush, but force threshold reached
|
||||||
|
require.Nil(t, failpoint.Enable("tikvclient/pipelinedMemDBMinFlushKeys", `return(2)`))
|
||||||
|
require.Nil(t, failpoint.Enable("tikvclient/pipelinedMemDBMinFlushSize", `return(1)`))
|
||||||
|
require.Nil(t, failpoint.Enable("tikvclient/pipelinedMemDBForceFlushSizeThreshold", `return(2)`))
|
||||||
|
memdb = NewPipelinedMemDB(emptyBufferBatchGetter, func(_ uint64, db *MemDB) error { return nil })
|
||||||
|
memdb.Set([]byte("key"), []byte("value"))
|
||||||
|
flushed, err = memdb.Flush(false)
|
||||||
|
require.Nil(t, err)
|
||||||
|
require.True(t, flushed)
|
||||||
|
|
||||||
|
require.Nil(t, failpoint.Disable("tikvclient/pipelinedMemDBMinFlushKeys"))
|
||||||
|
require.Nil(t, failpoint.Disable("tikvclient/pipelinedMemDBMinFlushSize"))
|
||||||
|
require.Nil(t, failpoint.Disable("tikvclient/pipelinedMemDBForceFlushSizeThreshold"))
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -204,8 +204,13 @@ func (s *KVSnapshot) BatchGet(ctx context.Context, keys [][]byte) (map[string][]
|
||||||
return s.BatchGetWithTier(ctx, keys, BatchGetSnapshotTier)
|
return s.BatchGetWithTier(ctx, keys, BatchGetSnapshotTier)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// BatchGet tiers indicate the read tier of the batch get request.
|
||||||
|
// BatchGet read keys in regions. The keys location and region error retry mechanism are shared.
|
||||||
const (
|
const (
|
||||||
|
// BatchGetSnapshotTier indicates the batch get reads from a snapshot.
|
||||||
BatchGetSnapshotTier = 1 << iota
|
BatchGetSnapshotTier = 1 << iota
|
||||||
|
// BatchGetBufferTier indicates the batch get reads from the pipelined flushed buffer, only read locks in the current txn.
|
||||||
|
// this only works when the txn is created with a pipelined memdb, unless an error will be returned.
|
||||||
BatchGetBufferTier
|
BatchGetBufferTier
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue