Make FlushSegments async (#8159)

Resolves: #8122

Signed-off-by: yangxuan <xuan.yang@zilliz.com>
pull/8405/head
XuanYang-cn 2021-09-23 16:03:54 +08:00 committed by GitHub
parent 99296e91b8
commit d816a81ee3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 47 additions and 88 deletions

View File

@ -275,7 +275,7 @@ func (node *DataNode) NewDataSyncService(vchan *datapb.VchannelInfo) error {
)
flushChan := make(chan *flushMsg, 100)
dataSyncService, err := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan, node.clearSignal, node.dataCoord)
dataSyncService, err := newDataSyncService(node.ctx, flushChan, replica, alloc, node.msFactory, vchan, node.clearSignal, node.dataCoord, node.segmentCache)
if err != nil {
return err
}
@ -495,20 +495,17 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
return status, nil
}
numOfFlushingSeg := len(req.SegmentIDs)
log.Debug("FlushSegments ...",
log.Debug("Receive FlushSegments req",
zap.Int("num", len(req.SegmentIDs)),
zap.Int64s("segments", req.SegmentIDs),
)
dmlFlushedCh := make(chan []*datapb.FieldBinlog, len(req.SegmentIDs))
for _, id := range req.SegmentIDs {
chanName := node.getChannelNamebySegmentID(id)
log.Debug("vchannel",
zap.String("name", chanName),
zap.Int64("SegmentID", id))
log.Debug("vchannel", zap.String("name", chanName), zap.Int64("SegmentID", id))
if len(chanName) == 0 {
log.Warn("DataNode not find segment", zap.Int64("ID", id))
status.Reason = fmt.Sprintf("DataNode not find segment %d!", id)
return status, nil
}
@ -516,7 +513,6 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
if node.segmentCache.checkIfCached(id) {
// Segment in flushing or flushed, ignore
log.Info("Segment in flushing, ignore it", zap.Int64("ID", id))
numOfFlushingSeg--
continue
}
@ -526,7 +522,6 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
flushCh, ok := node.vchan2FlushCh[chanName]
node.chanMut.RUnlock()
if !ok {
// TODO restart DataNode or reshape vchan2FlushCh and vchan2SyncService
status.Reason = "DataNode abnormal, restarting"
return status, nil
}
@ -536,29 +531,11 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
timestamp: req.Base.Timestamp,
segmentID: id,
collectionID: req.CollectionID,
dmlFlushedCh: dmlFlushedCh,
}
flushCh <- flushmsg
}
failedSegments := ""
for i := 0; i < numOfFlushingSeg; i++ {
msg := <-dmlFlushedCh
if len(msg) != 1 {
panic("flush size expect to 1")
}
if msg[0].Binlogs == nil {
failedSegments += fmt.Sprintf(" %d", msg[0].FieldID)
}
}
if len(failedSegments) != 0 {
status.Reason = fmt.Sprintf("flush failed segment list = %s", failedSegments)
return status, nil
}
node.segmentCache.Remove(req.SegmentIDs...)
log.Debug("FlushSegments Done",
log.Debug("FlushSegments tasks triggered",
zap.Int64s("segments", req.SegmentIDs))
status.ErrorCode = commonpb.ErrorCode_Success

View File

@ -39,6 +39,8 @@ type dataSyncService struct {
dataCoord types.DataCoord
clearSignal chan<- UniqueID
flushingSegCache *Cache
saveBinlog func(fu *segmentFlushUnit) error
}
@ -50,6 +52,7 @@ func newDataSyncService(ctx context.Context,
vchan *datapb.VchannelInfo,
clearSignal chan<- UniqueID,
dataCoord types.DataCoord,
flushingSegCache *Cache,
) (*dataSyncService, error) {
@ -60,16 +63,17 @@ func newDataSyncService(ctx context.Context,
ctx1, cancel := context.WithCancel(ctx)
service := &dataSyncService{
ctx: ctx1,
cancelFn: cancel,
fg: nil,
flushChan: flushChan,
replica: replica,
idAllocator: alloc,
msFactory: factory,
collectionID: vchan.GetCollectionID(),
dataCoord: dataCoord,
clearSignal: clearSignal,
ctx: ctx1,
cancelFn: cancel,
fg: nil,
flushChan: flushChan,
replica: replica,
idAllocator: alloc,
msFactory: factory,
collectionID: vchan.GetCollectionID(),
dataCoord: dataCoord,
clearSignal: clearSignal,
flushingSegCache: flushingSegCache,
}
if err := service.initNodes(vchan); err != nil {
@ -176,6 +180,7 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
dsService.flushChan,
saveBinlog,
vchanInfo.GetChannelName(),
dsService.flushingSegCache,
)
if err != nil {
return err

View File

@ -105,6 +105,7 @@ func TestDataSyncService_newDataSyncService(te *testing.T) {
getVchanInfo(test.isValidCase, test.collID, test.ufCollID, test.ufSegID, test.chanName, test.ufchanName, test.ufNor),
make(chan UniqueID),
df,
newCache(),
)
if !test.isValidCase {
@ -186,7 +187,7 @@ func TestDataSyncService_Start(t *testing.T) {
}
signalCh := make(chan UniqueID, 100)
sync, err := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataCoordFactory{})
sync, err := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataCoordFactory{}, newCache())
assert.Nil(t, err)
// sync.replica.addCollection(collMeta.ID, collMeta.Schema)

View File

@ -58,8 +58,9 @@ type insertBufferNode struct {
replica Replica
idAllocator allocatorInterface
flushMap sync.Map
flushChan <-chan *flushMsg
flushMap sync.Map
flushChan <-chan *flushMsg
flushingSegCache *Cache
minIOKV kv.BaseKV
@ -299,7 +300,6 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
flushed: true,
})
ibNode.replica.segmentFlushed(currentSegID)
fmsg.dmlFlushedCh <- []*datapb.FieldBinlog{{FieldID: currentSegID, Binlogs: []string{}}}
} else { //insertBuffer(not empty) -> binLogs -> minIO/S3
log.Debug(".. Buffer not empty, flushing ..")
finishCh := make(chan segmentFlushUnit, 1)
@ -311,7 +311,6 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
log.Debug(".. Clearing flush Buffer ..")
ibNode.flushMap.Delete(currentSegID)
close(finishCh)
fmsg.dmlFlushedCh <- []*datapb.FieldBinlog{{FieldID: currentSegID, Binlogs: nil}}
}
collID, partitionID, err := ibNode.getCollectionandPartitionIDbySegID(currentSegID)
@ -341,15 +340,14 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
log.Debug("Data service save binlog path failed", zap.Error(err))
} else {
ibNode.replica.segmentFlushed(fu.segID)
ibNode.flushingSegCache.Remove(fu.segID)
}
}
fmsg.dmlFlushedCh <- []*datapb.FieldBinlog{{FieldID: currentSegID, Binlogs: []string{}}}
}
default:
}
// TODO write timetick
if err := ibNode.writeHardTimeTick(iMsg.timeRange.timestampMax); err != nil {
log.Error("send hard time tick into pulsar channel failed", zap.Error(err))
}
@ -858,6 +856,7 @@ func newInsertBufferNode(
flushCh <-chan *flushMsg,
saveBinlog func(*segmentFlushUnit) error,
channelName string,
flushingSegCache *Cache,
) (*insertBufferNode, error) {
maxQueueLength := Params.FlowGraphMaxQueueLength
@ -917,10 +916,11 @@ func newInsertBufferNode(
timeTickStream: wTtMsgStream,
segmentStatisticsStream: segStatisticsMsgStream,
replica: replica,
flushMap: sync.Map{},
flushChan: flushCh,
idAllocator: idAllocator,
dsSaveBinlog: saveBinlog,
replica: replica,
flushMap: sync.Map{},
flushChan: flushCh,
idAllocator: idAllocator,
dsSaveBinlog: saveBinlog,
flushingSegCache: flushingSegCache,
}, nil
}

View File

@ -23,20 +23,18 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.uber.org/zap"
memkv "github.com/milvus-io/milvus/internal/kv/mem"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/flowgraph"
)
// CDFMsFactory count down fails msg factory
@ -87,13 +85,13 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
}
flushChan := make(chan *flushMsg, 100)
iBNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
iBNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
assert.NotNil(t, iBNode)
require.NoError(t, err)
ctxDone, cancel := context.WithCancel(ctx)
cancel() // cancel now to make context done
_, err = newInsertBufferNode(ctxDone, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
_, err = newInsertBufferNode(ctxDone, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
assert.Error(t, err)
cdf := &CDFMsFactory{
@ -101,13 +99,13 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
cd: 0,
}
_, err = newInsertBufferNode(ctx, replica, cdf, NewAllocatorFactory(), flushChan, saveBinlog, "string")
_, err = newInsertBufferNode(ctx, replica, cdf, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
assert.Error(t, err)
cdf = &CDFMsFactory{
Factory: msFactory,
cd: 1,
}
_, err = newInsertBufferNode(ctx, replica, cdf, NewAllocatorFactory(), flushChan, saveBinlog, "string")
_, err = newInsertBufferNode(ctx, replica, cdf, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
assert.Error(t, err)
}
@ -173,25 +171,19 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
}
flushChan := make(chan *flushMsg, 100)
iBNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
iBNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
require.NoError(t, err)
dmlFlushedCh := make(chan []*datapb.FieldBinlog, 1)
flushChan <- &flushMsg{
msgID: 1,
timestamp: 2000,
segmentID: UniqueID(1),
collectionID: UniqueID(1),
dmlFlushedCh: dmlFlushedCh,
}
inMsg := genInsertMsg(insertChannelName)
var iMsg flowgraph.Msg = &inMsg
iBNode.Operate([]flowgraph.Msg{iMsg})
isflushed := <-dmlFlushedCh
assert.NotNil(t, isflushed)
log.Debug("DML binlog paths", zap.Any("paths", isflushed))
}
func genInsertMsg(insertChannelName string) insertMsg {
@ -279,7 +271,7 @@ func TestFlushSegment(t *testing.T) {
saveBinlog := func(*segmentFlushUnit) error {
return nil
}
ibNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
ibNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
require.NoError(t, err)
flushSegment(collMeta,
@ -394,7 +386,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
}
flushChan := make(chan *flushMsg, 100)
iBNode, err := newInsertBufferNode(ctx, colRep, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
iBNode, err := newInsertBufferNode(ctx, colRep, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
require.NoError(t, err)
// Auto flush number of rows set to 2
@ -512,14 +504,11 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
assert.Equal(t, len(iBNode.insertBuffer.insertData), 1)
assert.Equal(t, iBNode.insertBuffer.size(3), int32(50+16000))
dmlFlushedCh := make(chan []*datapb.FieldBinlog, 1)
flushChan <- &flushMsg{
msgID: 3,
timestamp: 456,
segmentID: UniqueID(1),
collectionID: UniqueID(1),
dmlFlushedCh: dmlFlushedCh,
}
inMsg.insertMessages = []*msgstream.InsertMsg{}
@ -527,12 +516,6 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
inMsg.endPositions = []*internalpb.MsgPosition{{Timestamp: 456}}
iBNode.Operate([]flowgraph.Msg{iMsg})
flushSeg := <-dmlFlushedCh
assert.NotNil(t, flushSeg)
assert.Equal(t, len(flushSeg), 1)
assert.Equal(t, flushSeg[0].FieldID, int64(1))
assert.NotNil(t, flushSeg[0].Binlogs)
assert.Equal(t, len(flushUnit), 3)
assert.Equal(t, flushUnit[2].segID, int64(1))
assert.Equal(t, len(flushUnit[2].checkPoint), 3)
@ -553,14 +536,9 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
timestamp: 567,
segmentID: UniqueID(3),
collectionID: UniqueID(3),
dmlFlushedCh: dmlFlushedCh,
}
iBNode.Operate([]flowgraph.Msg{iMsg})
flushSeg = <-dmlFlushedCh
assert.NotNil(t, flushSeg)
assert.Equal(t, len(flushSeg), 1)
assert.Equal(t, flushSeg[0].FieldID, int64(3))
assert.NotNil(t, flushSeg[0].Binlogs)
assert.Equal(t, len(flushUnit), 4)
assert.Equal(t, flushUnit[3].segID, int64(3))
assert.Equal(t, len(flushUnit[3].checkPoint), 2)
@ -633,7 +611,7 @@ func TestInsertBufferNode_getCollMetaBySegID(t *testing.T) {
}
flushChan := make(chan *flushMsg, 100)
iBNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
iBNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
require.NoError(t, err)
meta, err := iBNode.getCollMetabySegID(1, 101)
@ -686,7 +664,7 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
}
flushChan := make(chan *flushMsg, 100)
iBNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string")
iBNode, err := newInsertBufferNode(ctx, replica, msFactory, NewAllocatorFactory(), flushChan, saveBinlog, "string", newCache())
require.NoError(t, err)
inMsg := genInsertMsg(insertChannelName)

View File

@ -13,7 +13,6 @@ package datanode
import (
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/flowgraph"
)
@ -35,7 +34,6 @@ type flushMsg struct {
timestamp Timestamp
segmentID UniqueID
collectionID UniqueID
dmlFlushedCh chan<- []*datapb.FieldBinlog
}
func (iMsg *insertMsg) TimeTick() Timestamp {