未验证 提交 af1d84e5 编写于 作者: X Xiaofan 提交者: GitHub

Support sync when cp is lagged (#24989)

Signed-off-by: Nxiaofan-luan <xiaofan.luan@zilliz.com>
上级 a6310050
......@@ -8,7 +8,7 @@ class MilvusConan(ConanFile):
"boost/1.79.0",
"onetbb/2021.7.0",
"nlohmann_json/3.11.2",
"zstd/1.5.5",
"zstd/1.5.4",
"lz4/1.9.4",
"snappy/1.1.9",
"lzo/2.10",
......
......@@ -156,6 +156,7 @@ func newChannel(channelName string, collID UniqueID, schema *schemapb.Collection
syncPolicies: []segmentSyncPolicy{
syncPeriodically(),
syncMemoryTooHigh(),
syncCPLagTooBehind(),
},
metaService: metaService,
......
......@@ -435,14 +435,18 @@ func TestDataSyncService_Close(t *testing.T) {
paramtable.Get().Reset(Params.DataNodeCfg.FlushInsertBufferSize.Key)
channel := newChannel(insertChannelName, collMeta.ID, collMeta.GetSchema(), mockRootCoord, cm)
channel.syncPolicies = []segmentSyncPolicy{
syncPeriodically(),
syncMemoryTooHigh(),
}
atimeTickSender := newTimeTickManager(mockDataCoord, 0)
sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, alloc, dispClient, factory, vchan, signalCh, mockDataCoord, newCache(), cm, newCompactionExecutor(), genTestTickler(), 0, atimeTickSender)
syncService, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, alloc, dispClient, factory, vchan, signalCh, mockDataCoord, newCache(), cm, newCompactionExecutor(), genTestTickler(), 0, atimeTickSender)
assert.NoError(t, err)
sync.flushListener = make(chan *segmentFlushPack, 10)
defer close(sync.flushListener)
syncService.flushListener = make(chan *segmentFlushPack, 10)
defer close(syncService.flushListener)
sync.start()
syncService.start()
var (
dataFactory = NewDataFactory()
......@@ -525,26 +529,26 @@ func TestDataSyncService_Close(t *testing.T) {
assert.NoError(t, err)
// wait for delete, no auto flush leads to all data in buffer.
require.Eventually(t, func() bool { return sync.delBufferManager.GetEntriesNum(1) == 1 },
require.Eventually(t, func() bool { return syncService.delBufferManager.GetEntriesNum(1) == 1 },
5*time.Second, 100*time.Millisecond)
assert.Equal(t, 0, len(sync.flushListener))
assert.Equal(t, 0, len(syncService.flushListener))
// close will trigger a force sync
sync.close()
assert.Eventually(t, func() bool { return len(sync.flushListener) == 1 },
syncService.close()
assert.Eventually(t, func() bool { return len(syncService.flushListener) == 1 },
5*time.Second, 100*time.Millisecond)
flushPack, ok := <-sync.flushListener
flushPack, ok := <-syncService.flushListener
assert.True(t, ok)
assert.Equal(t, UniqueID(1), flushPack.segmentID)
assert.True(t, len(flushPack.insertLogs) == 12)
assert.True(t, len(flushPack.statsLogs) == 1)
assert.True(t, len(flushPack.deltaLogs) == 1)
<-sync.ctx.Done()
<-syncService.ctx.Done()
// Double close is safe
sync.close()
<-sync.ctx.Done()
syncService.close()
<-syncService.ctx.Done()
}
func genBytes() (rawData []byte) {
......
......@@ -46,8 +46,7 @@ func syncPeriodically() segmentSyncPolicy {
}
}
if len(segsToSync) > 0 {
log.Debug("sync segment periodically",
zap.Int64s("segmentID", segsToSync))
log.Info("sync segment periodically", zap.Int64s("segmentID", segsToSync))
}
return segsToSync
}
......@@ -76,3 +75,45 @@ func syncMemoryTooHigh() segmentSyncPolicy {
return syncSegments
}
}
// syncCPLagTooBehind force sync the segments lagging too behind the channel checkPoint
func syncCPLagTooBehind() segmentSyncPolicy {
segmentMinTs := func(segment *Segment) uint64 {
var minTs uint64 = math.MaxUint64
if segment.curInsertBuf != nil && segment.curInsertBuf.startPos != nil && segment.curInsertBuf.startPos.Timestamp < minTs {
minTs = segment.curInsertBuf.startPos.Timestamp
}
if segment.curDeleteBuf != nil && segment.curDeleteBuf.startPos != nil && segment.curDeleteBuf.startPos.Timestamp < minTs {
minTs = segment.curDeleteBuf.startPos.Timestamp
}
for _, ib := range segment.historyInsertBuf {
if ib != nil && ib.startPos != nil && ib.startPos.Timestamp < minTs {
minTs = ib.startPos.Timestamp
}
}
for _, db := range segment.historyDeleteBuf {
if db != nil && db.startPos != nil && db.startPos.Timestamp < minTs {
minTs = db.startPos.Timestamp
}
}
return minTs
}
return func(segments []*Segment, ts Timestamp, _ *atomic.Bool) []UniqueID {
segmentsToSync := make([]UniqueID, 0)
for _, segment := range segments {
segmentMinTs := segmentMinTs(segment)
segmentStartTime := tsoutil.PhysicalTime(segmentMinTs)
cpLagDuration := tsoutil.PhysicalTime(ts).Sub(segmentStartTime)
shouldSync := cpLagDuration > Params.DataNodeCfg.CpLagPeriod.GetAsDuration(time.Second) && !segment.isBufferEmpty()
if shouldSync {
segmentsToSync = append(segmentsToSync, segment.segmentID)
}
}
if len(segmentsToSync) > 0 {
log.Info("sync segment for cp lag behind too much",
zap.Int64s("segmentID", segmentsToSync))
}
return segmentsToSync
}
}
......@@ -21,6 +21,7 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/stretchr/testify/assert"
"go.uber.org/atomic"
......@@ -94,3 +95,70 @@ func TestSyncMemoryTooHigh(t *testing.T) {
})
}
}
func TestSyncCpLagBehindTooMuch(t *testing.T) {
nowTs := tsoutil.ComposeTSByTime(time.Now(), 0)
laggedTs := tsoutil.AddPhysicalDurationOnTs(nowTs, -2*Params.DataNodeCfg.CpLagPeriod.GetAsDuration(time.Second))
tests := []struct {
testName string
segments []*Segment
idsToSync []int64
}{
{"test_current_buf_lag_behind",
[]*Segment{
{
segmentID: 1,
curInsertBuf: &BufferData{
startPos: &msgpb.MsgPosition{
Timestamp: laggedTs,
},
},
},
{
segmentID: 2,
curDeleteBuf: &DelDataBuf{
startPos: &msgpb.MsgPosition{
Timestamp: laggedTs,
},
},
},
},
[]int64{1, 2},
},
{"test_history_buf_lag_behind",
[]*Segment{
{
segmentID: 1,
historyInsertBuf: []*BufferData{
{
startPos: &msgpb.MsgPosition{
Timestamp: laggedTs,
},
},
},
},
{
segmentID: 2,
historyDeleteBuf: []*DelDataBuf{
{
startPos: &msgpb.MsgPosition{
Timestamp: laggedTs,
},
},
},
},
{
segmentID: 3,
},
},
[]int64{1, 2},
},
}
for _, test := range tests {
t.Run(test.testName, func(t *testing.T) {
policy := syncCPLagTooBehind()
ids := policy(test.segments, tsoutil.ComposeTSByTime(time.Now(), 0), nil)
assert.ElementsMatch(t, test.idsToSync, ids)
})
}
}
......@@ -2138,6 +2138,7 @@ type dataNodeConfig struct {
FlushDeleteBufferBytes ParamItem `refreshable:"true"`
BinLogMaxSize ParamItem `refreshable:"true"`
SyncPeriod ParamItem `refreshable:"true"`
CpLagPeriod ParamItem `refreshable:"true"`
// watchEvent
WatchEventTicklerInterval ParamItem `refreshable:"false"`
......@@ -2258,6 +2259,15 @@ func (p *dataNodeConfig) init(base *BaseTable) {
}
p.SyncPeriod.Init(base.mgr)
p.CpLagPeriod = ParamItem{
Key: "datanode.segment.cpLagPeriod",
Version: "2.2.0",
DefaultValue: "600",
Doc: "The period to sync segments if buffer is not empty.",
Export: true,
}
p.CpLagPeriod.Init(base.mgr)
p.WatchEventTicklerInterval = ParamItem{
Key: "datanode.segment.watchEventTicklerInterval",
Version: "2.2.3",
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册