mirror of https://github.com/milvus-io/milvus.git
Make FlushSegments async (#8159)
Resolves: #8122 Signed-off-by: yangxuan <xuan.yang@zilliz.com>pull/8405/head
parent
99296e91b8
commit
d816a81ee3
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
Loading…
Reference in New Issue