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.
|
||||
generation uint64
|
||||
entryLimit, bufferLimit uint64
|
||||
flushOption flushOption
|
||||
}
|
||||
|
||||
const (
|
||||
|
|
@ -57,6 +58,30 @@ const (
|
|||
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{}
|
||||
|
||||
// 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 {
|
||||
memdb := newMemDB()
|
||||
memdb.setSkipMutex(true)
|
||||
flushOptoin := newFlushOption()
|
||||
return &PipelinedMemDB{
|
||||
memDB: memdb,
|
||||
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.
|
||||
entryLimit: memdb.entrySizeLimit,
|
||||
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, 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
|
||||
}
|
||||
if p.onFlushing.Load() && size < ForceFlushSizeThreshold {
|
||||
if p.onFlushing.Load() && size < p.flushOption.ForceFlushSizeThreshold {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
|
|
|
|||
|
|
@ -20,8 +20,10 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/failpoint"
|
||||
"github.com/stretchr/testify/require"
|
||||
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) {
|
||||
|
|
@ -297,3 +299,44 @@ func TestErrorIterator(t *testing.T) {
|
|||
iteratorToErr(memdb.SnapshotIter(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)
|
||||
}
|
||||
|
||||
// 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 (
|
||||
// BatchGetSnapshotTier indicates the batch get reads from a snapshot.
|
||||
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
|
||||
)
|
||||
|
||||
|
|
|
|||
Loading…
Reference in New Issue