mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-08 01:58:34 +08:00
Support sync all segments while close (#21421)
Signed-off-by: xiaofan-luan <xiaofan.luan@zilliz.com>
This commit is contained in:
parent
6fb3542f2a
commit
4b4944ecee
@ -47,6 +47,8 @@ type DelBufferManager struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (bm *DelBufferManager) GetSegDelBufMemSize(segID UniqueID) int64 {
|
func (bm *DelBufferManager) GetSegDelBufMemSize(segID UniqueID) int64 {
|
||||||
|
bm.mu.Lock()
|
||||||
|
defer bm.mu.Unlock()
|
||||||
if delDataBuf, ok := bm.channel.getCurDeleteBuffer(segID); ok {
|
if delDataBuf, ok := bm.channel.getCurDeleteBuffer(segID); ok {
|
||||||
return delDataBuf.item.memorySize
|
return delDataBuf.item.memorySize
|
||||||
}
|
}
|
||||||
@ -54,6 +56,8 @@ func (bm *DelBufferManager) GetSegDelBufMemSize(segID UniqueID) int64 {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (bm *DelBufferManager) GetEntriesNum(segID UniqueID) int64 {
|
func (bm *DelBufferManager) GetEntriesNum(segID UniqueID) int64 {
|
||||||
|
bm.mu.Lock()
|
||||||
|
defer bm.mu.Unlock()
|
||||||
if delDataBuf, ok := bm.channel.getCurDeleteBuffer(segID); ok {
|
if delDataBuf, ok := bm.channel.getCurDeleteBuffer(segID); ok {
|
||||||
return delDataBuf.GetEntriesNum()
|
return delDataBuf.GetEntriesNum()
|
||||||
}
|
}
|
||||||
|
|||||||
@ -191,10 +191,10 @@ func (c *ChannelMeta) maxRowCountPerSegment(ts Timestamp) (int64, error) {
|
|||||||
// Make sure to verify `channel.hasSegment(segID)` == false before calling `channel.addSegment()`.
|
// Make sure to verify `channel.hasSegment(segID)` == false before calling `channel.addSegment()`.
|
||||||
func (c *ChannelMeta) addSegment(req addSegmentReq) error {
|
func (c *ChannelMeta) addSegment(req addSegmentReq) error {
|
||||||
if req.collID != c.collectionID {
|
if req.collID != c.collectionID {
|
||||||
log.Warn("collection mismatch",
|
log.Warn("failed to addSegment, collection mismatch",
|
||||||
zap.Int64("current collection ID", req.collID),
|
zap.Int64("current collection ID", req.collID),
|
||||||
zap.Int64("expected collection ID", c.collectionID))
|
zap.Int64("expected collection ID", c.collectionID))
|
||||||
return fmt.Errorf("mismatch collection, ID=%d", req.collID)
|
return fmt.Errorf("failed to addSegment, mismatch collection, ID=%d", req.collID)
|
||||||
}
|
}
|
||||||
log.Info("adding segment",
|
log.Info("adding segment",
|
||||||
zap.String("type", req.segType.String()),
|
zap.String("type", req.segType.String()),
|
||||||
@ -500,8 +500,11 @@ func (c *ChannelMeta) getCollectionID() UniqueID {
|
|||||||
//
|
//
|
||||||
// If you want the latest collection schema, ts should be 0.
|
// If you want the latest collection schema, ts should be 0.
|
||||||
func (c *ChannelMeta) getCollectionSchema(collID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error) {
|
func (c *ChannelMeta) getCollectionSchema(collID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error) {
|
||||||
if !c.validCollection(collID) {
|
if collID != c.collectionID {
|
||||||
return nil, fmt.Errorf("mismatch collection, want %d, actual %d", c.collectionID, collID)
|
log.Warn("failed to getCollectionSchema, collection mismatch",
|
||||||
|
zap.Int64("current collection ID", collID),
|
||||||
|
zap.Int64("expected collection ID", c.collectionID))
|
||||||
|
return nil, fmt.Errorf("failed to getCollectionSchema, mismatch collection, want %d, actual %d", c.collectionID, collID)
|
||||||
}
|
}
|
||||||
|
|
||||||
c.schemaMut.RLock()
|
c.schemaMut.RLock()
|
||||||
@ -524,12 +527,7 @@ func (c *ChannelMeta) getCollectionSchema(collID UniqueID, ts Timestamp) (*schem
|
|||||||
return c.collSchema, nil
|
return c.collSchema, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *ChannelMeta) validCollection(collID UniqueID) bool {
|
|
||||||
return collID == c.collectionID
|
|
||||||
}
|
|
||||||
|
|
||||||
func (c *ChannelMeta) mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error {
|
func (c *ChannelMeta) mergeFlushedSegments(seg *Segment, planID UniqueID, compactedFrom []UniqueID) error {
|
||||||
|
|
||||||
log := log.With(
|
log := log.With(
|
||||||
zap.Int64("segment ID", seg.segmentID),
|
zap.Int64("segment ID", seg.segmentID),
|
||||||
zap.Int64("collection ID", seg.collectionID),
|
zap.Int64("collection ID", seg.collectionID),
|
||||||
@ -539,9 +537,10 @@ func (c *ChannelMeta) mergeFlushedSegments(seg *Segment, planID UniqueID, compac
|
|||||||
zap.String("channel name", c.channelName))
|
zap.String("channel name", c.channelName))
|
||||||
|
|
||||||
if seg.collectionID != c.collectionID {
|
if seg.collectionID != c.collectionID {
|
||||||
log.Warn("Mismatch collection",
|
log.Warn("failed to mergeFlushedSegments, collection mismatch",
|
||||||
zap.Int64("expected collectionID", c.collectionID))
|
zap.Int64("current collection ID", seg.collectionID),
|
||||||
return fmt.Errorf("mismatch collection, ID=%d", seg.collectionID)
|
zap.Int64("expected collection ID", c.collectionID))
|
||||||
|
return fmt.Errorf("failed to mergeFlushedSegments, mismatch collection, ID=%d", seg.collectionID)
|
||||||
}
|
}
|
||||||
|
|
||||||
compactedFrom = lo.Filter(compactedFrom, func(segID int64, _ int) bool {
|
compactedFrom = lo.Filter(compactedFrom, func(segID int64, _ int) bool {
|
||||||
@ -578,10 +577,10 @@ func (c *ChannelMeta) mergeFlushedSegments(seg *Segment, planID UniqueID, compac
|
|||||||
// for tests only
|
// for tests only
|
||||||
func (c *ChannelMeta) addFlushedSegmentWithPKs(segID, collID, partID UniqueID, numOfRows int64, ids storage.FieldData) error {
|
func (c *ChannelMeta) addFlushedSegmentWithPKs(segID, collID, partID UniqueID, numOfRows int64, ids storage.FieldData) error {
|
||||||
if collID != c.collectionID {
|
if collID != c.collectionID {
|
||||||
log.Warn("Mismatch collection",
|
log.Warn("failed to addFlushedSegmentWithPKs, collection mismatch",
|
||||||
zap.Int64("input ID", collID),
|
zap.Int64("current collection ID", collID),
|
||||||
zap.Int64("expected ID", c.collectionID))
|
zap.Int64("expected collection ID", c.collectionID))
|
||||||
return fmt.Errorf("mismatch collection, ID=%d", collID)
|
return fmt.Errorf("failed to addFlushedSegmentWithPKs, mismatch collection, ID=%d", collID)
|
||||||
}
|
}
|
||||||
|
|
||||||
log.Info("Add Flushed segment",
|
log.Info("Add Flushed segment",
|
||||||
|
|||||||
@ -20,6 +20,7 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"errors"
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"sync"
|
||||||
|
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
@ -44,10 +45,10 @@ type dataSyncService struct {
|
|||||||
ctx context.Context
|
ctx context.Context
|
||||||
cancelFn context.CancelFunc
|
cancelFn context.CancelFunc
|
||||||
fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages
|
fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages
|
||||||
flushCh chan flushMsg // chan to notify flush
|
flushCh chan flushMsg
|
||||||
resendTTCh chan resendTTMsg // chan to ask for resending DataNode time tick message.
|
resendTTCh chan resendTTMsg // chan to ask for resending DataNode time tick message.
|
||||||
channel Channel // channel stores meta of channel
|
channel Channel // channel stores meta of channel
|
||||||
idAllocator allocatorInterface // id/timestamp allocator
|
idAllocator allocatorInterface // id/timestamp allocator
|
||||||
msFactory msgstream.Factory
|
msFactory msgstream.Factory
|
||||||
collectionID UniqueID // collection id of vchan for which this data sync service serves
|
collectionID UniqueID // collection id of vchan for which this data sync service serves
|
||||||
vchannelName string
|
vchannelName string
|
||||||
@ -59,6 +60,9 @@ type dataSyncService struct {
|
|||||||
flushManager flushManager // flush manager handles flush process
|
flushManager flushManager // flush manager handles flush process
|
||||||
chunkManager storage.ChunkManager
|
chunkManager storage.ChunkManager
|
||||||
compactor *compactionExecutor // reference to compaction executor
|
compactor *compactionExecutor // reference to compaction executor
|
||||||
|
|
||||||
|
stopOnce sync.Once
|
||||||
|
flushListener chan *segmentFlushPack // chan to listen flush event
|
||||||
}
|
}
|
||||||
|
|
||||||
func newDataSyncService(ctx context.Context,
|
func newDataSyncService(ctx context.Context,
|
||||||
@ -132,7 +136,7 @@ func newParallelConfig() parallelConfig {
|
|||||||
return parallelConfig{Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(), Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()}
|
return parallelConfig{Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(), Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()}
|
||||||
}
|
}
|
||||||
|
|
||||||
// start starts the flow graph in datasyncservice
|
// start the flow graph in datasyncservice
|
||||||
func (dsService *dataSyncService) start() {
|
func (dsService *dataSyncService) start() {
|
||||||
if dsService.fg != nil {
|
if dsService.fg != nil {
|
||||||
log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID),
|
log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID),
|
||||||
@ -145,18 +149,20 @@ func (dsService *dataSyncService) start() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (dsService *dataSyncService) close() {
|
func (dsService *dataSyncService) close() {
|
||||||
if dsService.fg != nil {
|
dsService.stopOnce.Do(func() {
|
||||||
log.Info("dataSyncService closing flowgraph", zap.Int64("collectionID", dsService.collectionID),
|
if dsService.fg != nil {
|
||||||
zap.String("vChanName", dsService.vchannelName))
|
log.Info("dataSyncService closing flowgraph", zap.Int64("collectionID", dsService.collectionID),
|
||||||
dsService.fg.Close()
|
zap.String("vChanName", dsService.vchannelName))
|
||||||
metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec()
|
dsService.fg.Close()
|
||||||
metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Sub(2) // timeTickChannel + deltaChannel
|
metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec()
|
||||||
}
|
metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Sub(2) // timeTickChannel + deltaChannel
|
||||||
|
}
|
||||||
|
|
||||||
dsService.clearGlobalFlushingCache()
|
dsService.clearGlobalFlushingCache()
|
||||||
|
close(dsService.flushCh)
|
||||||
dsService.cancelFn()
|
dsService.flushManager.close()
|
||||||
dsService.flushManager.close()
|
dsService.cancelFn()
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
func (dsService *dataSyncService) clearGlobalFlushingCache() {
|
func (dsService *dataSyncService) clearGlobalFlushingCache() {
|
||||||
@ -171,6 +177,11 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
|
|||||||
dsService.flushManager = NewRendezvousFlushManager(dsService.idAllocator, dsService.chunkManager, dsService.channel,
|
dsService.flushManager = NewRendezvousFlushManager(dsService.idAllocator, dsService.chunkManager, dsService.channel,
|
||||||
flushNotifyFunc(dsService, retry.Attempts(50)), dropVirtualChannelFunc(dsService))
|
flushNotifyFunc(dsService, retry.Attempts(50)), dropVirtualChannelFunc(dsService))
|
||||||
|
|
||||||
|
log.Info("begin to init data sync service", zap.Int64("collection", vchanInfo.CollectionID),
|
||||||
|
zap.String("Chan", vchanInfo.ChannelName),
|
||||||
|
zap.Int64s("unflushed", vchanInfo.GetUnflushedSegmentIds()),
|
||||||
|
zap.Int64s("flushed", vchanInfo.GetFlushedSegmentIds()),
|
||||||
|
)
|
||||||
var err error
|
var err error
|
||||||
// recover segment checkpoints
|
// recover segment checkpoints
|
||||||
unflushedSegmentInfos, err := dsService.getSegmentInfos(vchanInfo.GetUnflushedSegmentIds())
|
unflushedSegmentInfos, err := dsService.getSegmentInfos(vchanInfo.GetUnflushedSegmentIds())
|
||||||
@ -187,7 +198,7 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
|
|||||||
for _, us := range unflushedSegmentInfos {
|
for _, us := range unflushedSegmentInfos {
|
||||||
if us.CollectionID != dsService.collectionID ||
|
if us.CollectionID != dsService.collectionID ||
|
||||||
us.GetInsertChannel() != vchanInfo.ChannelName {
|
us.GetInsertChannel() != vchanInfo.ChannelName {
|
||||||
log.Warn("Collection ID or ChannelName not compact",
|
log.Warn("Collection ID or ChannelName not match",
|
||||||
zap.Int64("Wanted ID", dsService.collectionID),
|
zap.Int64("Wanted ID", dsService.collectionID),
|
||||||
zap.Int64("Actual ID", us.CollectionID),
|
zap.Int64("Actual ID", us.CollectionID),
|
||||||
zap.String("Wanted Channel Name", vchanInfo.ChannelName),
|
zap.String("Wanted Channel Name", vchanInfo.ChannelName),
|
||||||
@ -224,7 +235,7 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
|
|||||||
for _, fs := range flushedSegmentInfos {
|
for _, fs := range flushedSegmentInfos {
|
||||||
if fs.CollectionID != dsService.collectionID ||
|
if fs.CollectionID != dsService.collectionID ||
|
||||||
fs.GetInsertChannel() != vchanInfo.ChannelName {
|
fs.GetInsertChannel() != vchanInfo.ChannelName {
|
||||||
log.Warn("Collection ID or ChannelName not compact",
|
log.Warn("Collection ID or ChannelName not match",
|
||||||
zap.Int64("Wanted ID", dsService.collectionID),
|
zap.Int64("Wanted ID", dsService.collectionID),
|
||||||
zap.Int64("Actual ID", fs.CollectionID),
|
zap.Int64("Actual ID", fs.CollectionID),
|
||||||
zap.String("Wanted Channel Name", vchanInfo.ChannelName),
|
zap.String("Wanted Channel Name", vchanInfo.ChannelName),
|
||||||
|
|||||||
@ -21,6 +21,7 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"math"
|
"math"
|
||||||
|
"os"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
@ -37,6 +38,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service"
|
var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service"
|
||||||
@ -190,38 +192,36 @@ func TestDataSyncService_newDataSyncService(te *testing.T) {
|
|||||||
|
|
||||||
// NOTE: start pulsar before test
|
// NOTE: start pulsar before test
|
||||||
func TestDataSyncService_Start(t *testing.T) {
|
func TestDataSyncService_Start(t *testing.T) {
|
||||||
t.Skip()
|
const ctxTimeInMillisecond = 10000
|
||||||
const ctxTimeInMillisecond = 2000
|
|
||||||
|
|
||||||
delay := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
|
delay := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
|
||||||
ctx, cancel := context.WithDeadline(context.Background(), delay)
|
ctx, cancel := context.WithDeadline(context.Background(), delay)
|
||||||
defer cancel()
|
defer cancel()
|
||||||
|
|
||||||
// init data node
|
// init data node
|
||||||
|
insertChannelName := "by-dev-rootcoord-dml"
|
||||||
insertChannelName := "data_sync_service_test_dml"
|
ddlChannelName := "by-dev-rootcoord-ddl"
|
||||||
ddlChannelName := "data_sync_service_test_ddl"
|
|
||||||
|
|
||||||
Factory := &MetaFactory{}
|
Factory := &MetaFactory{}
|
||||||
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
|
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
|
||||||
mockRootCoord := &RootCoordFactory{
|
mockRootCoord := &RootCoordFactory{
|
||||||
pkType: schemapb.DataType_Int64,
|
pkType: schemapb.DataType_Int64,
|
||||||
}
|
}
|
||||||
collectionID := UniqueID(1)
|
|
||||||
|
|
||||||
flushChan := make(chan flushMsg, 100)
|
flushChan := make(chan flushMsg, 100)
|
||||||
resendTTChan := make(chan resendTTMsg, 100)
|
resendTTChan := make(chan resendTTMsg, 100)
|
||||||
cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
|
cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
|
||||||
defer cm.RemoveWithPrefix(ctx, cm.RootPath())
|
defer cm.RemoveWithPrefix(ctx, cm.RootPath())
|
||||||
channel := newChannel(insertChannelName, collectionID, collMeta.GetSchema(), mockRootCoord, cm)
|
channel := newChannel(insertChannelName, collMeta.ID, collMeta.GetSchema(), mockRootCoord, cm)
|
||||||
|
|
||||||
allocFactory := NewAllocatorFactory(1)
|
allocFactory := NewAllocatorFactory(1)
|
||||||
factory := dependency.NewDefaultFactory(true)
|
factory := dependency.NewDefaultFactory(true)
|
||||||
|
defer os.RemoveAll("/tmp/milvus")
|
||||||
paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "1")
|
paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "1")
|
||||||
|
|
||||||
ufs := []*datapb.SegmentInfo{{
|
ufs := []*datapb.SegmentInfo{{
|
||||||
CollectionID: collMeta.ID,
|
CollectionID: collMeta.ID,
|
||||||
|
PartitionID: 1,
|
||||||
InsertChannel: insertChannelName,
|
InsertChannel: insertChannelName,
|
||||||
ID: 0,
|
ID: 0,
|
||||||
NumOfRows: 0,
|
NumOfRows: 0,
|
||||||
@ -251,18 +251,38 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
signalCh := make(chan string, 100)
|
signalCh := make(chan string, 100)
|
||||||
sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, allocFactory, factory, vchan, signalCh, &DataCoordFactory{}, newCache(), cm, newCompactionExecutor())
|
|
||||||
|
|
||||||
|
dataCoord := &DataCoordFactory{}
|
||||||
|
dataCoord.UserSegmentInfo = map[int64]*datapb.SegmentInfo{
|
||||||
|
0: {
|
||||||
|
ID: 0,
|
||||||
|
CollectionID: collMeta.ID,
|
||||||
|
PartitionID: 1,
|
||||||
|
InsertChannel: insertChannelName,
|
||||||
|
},
|
||||||
|
|
||||||
|
1: {
|
||||||
|
ID: 1,
|
||||||
|
CollectionID: collMeta.ID,
|
||||||
|
PartitionID: 1,
|
||||||
|
InsertChannel: insertChannelName,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, allocFactory, factory, vchan, signalCh, dataCoord, newCache(), cm, newCompactionExecutor())
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
// sync.channel.addCollection(collMeta.ID, collMeta.Schema)
|
|
||||||
|
sync.flushListener = make(chan *segmentFlushPack)
|
||||||
|
defer close(sync.flushListener)
|
||||||
sync.start()
|
sync.start()
|
||||||
|
defer sync.close()
|
||||||
|
|
||||||
timeRange := TimeRange{
|
timeRange := TimeRange{
|
||||||
timestampMin: 0,
|
timestampMin: 0,
|
||||||
timestampMax: math.MaxUint64,
|
timestampMax: math.MaxUint64,
|
||||||
}
|
}
|
||||||
dataFactory := NewDataFactory()
|
dataFactory := NewDataFactory()
|
||||||
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName)
|
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName, tsoutil.GetCurrentTime())
|
||||||
|
|
||||||
msgPack := msgstream.MsgPack{
|
msgPack := msgstream.MsgPack{
|
||||||
BeginTs: timeRange.timestampMin,
|
BeginTs: timeRange.timestampMin,
|
||||||
@ -315,10 +335,213 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||||||
_, err = ddMsgStream.Broadcast(&timeTickMsgPack)
|
_, err = ddMsgStream.Broadcast(&timeTickMsgPack)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
// dataSync
|
select {
|
||||||
<-sync.ctx.Done()
|
case flushPack := <-sync.flushListener:
|
||||||
|
assert.True(t, flushPack.segmentID == 1)
|
||||||
|
return
|
||||||
|
case <-sync.ctx.Done():
|
||||||
|
assert.Fail(t, "test timeout")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestDataSyncService_Close(t *testing.T) {
|
||||||
|
const ctxTimeInMillisecond = 1000000
|
||||||
|
|
||||||
|
delay := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
|
||||||
|
ctx, cancel := context.WithDeadline(context.Background(), delay)
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
os.RemoveAll("/tmp/milvus")
|
||||||
|
|
||||||
|
// init data node
|
||||||
|
insertChannelName := "by-dev-rootcoord-dml2"
|
||||||
|
ddlChannelName := "by-dev-rootcoord-ddl2"
|
||||||
|
|
||||||
|
Factory := &MetaFactory{}
|
||||||
|
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
|
||||||
|
mockRootCoord := &RootCoordFactory{
|
||||||
|
pkType: schemapb.DataType_Int64,
|
||||||
|
}
|
||||||
|
|
||||||
|
flushChan := make(chan flushMsg, 100)
|
||||||
|
resendTTChan := make(chan resendTTMsg, 100)
|
||||||
|
cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
|
||||||
|
defer cm.RemoveWithPrefix(ctx, cm.RootPath())
|
||||||
|
channel := newChannel(insertChannelName, collMeta.ID, collMeta.GetSchema(), mockRootCoord, cm)
|
||||||
|
|
||||||
|
allocFactory := NewAllocatorFactory(1)
|
||||||
|
factory := dependency.NewDefaultFactory(true)
|
||||||
|
defer os.RemoveAll("/tmp/milvus")
|
||||||
|
|
||||||
|
paramtable.Get().Remove(Params.DataNodeCfg.FlushInsertBufferSize.Key)
|
||||||
|
ufs := []*datapb.SegmentInfo{{
|
||||||
|
CollectionID: collMeta.ID,
|
||||||
|
PartitionID: 1,
|
||||||
|
InsertChannel: insertChannelName,
|
||||||
|
ID: 0,
|
||||||
|
NumOfRows: 0,
|
||||||
|
DmlPosition: &internalpb.MsgPosition{},
|
||||||
|
}}
|
||||||
|
fs := []*datapb.SegmentInfo{{
|
||||||
|
CollectionID: collMeta.ID,
|
||||||
|
PartitionID: 1,
|
||||||
|
InsertChannel: insertChannelName,
|
||||||
|
ID: 1,
|
||||||
|
NumOfRows: 0,
|
||||||
|
DmlPosition: &internalpb.MsgPosition{},
|
||||||
|
}}
|
||||||
|
var ufsIds []int64
|
||||||
|
var fsIds []int64
|
||||||
|
for _, segmentInfo := range ufs {
|
||||||
|
ufsIds = append(ufsIds, segmentInfo.ID)
|
||||||
|
}
|
||||||
|
for _, segmentInfo := range fs {
|
||||||
|
fsIds = append(fsIds, segmentInfo.ID)
|
||||||
|
}
|
||||||
|
vchan := &datapb.VchannelInfo{
|
||||||
|
CollectionID: collMeta.ID,
|
||||||
|
ChannelName: insertChannelName,
|
||||||
|
UnflushedSegmentIds: ufsIds,
|
||||||
|
FlushedSegmentIds: fsIds,
|
||||||
|
}
|
||||||
|
|
||||||
|
signalCh := make(chan string, 100)
|
||||||
|
|
||||||
|
dataCoord := &DataCoordFactory{}
|
||||||
|
dataCoord.UserSegmentInfo = map[int64]*datapb.SegmentInfo{
|
||||||
|
0: {
|
||||||
|
ID: 0,
|
||||||
|
CollectionID: collMeta.ID,
|
||||||
|
PartitionID: 1,
|
||||||
|
InsertChannel: insertChannelName,
|
||||||
|
},
|
||||||
|
|
||||||
|
1: {
|
||||||
|
ID: 1,
|
||||||
|
CollectionID: collMeta.ID,
|
||||||
|
PartitionID: 1,
|
||||||
|
InsertChannel: insertChannelName,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
sync, err := newDataSyncService(ctx, flushChan, resendTTChan, channel, allocFactory, factory, vchan, signalCh, dataCoord, newCache(), cm, newCompactionExecutor())
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
sync.flushListener = make(chan *segmentFlushPack, 10)
|
||||||
|
defer close(sync.flushListener)
|
||||||
|
sync.start()
|
||||||
|
|
||||||
|
dataFactory := NewDataFactory()
|
||||||
|
ts := tsoutil.GetCurrentTime()
|
||||||
|
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName, ts)
|
||||||
|
msgPack := msgstream.MsgPack{
|
||||||
|
BeginTs: ts,
|
||||||
|
EndTs: ts,
|
||||||
|
Msgs: insertMessages,
|
||||||
|
StartPositions: []*internalpb.MsgPosition{{
|
||||||
|
ChannelName: insertChannelName,
|
||||||
|
}},
|
||||||
|
EndPositions: []*internalpb.MsgPosition{{
|
||||||
|
ChannelName: insertChannelName,
|
||||||
|
}},
|
||||||
|
}
|
||||||
|
|
||||||
|
// 400 is the actual data
|
||||||
|
int64Pks := []primaryKey{
|
||||||
|
newInt64PrimaryKey(400),
|
||||||
|
}
|
||||||
|
deleteMessages := dataFactory.GenMsgStreamDeleteMsgWithTs(0, int64Pks, insertChannelName, ts+1)
|
||||||
|
inMsgs := make([]msgstream.TsMsg, 0)
|
||||||
|
inMsgs = append(inMsgs, deleteMessages)
|
||||||
|
|
||||||
|
msgPackDelete := msgstream.MsgPack{
|
||||||
|
BeginTs: ts + 1,
|
||||||
|
EndTs: ts + 1,
|
||||||
|
Msgs: inMsgs,
|
||||||
|
StartPositions: []*internalpb.MsgPosition{{
|
||||||
|
ChannelName: insertChannelName,
|
||||||
|
}},
|
||||||
|
EndPositions: []*internalpb.MsgPosition{{
|
||||||
|
ChannelName: insertChannelName,
|
||||||
|
}},
|
||||||
|
}
|
||||||
|
|
||||||
|
// generate timeTick
|
||||||
|
timeTickMsg := &msgstream.TimeTickMsg{
|
||||||
|
BaseMsg: msgstream.BaseMsg{
|
||||||
|
BeginTimestamp: ts,
|
||||||
|
EndTimestamp: ts + 2,
|
||||||
|
HashValues: []uint32{0},
|
||||||
|
},
|
||||||
|
TimeTickMsg: internalpb.TimeTickMsg{
|
||||||
|
Base: &commonpb.MsgBase{
|
||||||
|
MsgType: commonpb.MsgType_TimeTick,
|
||||||
|
MsgID: UniqueID(2),
|
||||||
|
Timestamp: ts + 2,
|
||||||
|
SourceID: 0,
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
timeTickMsgPack := msgstream.MsgPack{
|
||||||
|
BeginTs: ts + 2,
|
||||||
|
EndTs: ts + 2,
|
||||||
|
StartPositions: []*internalpb.MsgPosition{{
|
||||||
|
ChannelName: insertChannelName,
|
||||||
|
}},
|
||||||
|
EndPositions: []*internalpb.MsgPosition{{
|
||||||
|
ChannelName: insertChannelName,
|
||||||
|
}},
|
||||||
|
}
|
||||||
|
timeTickMsgPack.Msgs = append(timeTickMsgPack.Msgs, timeTickMsg)
|
||||||
|
|
||||||
|
// pulsar produce
|
||||||
|
assert.NoError(t, err)
|
||||||
|
insertStream, _ := factory.NewMsgStream(ctx)
|
||||||
|
insertStream.AsProducer([]string{insertChannelName})
|
||||||
|
|
||||||
|
ddStream, _ := factory.NewMsgStream(ctx)
|
||||||
|
ddStream.AsProducer([]string{ddlChannelName})
|
||||||
|
|
||||||
|
var insertMsgStream msgstream.MsgStream = insertStream
|
||||||
|
var ddMsgStream msgstream.MsgStream = ddStream
|
||||||
|
|
||||||
|
err = insertMsgStream.Produce(&msgPack)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
err = insertMsgStream.Produce(&msgPackDelete)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
_, err = insertMsgStream.Broadcast(&timeTickMsgPack)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
_, err = ddMsgStream.Broadcast(&timeTickMsgPack)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
// wait for delete
|
||||||
|
for sync.delBufferManager.GetEntriesNum(1) == 0 {
|
||||||
|
time.Sleep(100)
|
||||||
|
}
|
||||||
|
|
||||||
|
// close and wait for flush
|
||||||
sync.close()
|
sync.close()
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case flushPack, ok := <-sync.flushListener:
|
||||||
|
assert.True(t, ok)
|
||||||
|
if flushPack.segmentID == 1 {
|
||||||
|
assert.True(t, len(flushPack.insertLogs) == 12)
|
||||||
|
assert.True(t, len(flushPack.statsLogs) == 1)
|
||||||
|
assert.True(t, len(flushPack.deltaLogs) == 1)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
if flushPack.segmentID == 0 {
|
||||||
|
assert.True(t, len(flushPack.insertLogs) == 0)
|
||||||
|
assert.True(t, len(flushPack.statsLogs) == 0)
|
||||||
|
assert.True(t, len(flushPack.deltaLogs) == 0)
|
||||||
|
}
|
||||||
|
case <-sync.ctx.Done():
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func genBytes() (rawData []byte) {
|
func genBytes() (rawData []byte) {
|
||||||
@ -405,6 +628,22 @@ func TestGetSegmentInfos(t *testing.T) {
|
|||||||
segmentInfos3, err := dsService.getSegmentInfos([]int64{1})
|
segmentInfos3, err := dsService.getSegmentInfos([]int64{1})
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
assert.Empty(t, segmentInfos3)
|
assert.Empty(t, segmentInfos3)
|
||||||
|
|
||||||
|
dataCoord.GetSegmentInfosError = false
|
||||||
|
dataCoord.GetSegmentInfosNotSuccess = false
|
||||||
|
dataCoord.UserSegmentInfo = map[int64]*datapb.SegmentInfo{
|
||||||
|
5: {
|
||||||
|
ID: 100,
|
||||||
|
CollectionID: 101,
|
||||||
|
PartitionID: 102,
|
||||||
|
InsertChannel: "by-dev-rootcoord-dml-test_v1",
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
segmentInfos, err = dsService.getSegmentInfos([]int64{5})
|
||||||
|
assert.NoError(t, err)
|
||||||
|
assert.Equal(t, 1, len(segmentInfos))
|
||||||
|
assert.Equal(t, int64(100), segmentInfos[0].ID)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestClearGlobalFlushingCache(t *testing.T) {
|
func TestClearGlobalFlushingCache(t *testing.T) {
|
||||||
|
|||||||
@ -86,37 +86,60 @@ func (ddn *ddNode) Name() string {
|
|||||||
return fmt.Sprintf("ddNode-%d-%s", ddn.collectionID, ddn.vChannelName)
|
return fmt.Sprintf("ddNode-%d-%s", ddn.collectionID, ddn.vChannelName)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (ddn *ddNode) IsValidInMsg(in []Msg) bool {
|
||||||
|
if !ddn.BaseNode.IsValidInMsg(in) {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
_, ok := in[0].(*MsgStreamMsg)
|
||||||
|
if !ok {
|
||||||
|
log.Warn("type assertion failed for MsgStreamMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
// Operate handles input messages, implementing flowgrpah.Node
|
// Operate handles input messages, implementing flowgrpah.Node
|
||||||
func (ddn *ddNode) Operate(in []Msg) []Msg {
|
func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||||
if in == nil {
|
|
||||||
log.Debug("type assertion failed for MsgStreamMsg because it's nil")
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
if len(in) != 1 {
|
|
||||||
log.Warn("Invalid operate message input in ddNode", zap.Int("input length", len(in)))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
msMsg, ok := in[0].(*MsgStreamMsg)
|
msMsg, ok := in[0].(*MsgStreamMsg)
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("type assertion failed for MsgStreamMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
log.Warn("type assertion failed for MsgStreamMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
||||||
return []Msg{}
|
return []Msg{}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if msMsg.IsCloseMsg() {
|
||||||
|
var fgMsg = flowGraphMsg{
|
||||||
|
BaseMsg: flowgraph.NewBaseMsg(true),
|
||||||
|
insertMessages: make([]*msgstream.InsertMsg, 0),
|
||||||
|
timeRange: TimeRange{
|
||||||
|
timestampMin: msMsg.TimestampMin(),
|
||||||
|
timestampMax: msMsg.TimestampMax(),
|
||||||
|
},
|
||||||
|
startPositions: msMsg.StartPositions(),
|
||||||
|
endPositions: msMsg.EndPositions(),
|
||||||
|
dropCollection: false,
|
||||||
|
}
|
||||||
|
log.Warn("MsgStream closed", zap.Any("ddNode node", ddn.Name()), zap.Int64("collection", ddn.collectionID), zap.String("channel", ddn.vChannelName))
|
||||||
|
return []Msg{&fgMsg}
|
||||||
|
}
|
||||||
|
|
||||||
|
if load := ddn.dropMode.Load(); load != nil && load.(bool) {
|
||||||
|
log.Debug("ddNode in dropMode",
|
||||||
|
zap.String("vChannelName", ddn.vChannelName),
|
||||||
|
zap.Int64("collection ID", ddn.collectionID))
|
||||||
|
return []Msg{}
|
||||||
|
}
|
||||||
|
|
||||||
var spans []opentracing.Span
|
var spans []opentracing.Span
|
||||||
for _, msg := range msMsg.TsMessages() {
|
for _, msg := range msMsg.TsMessages() {
|
||||||
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
||||||
spans = append(spans, sp)
|
spans = append(spans, sp)
|
||||||
msg.SetTraceCtx(ctx)
|
msg.SetTraceCtx(ctx)
|
||||||
}
|
}
|
||||||
|
defer func() {
|
||||||
if load := ddn.dropMode.Load(); load != nil && load.(bool) {
|
for _, sp := range spans {
|
||||||
log.Debug("ddNode in dropMode",
|
sp.Finish()
|
||||||
zap.String("vChannelName", ddn.vChannelName),
|
}
|
||||||
zap.Int64("collection ID", ddn.collectionID))
|
}()
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
var fgMsg = flowGraphMsg{
|
var fgMsg = flowGraphMsg{
|
||||||
insertMessages: make([]*msgstream.InsertMsg, 0),
|
insertMessages: make([]*msgstream.InsertMsg, 0),
|
||||||
@ -193,7 +216,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
|||||||
case commonpb.MsgType_Delete:
|
case commonpb.MsgType_Delete:
|
||||||
dmsg := msg.(*msgstream.DeleteMsg)
|
dmsg := msg.(*msgstream.DeleteMsg)
|
||||||
log.Debug("DDNode receive delete messages",
|
log.Debug("DDNode receive delete messages",
|
||||||
zap.Int64("num", dmsg.NumRows),
|
zap.Int64("numRows", dmsg.NumRows),
|
||||||
zap.String("vChannelName", ddn.vChannelName))
|
zap.String("vChannelName", ddn.vChannelName))
|
||||||
for i := int64(0); i < dmsg.NumRows; i++ {
|
for i := int64(0); i < dmsg.NumRows; i++ {
|
||||||
dmsg.HashValues = append(dmsg.HashValues, uint32(0))
|
dmsg.HashValues = append(dmsg.HashValues, uint32(0))
|
||||||
@ -231,10 +254,6 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
|||||||
fgMsg.startPositions = append(fgMsg.startPositions, msMsg.StartPositions()...)
|
fgMsg.startPositions = append(fgMsg.startPositions, msMsg.StartPositions()...)
|
||||||
fgMsg.endPositions = append(fgMsg.endPositions, msMsg.EndPositions()...)
|
fgMsg.endPositions = append(fgMsg.endPositions, msMsg.EndPositions()...)
|
||||||
|
|
||||||
for _, sp := range spans {
|
|
||||||
sp.Finish()
|
|
||||||
}
|
|
||||||
|
|
||||||
return []Msg{&fgMsg}
|
return []Msg{&fgMsg}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -110,11 +110,9 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
|
|||||||
for _, test := range invalidInTests {
|
for _, test := range invalidInTests {
|
||||||
t.Run(test.description, func(t *testing.T) {
|
t.Run(test.description, func(t *testing.T) {
|
||||||
ddn := ddNode{}
|
ddn := ddNode{}
|
||||||
rt := ddn.Operate(test.in)
|
assert.False(t, ddn.IsValidInMsg(test.in))
|
||||||
assert.Empty(t, rt)
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
// valid inputs
|
// valid inputs
|
||||||
tests := []struct {
|
tests := []struct {
|
||||||
ddnCollID UniqueID
|
ddnCollID UniqueID
|
||||||
|
|||||||
@ -67,23 +67,21 @@ func (dn *deleteNode) showDelBuf(segIDs []UniqueID, ts Timestamp) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Operate implementing flowgraph.Node, performs delete data process
|
func (dn *deleteNode) IsValidInMsg(in []Msg) bool {
|
||||||
func (dn *deleteNode) Operate(in []Msg) []Msg {
|
if !dn.BaseNode.IsValidInMsg(in) {
|
||||||
if in == nil {
|
return false
|
||||||
log.Debug("type assertion failed for flowGraphMsg because it's nil")
|
|
||||||
return []Msg{}
|
|
||||||
}
|
}
|
||||||
|
_, ok := in[0].(*flowGraphMsg)
|
||||||
if len(in) != 1 {
|
|
||||||
log.Warn("Invalid operate message input in deleteNode", zap.Int("input length", len(in)))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
fgMsg, ok := in[0].(*flowGraphMsg)
|
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
||||||
return []Msg{}
|
return false
|
||||||
}
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
// Operate implementing flowgraph.Node, performs delete data process
|
||||||
|
func (dn *deleteNode) Operate(in []Msg) []Msg {
|
||||||
|
fgMsg := in[0].(*flowGraphMsg)
|
||||||
|
|
||||||
var spans []opentracing.Span
|
var spans []opentracing.Span
|
||||||
for _, msg := range fgMsg.deleteMessages {
|
for _, msg := range fgMsg.deleteMessages {
|
||||||
@ -127,6 +125,7 @@ func (dn *deleteNode) Operate(in []Msg) []Msg {
|
|||||||
// no related delta data to flush, send empty buf to complete flush life-cycle
|
// no related delta data to flush, send empty buf to complete flush life-cycle
|
||||||
dn.flushManager.flushDelData(nil, segmentToFlush, fgMsg.endPositions[0])
|
dn.flushManager.flushDelData(nil, segmentToFlush, fgMsg.endPositions[0])
|
||||||
} else {
|
} else {
|
||||||
|
// TODO, this has to be async, no need to block here
|
||||||
err := retry.Do(dn.ctx, func() error {
|
err := retry.Do(dn.ctx, func() error {
|
||||||
return dn.flushManager.flushDelData(buf, segmentToFlush, fgMsg.endPositions[0])
|
return dn.flushManager.flushDelData(buf, segmentToFlush, fgMsg.endPositions[0])
|
||||||
}, getFlowGraphRetryOpt())
|
}, getFlowGraphRetryOpt())
|
||||||
|
|||||||
@ -151,8 +151,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
|
|||||||
for _, test := range invalidInTests {
|
for _, test := range invalidInTests {
|
||||||
te.Run(test.desc, func(t *testing.T) {
|
te.Run(test.desc, func(t *testing.T) {
|
||||||
dn := deleteNode{}
|
dn := deleteNode{}
|
||||||
rt := dn.Operate(test.in)
|
assert.False(t, dn.IsValidInMsg(test.in))
|
||||||
assert.Empty(t, rt)
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
@ -439,6 +438,11 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
|
|||||||
//2. here we set flushing segments inside fgmsg to empty
|
//2. here we set flushing segments inside fgmsg to empty
|
||||||
//in order to verify the validity of auto flush function
|
//in order to verify the validity of auto flush function
|
||||||
msg := genFlowGraphDeleteMsg(int64Pks, chanName)
|
msg := genFlowGraphDeleteMsg(int64Pks, chanName)
|
||||||
|
|
||||||
|
// delete has to match segment partition ID
|
||||||
|
for _, msg := range msg.deleteMessages {
|
||||||
|
msg.PartitionID = 0
|
||||||
|
}
|
||||||
msg.segmentsToSync = []UniqueID{}
|
msg.segmentsToSync = []UniqueID{}
|
||||||
|
|
||||||
var fgMsg flowgraph.Msg = &msg
|
var fgMsg flowgraph.Msg = &msg
|
||||||
|
|||||||
@ -36,6 +36,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
"github.com/milvus-io/milvus/internal/util/commonpbutil"
|
"github.com/milvus-io/milvus/internal/util/commonpbutil"
|
||||||
|
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
"github.com/milvus-io/milvus/internal/util/retry"
|
||||||
@ -103,10 +104,36 @@ func (ibNode *insertBufferNode) Close() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
|
func (ibNode *insertBufferNode) IsValidInMsg(in []Msg) bool {
|
||||||
fgMsg, ok := ibNode.verifyInMsg(in)
|
if !ibNode.BaseNode.IsValidInMsg(in) {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
_, ok := in[0].(*flowGraphMsg)
|
||||||
if !ok {
|
if !ok {
|
||||||
return []Msg{}
|
log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
|
||||||
|
fgMsg := in[0].(*flowGraphMsg)
|
||||||
|
if fgMsg.IsCloseMsg() {
|
||||||
|
if len(fgMsg.endPositions) != 0 {
|
||||||
|
// try to sync all segments
|
||||||
|
segmentsToSync := ibNode.Sync(fgMsg, make([]UniqueID, 0), fgMsg.endPositions[0])
|
||||||
|
res := flowGraphMsg{
|
||||||
|
deleteMessages: []*msgstream.DeleteMsg{},
|
||||||
|
timeRange: fgMsg.timeRange,
|
||||||
|
startPositions: fgMsg.startPositions,
|
||||||
|
endPositions: fgMsg.endPositions,
|
||||||
|
segmentsToSync: segmentsToSync,
|
||||||
|
dropCollection: fgMsg.dropCollection,
|
||||||
|
BaseMsg: flowgraph.NewBaseMsg(true),
|
||||||
|
}
|
||||||
|
return []Msg{&res}
|
||||||
|
}
|
||||||
|
return in
|
||||||
}
|
}
|
||||||
|
|
||||||
if fgMsg.dropCollection {
|
if fgMsg.dropCollection {
|
||||||
@ -120,6 +147,12 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
|
|||||||
msg.SetTraceCtx(ctx)
|
msg.SetTraceCtx(ctx)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
defer func() {
|
||||||
|
for _, sp := range spans {
|
||||||
|
sp.Finish()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
// replace pchannel with vchannel
|
// replace pchannel with vchannel
|
||||||
startPositions := make([]*internalpb.MsgPosition, 0, len(fgMsg.startPositions))
|
startPositions := make([]*internalpb.MsgPosition, 0, len(fgMsg.startPositions))
|
||||||
for idx := range fgMsg.startPositions {
|
for idx := range fgMsg.startPositions {
|
||||||
@ -181,33 +214,10 @@ func (ibNode *insertBufferNode) Operate(in []Msg) []Msg {
|
|||||||
dropCollection: fgMsg.dropCollection,
|
dropCollection: fgMsg.dropCollection,
|
||||||
}
|
}
|
||||||
|
|
||||||
for _, sp := range spans {
|
|
||||||
sp.Finish()
|
|
||||||
}
|
|
||||||
|
|
||||||
// send delete msg to DeleteNode
|
// send delete msg to DeleteNode
|
||||||
return []Msg{&res}
|
return []Msg{&res}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ibNode *insertBufferNode) verifyInMsg(in []Msg) (*flowGraphMsg, bool) {
|
|
||||||
// while closing
|
|
||||||
if in == nil {
|
|
||||||
log.Warn("type assertion failed for flowGraphMsg because it's nil")
|
|
||||||
return nil, false
|
|
||||||
}
|
|
||||||
|
|
||||||
if len(in) != 1 {
|
|
||||||
log.Warn("Invalid operate message input in insertBufferNode", zap.Int("input length", len(in)))
|
|
||||||
return nil, false
|
|
||||||
}
|
|
||||||
|
|
||||||
fgMsg, ok := in[0].(*flowGraphMsg)
|
|
||||||
if !ok {
|
|
||||||
log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
|
||||||
}
|
|
||||||
return fgMsg, ok
|
|
||||||
}
|
|
||||||
|
|
||||||
func (ibNode *insertBufferNode) GetBufferIfFull(segID UniqueID) (*BufferData, bool) {
|
func (ibNode *insertBufferNode) GetBufferIfFull(segID UniqueID) (*BufferData, bool) {
|
||||||
if bd, ok := ibNode.channel.getCurInsertBuffer(segID); ok && bd.effectiveCap() <= 0 {
|
if bd, ok := ibNode.channel.getCurInsertBuffer(segID); ok && bd.effectiveCap() <= 0 {
|
||||||
return bd, true
|
return bd, true
|
||||||
@ -301,7 +311,6 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload
|
|||||||
|
|
||||||
for _, segID := range segmentIDs {
|
for _, segID := range segmentIDs {
|
||||||
buf := ibNode.GetBuffer(segID)
|
buf := ibNode.GetBuffer(segID)
|
||||||
|
|
||||||
syncTasks[segID] = &syncTask{
|
syncTasks[segID] = &syncTask{
|
||||||
buffer: buf, // nil is valid
|
buffer: buf, // nil is valid
|
||||||
segmentID: segID,
|
segmentID: segID,
|
||||||
@ -312,6 +321,32 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload
|
|||||||
return syncTasks
|
return syncTasks
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if fgMsg.IsCloseMsg() {
|
||||||
|
// All segments in the collection will be synced, not matter empty buffer or not
|
||||||
|
segmentIDs := ibNode.channel.listAllSegmentIDs()
|
||||||
|
log.Info("Receive close request and syncing all segments",
|
||||||
|
zap.Int64s("segments", segmentIDs),
|
||||||
|
zap.String("channel", ibNode.channelName),
|
||||||
|
)
|
||||||
|
|
||||||
|
for _, segID := range segmentIDs {
|
||||||
|
// if segment has data or delete then force sync
|
||||||
|
insertBuf, hasInsert := ibNode.channel.getCurInsertBuffer(segID)
|
||||||
|
deleteEntry := ibNode.delBufferManager.GetEntriesNum(segID)
|
||||||
|
// if insert buf or or delete buf is not empty, trigger sync
|
||||||
|
if (hasInsert && insertBuf.size > 0) || (deleteEntry > 0) {
|
||||||
|
syncTasks[segID] = &syncTask{
|
||||||
|
buffer: insertBuf, // nil is valid
|
||||||
|
segmentID: segID,
|
||||||
|
flushed: false,
|
||||||
|
dropped: false,
|
||||||
|
auto: true,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return syncTasks
|
||||||
|
}
|
||||||
|
|
||||||
// Auto Sync // TODO: move to segment_sync_policy
|
// Auto Sync // TODO: move to segment_sync_policy
|
||||||
for _, segID := range seg2Upload {
|
for _, segID := range seg2Upload {
|
||||||
if ibuffer, ok := ibNode.GetBufferIfFull(segID); ok {
|
if ibuffer, ok := ibNode.GetBufferIfFull(segID); ok {
|
||||||
@ -353,7 +388,7 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
if len(syncSegmentIDs) > 0 {
|
if len(syncSegmentIDs) > 0 {
|
||||||
log.Debug("sync segments", zap.String("vChannel", ibNode.channelName),
|
log.Info("sync segments", zap.String("vChannel", ibNode.channelName),
|
||||||
zap.Int64s("segIDs", syncSegmentIDs)) // TODO: maybe too many prints here
|
zap.Int64s("segIDs", syncSegmentIDs)) // TODO: maybe too many prints here
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -418,6 +453,7 @@ func (ibNode *insertBufferNode) Sync(fgMsg *flowGraphMsg, seg2Upload []UniqueID,
|
|||||||
)
|
)
|
||||||
// use the flushed pk stats to take current stat
|
// use the flushed pk stats to take current stat
|
||||||
var pkStats []*storage.PrimaryKeyStats
|
var pkStats []*storage.PrimaryKeyStats
|
||||||
|
// TODO, this has to be async flush, no need to block here.
|
||||||
err := retry.Do(ibNode.ctx, func() error {
|
err := retry.Do(ibNode.ctx, func() error {
|
||||||
statBlobs, err := ibNode.flushManager.flushBufferData(task.buffer,
|
statBlobs, err := ibNode.flushManager.flushBufferData(task.buffer,
|
||||||
task.segmentID,
|
task.segmentID,
|
||||||
|
|||||||
@ -129,12 +129,18 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
|
|||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
type mockMsg struct{}
|
type mockMsg struct {
|
||||||
|
BaseMsg
|
||||||
|
}
|
||||||
|
|
||||||
func (*mockMsg) TimeTick() Timestamp {
|
func (*mockMsg) TimeTick() Timestamp {
|
||||||
return 0
|
return 0
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (*mockMsg) IsClose() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
||||||
t.Run("Test iBNode Operate invalid Msg", func(te *testing.T) {
|
t.Run("Test iBNode Operate invalid Msg", func(te *testing.T) {
|
||||||
invalidInTests := []struct {
|
invalidInTests := []struct {
|
||||||
@ -154,8 +160,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||||||
ibn := &insertBufferNode{
|
ibn := &insertBufferNode{
|
||||||
ttMerger: newMergedTimeTickerSender(func(Timestamp, []int64) error { return nil }),
|
ttMerger: newMergedTimeTickerSender(func(Timestamp, []int64) error { return nil }),
|
||||||
}
|
}
|
||||||
rt := ibn.Operate(test.in)
|
assert.False(t0, ibn.IsValidInMsg(test.in))
|
||||||
assert.Empty(t0, rt)
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
@ -711,16 +716,15 @@ func (s *InsertBufferNodeSuite) SetupSuite() {
|
|||||||
pkType: schemapb.DataType_Int64,
|
pkType: schemapb.DataType_Int64,
|
||||||
}
|
}
|
||||||
|
|
||||||
delBufManager := &DelBufferManager{
|
s.collID = 1
|
||||||
|
s.partID = 10
|
||||||
|
s.channel = newChannel("channel", s.collID, nil, rc, s.cm)
|
||||||
|
|
||||||
|
s.delBufManager = &DelBufferManager{
|
||||||
channel: s.channel,
|
channel: s.channel,
|
||||||
delMemorySize: 0,
|
delMemorySize: 0,
|
||||||
delBufHeap: &PriorityQueue{},
|
delBufHeap: &PriorityQueue{},
|
||||||
}
|
}
|
||||||
|
|
||||||
s.collID = 1
|
|
||||||
s.partID = 10
|
|
||||||
s.channel = newChannel("channel", s.collID, nil, rc, s.cm)
|
|
||||||
s.delBufManager = delBufManager
|
|
||||||
s.cm = storage.NewLocalChunkManager(storage.RootPath(insertBufferNodeTestDir))
|
s.cm = storage.NewLocalChunkManager(storage.RootPath(insertBufferNodeTestDir))
|
||||||
|
|
||||||
s.originalConfig = Params.DataNodeCfg.FlushInsertBufferSize.GetAsInt64()
|
s.originalConfig = Params.DataNodeCfg.FlushInsertBufferSize.GetAsInt64()
|
||||||
@ -886,6 +890,27 @@ func (s *InsertBufferNodeSuite) TestFillInSyncTasks() {
|
|||||||
s.Assert().False(task.dropped)
|
s.Assert().False(task.dropped)
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
|
s.Run("test close", func() {
|
||||||
|
fgMsg := &flowGraphMsg{BaseMsg: flowgraph.NewBaseMsg(true)}
|
||||||
|
|
||||||
|
node := &insertBufferNode{
|
||||||
|
channelName: s.channel.channelName,
|
||||||
|
channel: s.channel,
|
||||||
|
delBufferManager: s.delBufManager,
|
||||||
|
flushChan: make(chan flushMsg, 100),
|
||||||
|
}
|
||||||
|
|
||||||
|
syncTasks := node.FillInSyncTasks(fgMsg, nil)
|
||||||
|
s.Assert().Equal(1, len(syncTasks))
|
||||||
|
for _, task := range syncTasks {
|
||||||
|
s.Assert().Equal(task.segmentID, int64(1))
|
||||||
|
s.Assert().False(task.dropped)
|
||||||
|
s.Assert().False(task.flushed)
|
||||||
|
s.Assert().True(task.auto)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestInsertBufferNodeSuite(t *testing.T) {
|
func TestInsertBufferNodeSuite(t *testing.T) {
|
||||||
|
|||||||
@ -27,6 +27,8 @@ type (
|
|||||||
// Msg is flowgraph.Msg
|
// Msg is flowgraph.Msg
|
||||||
Msg = flowgraph.Msg
|
Msg = flowgraph.Msg
|
||||||
|
|
||||||
|
BaseMsg = flowgraph.BaseMsg
|
||||||
|
|
||||||
// MsgStreamMsg is flowgraph.MsgStreamMsg
|
// MsgStreamMsg is flowgraph.MsgStreamMsg
|
||||||
MsgStreamMsg = flowgraph.MsgStreamMsg
|
MsgStreamMsg = flowgraph.MsgStreamMsg
|
||||||
|
|
||||||
@ -41,6 +43,7 @@ type (
|
|||||||
)
|
)
|
||||||
|
|
||||||
type flowGraphMsg struct {
|
type flowGraphMsg struct {
|
||||||
|
BaseMsg
|
||||||
insertMessages []*msgstream.InsertMsg
|
insertMessages []*msgstream.InsertMsg
|
||||||
deleteMessages []*msgstream.DeleteMsg
|
deleteMessages []*msgstream.DeleteMsg
|
||||||
timeRange TimeRange
|
timeRange TimeRange
|
||||||
@ -56,12 +59,18 @@ func (fgMsg *flowGraphMsg) TimeTick() Timestamp {
|
|||||||
return fgMsg.timeRange.timestampMax
|
return fgMsg.timeRange.timestampMax
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (fgMsg *flowGraphMsg) IsClose() bool {
|
||||||
|
return fgMsg.BaseMsg.IsCloseMsg()
|
||||||
|
}
|
||||||
|
|
||||||
// flush Msg is used in flowgraph insertBufferNode to flush the given segment
|
// flush Msg is used in flowgraph insertBufferNode to flush the given segment
|
||||||
type flushMsg struct {
|
type flushMsg struct {
|
||||||
msgID UniqueID
|
msgID UniqueID
|
||||||
timestamp Timestamp
|
timestamp Timestamp
|
||||||
segmentID UniqueID
|
segmentID UniqueID
|
||||||
collectionID UniqueID
|
collectionID UniqueID
|
||||||
|
//isFlush illustrates if this is a flush or normal sync
|
||||||
|
isFlush bool
|
||||||
}
|
}
|
||||||
|
|
||||||
type resendTTMsg struct {
|
type resendTTMsg struct {
|
||||||
|
|||||||
@ -56,22 +56,23 @@ func (ttn *ttNode) Name() string {
|
|||||||
return fmt.Sprintf("ttNode-%s", ttn.vChannelName)
|
return fmt.Sprintf("ttNode-%s", ttn.vChannelName)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Operate handles input messages, implementing flowgraph.Node
|
func (ttn *ttNode) IsValidInMsg(in []Msg) bool {
|
||||||
func (ttn *ttNode) Operate(in []Msg) []Msg {
|
if !ttn.BaseNode.IsValidInMsg(in) {
|
||||||
if in == nil {
|
return false
|
||||||
log.Warn("type assertion failed for flowGraphMsg because it's nil")
|
|
||||||
return []Msg{}
|
|
||||||
}
|
}
|
||||||
|
_, ok := in[0].(*flowGraphMsg)
|
||||||
if len(in) != 1 {
|
|
||||||
log.Warn("Invalid operate message input in ttNode", zap.Int("input length", len(in)))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
fgMsg, ok := in[0].(*flowGraphMsg)
|
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
||||||
return []Msg{}
|
return false
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
// Operate handles input messages, implementing flowgraph.Node
|
||||||
|
func (ttn *ttNode) Operate(in []Msg) []Msg {
|
||||||
|
fgMsg := in[0].(*flowGraphMsg)
|
||||||
|
if fgMsg.IsCloseMsg() {
|
||||||
|
return in
|
||||||
}
|
}
|
||||||
|
|
||||||
curTs, _ := tsoutil.ParseTS(fgMsg.timeRange.timestampMax)
|
curTs, _ := tsoutil.ParseTS(fgMsg.timeRange.timestampMax)
|
||||||
|
|||||||
@ -312,6 +312,7 @@ func (m *rendezvousFlushManager) handleInsertTask(segmentID UniqueID, task flush
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (m *rendezvousFlushManager) handleDeleteTask(segmentID UniqueID, task flushDeleteTask, deltaLogs *DelDataBuf, pos *internalpb.MsgPosition) {
|
func (m *rendezvousFlushManager) handleDeleteTask(segmentID UniqueID, task flushDeleteTask, deltaLogs *DelDataBuf, pos *internalpb.MsgPosition) {
|
||||||
|
log.Info("handling delete task", zap.Int64("segment ID", segmentID))
|
||||||
// in dropping mode
|
// in dropping mode
|
||||||
if m.dropping.Load() {
|
if m.dropping.Load() {
|
||||||
// preventing separate delete, check position exists in queue first
|
// preventing separate delete, check position exists in queue first
|
||||||
@ -567,6 +568,7 @@ func (m *rendezvousFlushManager) close() {
|
|||||||
queue.injectMut.Unlock()
|
queue.injectMut.Unlock()
|
||||||
return true
|
return true
|
||||||
})
|
})
|
||||||
|
m.waitForAllFlushQueue()
|
||||||
}
|
}
|
||||||
|
|
||||||
type flushBufferInsertTask struct {
|
type flushBufferInsertTask struct {
|
||||||
@ -788,6 +790,7 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet
|
|||||||
zap.Int64("SegmentID", pack.segmentID),
|
zap.Int64("SegmentID", pack.segmentID),
|
||||||
zap.Int64("CollectionID", dsService.collectionID),
|
zap.Int64("CollectionID", dsService.collectionID),
|
||||||
zap.Any("startPos", startPos),
|
zap.Any("startPos", startPos),
|
||||||
|
zap.Any("checkPoints", checkPoints),
|
||||||
zap.Int("Length of Field2BinlogPaths", len(fieldInsert)),
|
zap.Int("Length of Field2BinlogPaths", len(fieldInsert)),
|
||||||
zap.Int("Length of Field2Stats", len(fieldStats)),
|
zap.Int("Length of Field2Stats", len(fieldStats)),
|
||||||
zap.Int("Length of Field2Deltalogs", len(deltaInfos[0].GetBinlogs())),
|
zap.Int("Length of Field2Deltalogs", len(deltaInfos[0].GetBinlogs())),
|
||||||
@ -816,7 +819,7 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet
|
|||||||
rsp, err := dsService.dataCoord.SaveBinlogPaths(context.Background(), req)
|
rsp, err := dsService.dataCoord.SaveBinlogPaths(context.Background(), req)
|
||||||
// should be network issue, return error and retry
|
// should be network issue, return error and retry
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return fmt.Errorf(err.Error())
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Segment not found during stale segment flush. Segment might get compacted already.
|
// Segment not found during stale segment flush. Segment might get compacted already.
|
||||||
@ -854,6 +857,10 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet
|
|||||||
if pack.flushed || pack.dropped {
|
if pack.flushed || pack.dropped {
|
||||||
dsService.channel.segmentFlushed(pack.segmentID)
|
dsService.channel.segmentFlushed(pack.segmentID)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if dsService.flushListener != nil {
|
||||||
|
dsService.flushListener <- pack
|
||||||
|
}
|
||||||
dsService.flushingSegCache.Remove(req.GetSegmentID())
|
dsService.flushingSegCache.Remove(req.GetSegmentID())
|
||||||
dsService.channel.evictHistoryInsertBuffer(req.GetSegmentID(), pack.pos)
|
dsService.channel.evictHistoryInsertBuffer(req.GetSegmentID(), pack.pos)
|
||||||
dsService.channel.evictHistoryDeleteBuffer(req.GetSegmentID(), pack.pos)
|
dsService.channel.evictHistoryDeleteBuffer(req.GetSegmentID(), pack.pos)
|
||||||
|
|||||||
@ -218,7 +218,14 @@ func (t *flushTaskRunner) getFlushPack() *segmentFlushPack {
|
|||||||
dropped: t.dropped,
|
dropped: t.dropped,
|
||||||
}
|
}
|
||||||
log.Debug("flush pack composed",
|
log.Debug("flush pack composed",
|
||||||
zap.Any("pack", pack))
|
zap.Int64("segmentID", t.segmentID),
|
||||||
|
zap.Int("insertLogs", len(t.insertLogs)),
|
||||||
|
zap.Int("statsLogs", len(t.statsLogs)),
|
||||||
|
zap.Int("deleteLogs", len(t.deltaLogs)),
|
||||||
|
zap.Bool("flushed", t.flushed),
|
||||||
|
zap.Bool("dropped", t.dropped),
|
||||||
|
)
|
||||||
|
|
||||||
if t.insertErr != nil || t.deleteErr != nil {
|
if t.insertErr != nil || t.deleteErr != nil {
|
||||||
log.Warn("flush task error detected", zap.Error(t.insertErr), zap.Error(t.deleteErr))
|
log.Warn("flush task error detected", zap.Error(t.insertErr), zap.Error(t.deleteErr))
|
||||||
pack.err = errors.New("execution failed")
|
pack.err = errors.New("execution failed")
|
||||||
|
|||||||
@ -207,6 +207,7 @@ type DataCoordFactory struct {
|
|||||||
|
|
||||||
GetSegmentInfosError bool
|
GetSegmentInfosError bool
|
||||||
GetSegmentInfosNotSuccess bool
|
GetSegmentInfosNotSuccess bool
|
||||||
|
UserSegmentInfo map[int64]*datapb.SegmentInfo
|
||||||
|
|
||||||
AddSegmentError bool
|
AddSegmentError bool
|
||||||
AddSegmentNotSuccess bool
|
AddSegmentNotSuccess bool
|
||||||
@ -310,7 +311,9 @@ func (ds *DataCoordFactory) GetSegmentInfo(ctx context.Context, req *datapb.GetS
|
|||||||
}
|
}
|
||||||
var segmentInfos []*datapb.SegmentInfo
|
var segmentInfos []*datapb.SegmentInfo
|
||||||
for _, segmentID := range req.SegmentIDs {
|
for _, segmentID := range req.SegmentIDs {
|
||||||
if segInfo, ok := segID2SegInfo[segmentID]; ok {
|
if segInfo, ok := ds.UserSegmentInfo[segmentID]; ok {
|
||||||
|
segmentInfos = append(segmentInfos, segInfo)
|
||||||
|
} else if segInfo, ok := segID2SegInfo[segmentID]; ok {
|
||||||
segmentInfos = append(segmentInfos, segInfo)
|
segmentInfos = append(segmentInfos, segInfo)
|
||||||
} else {
|
} else {
|
||||||
segmentInfos = append(segmentInfos, &datapb.SegmentInfo{
|
segmentInfos = append(segmentInfos, &datapb.SegmentInfo{
|
||||||
@ -780,9 +783,39 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int, chanName string) *msgstrea
|
|||||||
return msg
|
return msg
|
||||||
}
|
}
|
||||||
|
|
||||||
func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int, chanName string) (inMsgs []msgstream.TsMsg) {
|
func (df *DataFactory) GenMsgStreamInsertMsgWithTs(idx int, chanName string, ts Timestamp) *msgstream.InsertMsg {
|
||||||
|
var msg = &msgstream.InsertMsg{
|
||||||
|
BaseMsg: msgstream.BaseMsg{
|
||||||
|
HashValues: []uint32{uint32(idx)},
|
||||||
|
BeginTimestamp: ts,
|
||||||
|
EndTimestamp: ts,
|
||||||
|
},
|
||||||
|
InsertRequest: internalpb.InsertRequest{
|
||||||
|
Base: &commonpb.MsgBase{
|
||||||
|
MsgType: commonpb.MsgType_Insert,
|
||||||
|
MsgID: 0,
|
||||||
|
Timestamp: ts,
|
||||||
|
SourceID: 0,
|
||||||
|
},
|
||||||
|
CollectionName: "col1",
|
||||||
|
PartitionName: "default",
|
||||||
|
SegmentID: 1,
|
||||||
|
CollectionID: UniqueID(0),
|
||||||
|
ShardName: chanName,
|
||||||
|
Timestamps: []Timestamp{ts},
|
||||||
|
RowIDs: []UniqueID{UniqueID(idx)},
|
||||||
|
// RowData: []*commonpb.Blob{{Value: df.rawData}},
|
||||||
|
FieldsData: df.columnData,
|
||||||
|
Version: internalpb.InsertDataVersion_ColumnBased,
|
||||||
|
NumRows: 1,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
return msg
|
||||||
|
}
|
||||||
|
|
||||||
|
func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int, chanName string, ts Timestamp) (inMsgs []msgstream.TsMsg) {
|
||||||
for i := 0; i < n; i++ {
|
for i := 0; i < n; i++ {
|
||||||
var msg = df.GenMsgStreamInsertMsg(i, chanName)
|
var msg = df.GenMsgStreamInsertMsgWithTs(i, chanName, ts)
|
||||||
var tsMsg msgstream.TsMsg = msg
|
var tsMsg msgstream.TsMsg = msg
|
||||||
inMsgs = append(inMsgs, tsMsg)
|
inMsgs = append(inMsgs, tsMsg)
|
||||||
}
|
}
|
||||||
@ -816,6 +849,7 @@ func (df *DataFactory) GenMsgStreamDeleteMsg(pks []primaryKey, chanName string)
|
|||||||
},
|
},
|
||||||
CollectionName: "col1",
|
CollectionName: "col1",
|
||||||
PartitionName: "default",
|
PartitionName: "default",
|
||||||
|
PartitionID: 1,
|
||||||
ShardName: chanName,
|
ShardName: chanName,
|
||||||
PrimaryKeys: s.ParsePrimaryKeys2IDs(pks),
|
PrimaryKeys: s.ParsePrimaryKeys2IDs(pks),
|
||||||
Timestamps: timestamps,
|
Timestamps: timestamps,
|
||||||
@ -825,6 +859,33 @@ func (df *DataFactory) GenMsgStreamDeleteMsg(pks []primaryKey, chanName string)
|
|||||||
return msg
|
return msg
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (df *DataFactory) GenMsgStreamDeleteMsgWithTs(idx int, pks []primaryKey, chanName string, ts Timestamp) *msgstream.DeleteMsg {
|
||||||
|
var msg = &msgstream.DeleteMsg{
|
||||||
|
BaseMsg: msgstream.BaseMsg{
|
||||||
|
HashValues: []uint32{uint32(idx)},
|
||||||
|
BeginTimestamp: ts,
|
||||||
|
EndTimestamp: ts,
|
||||||
|
},
|
||||||
|
DeleteRequest: internalpb.DeleteRequest{
|
||||||
|
Base: &commonpb.MsgBase{
|
||||||
|
MsgType: commonpb.MsgType_Delete,
|
||||||
|
MsgID: 1,
|
||||||
|
Timestamp: ts,
|
||||||
|
SourceID: 0,
|
||||||
|
},
|
||||||
|
CollectionName: "col1",
|
||||||
|
PartitionName: "default",
|
||||||
|
PartitionID: 1,
|
||||||
|
CollectionID: UniqueID(0),
|
||||||
|
ShardName: chanName,
|
||||||
|
PrimaryKeys: s.ParsePrimaryKeys2IDs(pks),
|
||||||
|
Timestamps: []Timestamp{ts},
|
||||||
|
NumRows: int64(len(pks)),
|
||||||
|
},
|
||||||
|
}
|
||||||
|
return msg
|
||||||
|
}
|
||||||
|
|
||||||
func genFlowGraphInsertMsg(chanName string) flowGraphMsg {
|
func genFlowGraphInsertMsg(chanName string) flowGraphMsg {
|
||||||
timeRange := TimeRange{
|
timeRange := TimeRange{
|
||||||
timestampMin: 0,
|
timestampMin: 0,
|
||||||
|
|||||||
@ -19,7 +19,9 @@ package datanode
|
|||||||
import (
|
import (
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
// segmentSyncPolicy sync policy applies to segment
|
// segmentSyncPolicy sync policy applies to segment
|
||||||
@ -30,7 +32,10 @@ func syncPeriodically() segmentSyncPolicy {
|
|||||||
return func(segment *Segment, ts Timestamp) bool {
|
return func(segment *Segment, ts Timestamp) bool {
|
||||||
endTime := tsoutil.PhysicalTime(ts)
|
endTime := tsoutil.PhysicalTime(ts)
|
||||||
lastSyncTime := tsoutil.PhysicalTime(segment.lastSyncTs)
|
lastSyncTime := tsoutil.PhysicalTime(segment.lastSyncTs)
|
||||||
return endTime.Sub(lastSyncTime) >= Params.DataNodeCfg.SyncPeriod.GetAsDuration(time.Second) &&
|
shouldSync := endTime.Sub(lastSyncTime) >= Params.DataNodeCfg.SyncPeriod.GetAsDuration(time.Second) && !segment.isBufferEmpty()
|
||||||
!segment.isBufferEmpty()
|
if shouldSync {
|
||||||
|
log.Info("sync segment periodically ", zap.Time("now", endTime), zap.Time("last sync", lastSyncTime))
|
||||||
|
}
|
||||||
|
return shouldSync
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -53,30 +53,26 @@ func (dNode *deleteNode) Name() string {
|
|||||||
return fmt.Sprintf("dNode-%s", dNode.deltaVchannel)
|
return fmt.Sprintf("dNode-%s", dNode.deltaVchannel)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (dNode *deleteNode) IsValidInMsg(in []Msg) bool {
|
||||||
|
if !dNode.baseNode.IsValidInMsg(in) {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
_, ok := in[0].(*deleteMsg)
|
||||||
|
if !ok {
|
||||||
|
log.Warn("type assertion failed for deleteMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", dNode.Name()))
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
// Operate handles input messages, do delete operations
|
// Operate handles input messages, do delete operations
|
||||||
func (dNode *deleteNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
func (dNode *deleteNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
||||||
if in == nil {
|
|
||||||
log.Debug("type assertion failed for deleteMsg because it's nil", zap.String("name", dNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
if len(in) != 1 {
|
|
||||||
log.Warn("Invalid operate message input in deleteNode", zap.Int("input length", len(in)), zap.String("name", dNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
dMsg, ok := in[0].(*deleteMsg)
|
dMsg, ok := in[0].(*deleteMsg)
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("type assertion failed for deleteMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", dNode.Name()))
|
log.Warn("type assertion failed for deleteMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", dNode.Name()))
|
||||||
return []Msg{}
|
return []Msg{}
|
||||||
}
|
}
|
||||||
|
|
||||||
delData := &deleteData{
|
|
||||||
deleteIDs: map[UniqueID][]primaryKey{},
|
|
||||||
deleteTimestamps: map[UniqueID][]Timestamp{},
|
|
||||||
deleteOffset: map[UniqueID]int64{},
|
|
||||||
}
|
|
||||||
|
|
||||||
var spans []opentracing.Span
|
var spans []opentracing.Span
|
||||||
for _, msg := range dMsg.deleteMessages {
|
for _, msg := range dMsg.deleteMessages {
|
||||||
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
||||||
@ -84,6 +80,24 @@ func (dNode *deleteNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||||||
msg.SetTraceCtx(ctx)
|
msg.SetTraceCtx(ctx)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
defer func() {
|
||||||
|
for _, sp := range spans {
|
||||||
|
sp.Finish()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
if dMsg.IsCloseMsg() {
|
||||||
|
return []Msg{
|
||||||
|
&serviceTimeMsg{BaseMsg: flowgraph.NewBaseMsg(true)},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
delData := &deleteData{
|
||||||
|
deleteIDs: map[UniqueID][]primaryKey{},
|
||||||
|
deleteTimestamps: map[UniqueID][]Timestamp{},
|
||||||
|
deleteOffset: map[UniqueID]int64{},
|
||||||
|
}
|
||||||
|
|
||||||
// 1. filter segment by bloom filter
|
// 1. filter segment by bloom filter
|
||||||
for i, delMsg := range dMsg.deleteMessages {
|
for i, delMsg := range dMsg.deleteMessages {
|
||||||
traceID, _, _ := trace.InfoFromSpan(spans[i])
|
traceID, _, _ := trace.InfoFromSpan(spans[i])
|
||||||
@ -154,9 +168,6 @@ func (dNode *deleteNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||||||
var res Msg = &serviceTimeMsg{
|
var res Msg = &serviceTimeMsg{
|
||||||
timeRange: dMsg.timeRange,
|
timeRange: dMsg.timeRange,
|
||||||
}
|
}
|
||||||
for _, sp := range spans {
|
|
||||||
sp.Finish()
|
|
||||||
}
|
|
||||||
|
|
||||||
return []Msg{res}
|
return []Msg{res}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -43,23 +43,21 @@ func (fddNode *filterDeleteNode) Name() string {
|
|||||||
return fmt.Sprintf("fdNode-%s", fddNode.vchannel)
|
return fmt.Sprintf("fdNode-%s", fddNode.vchannel)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Operate handles input messages, to filter invalid delete messages
|
func (fddNode *filterDeleteNode) IsValidInMsg(in []Msg) bool {
|
||||||
func (fddNode *filterDeleteNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
if !fddNode.baseNode.IsValidInMsg(in) {
|
||||||
if in == nil {
|
return false
|
||||||
log.Debug("type assertion failed for MsgStreamMsg because it's nil", zap.String("name", fddNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
}
|
||||||
|
_, ok := in[0].(*MsgStreamMsg)
|
||||||
if len(in) != 1 {
|
|
||||||
log.Warn("Invalid operate message input in filterDDNode", zap.Int("input length", len(in)), zap.String("name", fddNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
msgStreamMsg, ok := in[0].(*MsgStreamMsg)
|
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("type assertion failed for MsgStreamMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", fddNode.Name()))
|
log.Warn("type assertion failed for MsgStreamMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", fddNode.Name()))
|
||||||
return []Msg{}
|
return false
|
||||||
}
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
// Operate handles input messages, to filter invalid delete messages
|
||||||
|
func (fddNode *filterDeleteNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
||||||
|
msgStreamMsg := in[0].(*MsgStreamMsg)
|
||||||
|
|
||||||
var spans []opentracing.Span
|
var spans []opentracing.Span
|
||||||
for _, msg := range msgStreamMsg.TsMessages() {
|
for _, msg := range msgStreamMsg.TsMessages() {
|
||||||
@ -68,6 +66,18 @@ func (fddNode *filterDeleteNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||||||
msg.SetTraceCtx(ctx)
|
msg.SetTraceCtx(ctx)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
defer func() {
|
||||||
|
for _, sp := range spans {
|
||||||
|
sp.Finish()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
if msgStreamMsg.IsCloseMsg() {
|
||||||
|
return []Msg{
|
||||||
|
&deleteMsg{BaseMsg: flowgraph.NewBaseMsg(true)},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
var dMsg = deleteMsg{
|
var dMsg = deleteMsg{
|
||||||
deleteMessages: make([]*msgstream.DeleteMsg, 0),
|
deleteMessages: make([]*msgstream.DeleteMsg, 0),
|
||||||
timeRange: TimeRange{
|
timeRange: TimeRange{
|
||||||
@ -102,11 +112,8 @@ func (fddNode *filterDeleteNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||||||
zap.String("vchannel", fddNode.vchannel))
|
zap.String("vchannel", fddNode.vchannel))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
var res Msg = &dMsg
|
|
||||||
for _, sp := range spans {
|
return []Msg{&dMsg}
|
||||||
sp.Finish()
|
|
||||||
}
|
|
||||||
return []Msg{res}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// filterInvalidDeleteMessage would filter invalid delete messages
|
// filterInvalidDeleteMessage would filter invalid delete messages
|
||||||
@ -142,7 +149,6 @@ func (fddNode *filterDeleteNode) filterInvalidDeleteMessage(msg *msgstream.Delet
|
|||||||
|
|
||||||
// newFilteredDeleteNode returns a new filterDeleteNode
|
// newFilteredDeleteNode returns a new filterDeleteNode
|
||||||
func newFilteredDeleteNode(metaReplica ReplicaInterface, collectionID UniqueID, vchannel Channel) *filterDeleteNode {
|
func newFilteredDeleteNode(metaReplica ReplicaInterface, collectionID UniqueID, vchannel Channel) *filterDeleteNode {
|
||||||
|
|
||||||
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()
|
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()
|
||||||
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism.GetAsInt32()
|
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism.GetAsInt32()
|
||||||
|
|
||||||
|
|||||||
@ -48,22 +48,21 @@ func (fdmNode *filterDmNode) Name() string {
|
|||||||
return fmt.Sprintf("fdmNode-%s", fdmNode.vchannel)
|
return fmt.Sprintf("fdmNode-%s", fdmNode.vchannel)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Operate handles input messages, to filter invalid insert messages
|
func (fdmNode *filterDmNode) IsValidInMsg(in []Msg) bool {
|
||||||
func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
if !fdmNode.baseNode.IsValidInMsg(in) {
|
||||||
if in == nil {
|
return false
|
||||||
log.Debug("type assertion failed for MsgStreamMsg because it's nil", zap.String("name", fdmNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
}
|
||||||
if len(in) != 1 {
|
_, ok := in[0].(*MsgStreamMsg)
|
||||||
log.Warn("Invalid operate message input in filterDmNode", zap.Int("input length", len(in)), zap.String("name", fdmNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
msgStreamMsg, ok := in[0].(*MsgStreamMsg)
|
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("type assertion failed for MsgStreamMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", fdmNode.Name()))
|
log.Warn("type assertion failed for MsgStreamMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", fdmNode.Name()))
|
||||||
return []Msg{}
|
return false
|
||||||
}
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
// Operate handles input messages, to filter invalid insert messages
|
||||||
|
func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
||||||
|
msgStreamMsg := in[0].(*MsgStreamMsg)
|
||||||
|
|
||||||
var spans []opentracing.Span
|
var spans []opentracing.Span
|
||||||
for _, msg := range msgStreamMsg.TsMessages() {
|
for _, msg := range msgStreamMsg.TsMessages() {
|
||||||
@ -71,6 +70,17 @@ func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||||||
spans = append(spans, sp)
|
spans = append(spans, sp)
|
||||||
msg.SetTraceCtx(ctx)
|
msg.SetTraceCtx(ctx)
|
||||||
}
|
}
|
||||||
|
defer func() {
|
||||||
|
for _, sp := range spans {
|
||||||
|
sp.Finish()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
if msgStreamMsg.IsCloseMsg() {
|
||||||
|
return []Msg{
|
||||||
|
&insertMsg{BaseMsg: flowgraph.NewBaseMsg(true)},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
var iMsg = insertMsg{
|
var iMsg = insertMsg{
|
||||||
insertMessages: make([]*msgstream.InsertMsg, 0),
|
insertMessages: make([]*msgstream.InsertMsg, 0),
|
||||||
@ -125,11 +135,7 @@ func (fdmNode *filterDmNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
var res Msg = &iMsg
|
return []Msg{&iMsg}
|
||||||
for _, sp := range spans {
|
|
||||||
sp.Finish()
|
|
||||||
}
|
|
||||||
return []Msg{res}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// filterInvalidDeleteMessage would filter out invalid delete messages
|
// filterInvalidDeleteMessage would filter out invalid delete messages
|
||||||
@ -231,7 +237,6 @@ func (fdmNode *filterDmNode) filterInvalidInsertMessage(msg *msgstream.InsertMsg
|
|||||||
|
|
||||||
// newFilteredDmNode returns a new filterDmNode
|
// newFilteredDmNode returns a new filterDmNode
|
||||||
func newFilteredDmNode(metaReplica ReplicaInterface, collectionID UniqueID, vchannel Channel) *filterDmNode {
|
func newFilteredDmNode(metaReplica ReplicaInterface, collectionID UniqueID, vchannel Channel) *filterDmNode {
|
||||||
|
|
||||||
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()
|
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()
|
||||||
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism.GetAsInt32()
|
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism.GetAsInt32()
|
||||||
|
|
||||||
|
|||||||
@ -71,22 +71,38 @@ func (iNode *insertNode) Name() string {
|
|||||||
return fmt.Sprintf("iNode-%s", iNode.vchannel)
|
return fmt.Sprintf("iNode-%s", iNode.vchannel)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Operate handles input messages, to execute insert operations
|
func (iNode *insertNode) IsValidInMsg(in []Msg) bool {
|
||||||
func (iNode *insertNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
if !iNode.baseNode.IsValidInMsg(in) {
|
||||||
if in == nil {
|
return false
|
||||||
log.Debug("type assertion failed for insertMsg because it's nil", zap.String("name", iNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
}
|
||||||
|
_, ok := in[0].(*insertMsg)
|
||||||
if len(in) != 1 {
|
|
||||||
log.Warn("Invalid operate message input in insertNode", zap.Int("input length", len(in)), zap.String("name", iNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
iMsg, ok := in[0].(*insertMsg)
|
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("type assertion failed for insertMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", iNode.Name()))
|
log.Warn("type assertion failed for insertMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", iNode.Name()))
|
||||||
return []Msg{}
|
return false
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
// Operate handles input messages, to execute insert operations
|
||||||
|
func (iNode *insertNode) Operate(in []Msg) []Msg {
|
||||||
|
iMsg := in[0].(*insertMsg)
|
||||||
|
|
||||||
|
var spans []opentracing.Span
|
||||||
|
for _, msg := range iMsg.insertMessages {
|
||||||
|
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
||||||
|
spans = append(spans, sp)
|
||||||
|
msg.SetTraceCtx(ctx)
|
||||||
|
}
|
||||||
|
defer func() {
|
||||||
|
for _, sp := range spans {
|
||||||
|
sp.Finish()
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
if iMsg.IsCloseMsg() {
|
||||||
|
return []Msg{
|
||||||
|
&serviceTimeMsg{BaseMsg: flowgraph.NewBaseMsg(true)},
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
iData := insertData{
|
iData := insertData{
|
||||||
@ -97,13 +113,6 @@ func (iNode *insertNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||||||
insertPKs: make(map[UniqueID][]primaryKey),
|
insertPKs: make(map[UniqueID][]primaryKey),
|
||||||
}
|
}
|
||||||
|
|
||||||
var spans []opentracing.Span
|
|
||||||
for _, msg := range iMsg.insertMessages {
|
|
||||||
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
|
||||||
spans = append(spans, sp)
|
|
||||||
msg.SetTraceCtx(ctx)
|
|
||||||
}
|
|
||||||
|
|
||||||
collection, err := iNode.metaReplica.getCollectionByID(iNode.collectionID)
|
collection, err := iNode.metaReplica.getCollectionByID(iNode.collectionID)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// QueryNode should add collection before start flow graph
|
// QueryNode should add collection before start flow graph
|
||||||
@ -295,9 +304,6 @@ func (iNode *insertNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||||||
var res Msg = &serviceTimeMsg{
|
var res Msg = &serviceTimeMsg{
|
||||||
timeRange: iMsg.timeRange,
|
timeRange: iMsg.timeRange,
|
||||||
}
|
}
|
||||||
for _, sp := range spans {
|
|
||||||
sp.Finish()
|
|
||||||
}
|
|
||||||
|
|
||||||
return []Msg{res}
|
return []Msg{res}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -23,12 +23,14 @@ import (
|
|||||||
|
|
||||||
// Msg is an interface which has a function named TimeTick
|
// Msg is an interface which has a function named TimeTick
|
||||||
type Msg = flowgraph.Msg
|
type Msg = flowgraph.Msg
|
||||||
|
type BaseMsg = flowgraph.BaseMsg
|
||||||
|
|
||||||
// MsgStreamMsg is an implementation of interface Msg
|
// MsgStreamMsg is an implementation of interface Msg
|
||||||
type MsgStreamMsg = flowgraph.MsgStreamMsg
|
type MsgStreamMsg = flowgraph.MsgStreamMsg
|
||||||
|
|
||||||
// insertMsg is an implementation of interface Msg
|
// insertMsg is an implementation of interface Msg
|
||||||
type insertMsg struct {
|
type insertMsg struct {
|
||||||
|
BaseMsg
|
||||||
insertMessages []*msgstream.InsertMsg
|
insertMessages []*msgstream.InsertMsg
|
||||||
deleteMessages []*msgstream.DeleteMsg
|
deleteMessages []*msgstream.DeleteMsg
|
||||||
timeRange TimeRange
|
timeRange TimeRange
|
||||||
@ -36,12 +38,14 @@ type insertMsg struct {
|
|||||||
|
|
||||||
// deleteMsg is an implementation of interface Msg
|
// deleteMsg is an implementation of interface Msg
|
||||||
type deleteMsg struct {
|
type deleteMsg struct {
|
||||||
|
BaseMsg
|
||||||
deleteMessages []*msgstream.DeleteMsg
|
deleteMessages []*msgstream.DeleteMsg
|
||||||
timeRange TimeRange
|
timeRange TimeRange
|
||||||
}
|
}
|
||||||
|
|
||||||
// serviceTimeMsg is an implementation of interface Msg
|
// serviceTimeMsg is an implementation of interface Msg
|
||||||
type serviceTimeMsg struct {
|
type serviceTimeMsg struct {
|
||||||
|
BaseMsg
|
||||||
timeRange TimeRange
|
timeRange TimeRange
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -50,12 +54,24 @@ func (iMsg *insertMsg) TimeTick() Timestamp {
|
|||||||
return iMsg.timeRange.timestampMax
|
return iMsg.timeRange.timestampMax
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (iMsg *insertMsg) IsClose() bool {
|
||||||
|
return iMsg.IsCloseMsg()
|
||||||
|
}
|
||||||
|
|
||||||
// TimeTick returns timestamp of deleteMsg
|
// TimeTick returns timestamp of deleteMsg
|
||||||
func (dMsg *deleteMsg) TimeTick() Timestamp {
|
func (dMsg *deleteMsg) TimeTick() Timestamp {
|
||||||
return dMsg.timeRange.timestampMax
|
return dMsg.timeRange.timestampMax
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (dMsg *deleteMsg) IsClose() bool {
|
||||||
|
return dMsg.IsCloseMsg()
|
||||||
|
}
|
||||||
|
|
||||||
// TimeTick returns timestamp of serviceTimeMsg
|
// TimeTick returns timestamp of serviceTimeMsg
|
||||||
func (stMsg *serviceTimeMsg) TimeTick() Timestamp {
|
func (stMsg *serviceTimeMsg) TimeTick() Timestamp {
|
||||||
return stMsg.timeRange.timestampMax
|
return stMsg.timeRange.timestampMax
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (stMsg *serviceTimeMsg) IsClose() bool {
|
||||||
|
return stMsg.IsCloseMsg()
|
||||||
|
}
|
||||||
|
|||||||
@ -41,22 +41,28 @@ func (stNode *serviceTimeNode) Name() string {
|
|||||||
return fmt.Sprintf("stNode-%s", stNode.vChannel)
|
return fmt.Sprintf("stNode-%s", stNode.vChannel)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Operate handles input messages, to execute insert operations
|
func (stNode *serviceTimeNode) IsValidInMsg(in []Msg) bool {
|
||||||
func (stNode *serviceTimeNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
if !stNode.baseNode.IsValidInMsg(in) {
|
||||||
if in == nil {
|
return false
|
||||||
log.Debug("type assertion failed for serviceTimeMsg because it's nil", zap.String("name", stNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
}
|
||||||
|
_, ok := in[0].(*serviceTimeMsg)
|
||||||
if len(in) != 1 {
|
|
||||||
log.Warn("Invalid operate message input in serviceTimeNode, input length = ", zap.Int("input node", len(in)), zap.String("name", stNode.Name()))
|
|
||||||
return []Msg{}
|
|
||||||
}
|
|
||||||
|
|
||||||
serviceTimeMsg, ok := in[0].(*serviceTimeMsg)
|
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("type assertion failed for serviceTimeMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", stNode.Name()))
|
log.Warn("type assertion failed for serviceTimeMsg", zap.String("msgType", reflect.TypeOf(in[0]).Name()), zap.String("name", stNode.Name()))
|
||||||
return []Msg{}
|
return false
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
// Operate handles input messages, to execute insert operations
|
||||||
|
func (stNode *serviceTimeNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
||||||
|
serviceTimeMsg := in[0].(*serviceTimeMsg)
|
||||||
|
if serviceTimeMsg.IsCloseMsg() {
|
||||||
|
log.Info("service node hit close msg",
|
||||||
|
zap.Int64("collectionID", stNode.collectionID),
|
||||||
|
zap.Uint64("tSafe", serviceTimeMsg.timeRange.timestampMax),
|
||||||
|
zap.String("channel", stNode.vChannel),
|
||||||
|
)
|
||||||
|
return in
|
||||||
}
|
}
|
||||||
|
|
||||||
// update service time
|
// update service time
|
||||||
|
|||||||
@ -58,12 +58,16 @@ func (m *numMsg) TimeTick() Timestamp {
|
|||||||
return Timestamp(0)
|
return Timestamp(0)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (m *numMsg) IsClose() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
func (n *nodeA) Name() string {
|
func (n *nodeA) Name() string {
|
||||||
return "NodeA"
|
return "NodeA"
|
||||||
}
|
}
|
||||||
|
|
||||||
func (n *nodeA) Operate(in []Msg) []Msg {
|
func (n *nodeA) Operate(in []Msg) []Msg {
|
||||||
// ignore `in` because nodeA doesn't have any upstream node.
|
// ignore `in` because nodeA doesn't have any upstream node.git s
|
||||||
a := <-n.inputChan
|
a := <-n.inputChan
|
||||||
var res Msg = &numMsg{
|
var res Msg = &numMsg{
|
||||||
num: a,
|
num: a,
|
||||||
|
|||||||
@ -37,6 +37,7 @@ import (
|
|||||||
type InputNode struct {
|
type InputNode struct {
|
||||||
BaseNode
|
BaseNode
|
||||||
inStream msgstream.MsgStream
|
inStream msgstream.MsgStream
|
||||||
|
lastMsg *msgstream.MsgPack
|
||||||
name string
|
name string
|
||||||
role string
|
role string
|
||||||
nodeID int64
|
nodeID int64
|
||||||
@ -61,6 +62,10 @@ func (inNode *InputNode) Close() {
|
|||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (inNode *InputNode) IsValidInMsg(in []Msg) bool {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
// Name returns node name
|
// Name returns node name
|
||||||
func (inNode *InputNode) Name() string {
|
func (inNode *InputNode) Name() string {
|
||||||
return inNode.name
|
return inNode.name
|
||||||
@ -76,8 +81,19 @@ func (inNode *InputNode) Operate(in []Msg) []Msg {
|
|||||||
msgPack, ok := <-inNode.inStream.Chan()
|
msgPack, ok := <-inNode.inStream.Chan()
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("MsgStream closed", zap.Any("input node", inNode.Name()))
|
log.Warn("MsgStream closed", zap.Any("input node", inNode.Name()))
|
||||||
|
if inNode.lastMsg != nil {
|
||||||
|
log.Info("trigger force sync", zap.Int64("collection", inNode.collectionID), zap.Any("position", inNode.lastMsg))
|
||||||
|
return []Msg{&MsgStreamMsg{
|
||||||
|
BaseMsg: NewBaseMsg(true),
|
||||||
|
tsMessages: []msgstream.TsMsg{},
|
||||||
|
timestampMin: inNode.lastMsg.BeginTs,
|
||||||
|
timestampMax: inNode.lastMsg.EndTs,
|
||||||
|
startPositions: inNode.lastMsg.StartPositions,
|
||||||
|
endPositions: inNode.lastMsg.EndPositions,
|
||||||
|
}}
|
||||||
|
}
|
||||||
return []Msg{&MsgStreamMsg{
|
return []Msg{&MsgStreamMsg{
|
||||||
isCloseMsg: true,
|
BaseMsg: NewBaseMsg(true),
|
||||||
}}
|
}}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -86,6 +102,7 @@ func (inNode *InputNode) Operate(in []Msg) []Msg {
|
|||||||
return []Msg{}
|
return []Msg{}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
inNode.lastMsg = msgPack
|
||||||
sub := tsoutil.SubByNow(msgPack.EndTs)
|
sub := tsoutil.SubByNow(msgPack.EndTs)
|
||||||
if inNode.role == typeutil.QueryNodeRole {
|
if inNode.role == typeutil.QueryNodeRole {
|
||||||
metrics.QueryNodeConsumerMsgCount.
|
metrics.QueryNodeConsumerMsgCount.
|
||||||
|
|||||||
@ -23,16 +23,31 @@ import (
|
|||||||
// Msg is an abstract class that contains a method to get the time tick of this message
|
// Msg is an abstract class that contains a method to get the time tick of this message
|
||||||
type Msg interface {
|
type Msg interface {
|
||||||
TimeTick() Timestamp
|
TimeTick() Timestamp
|
||||||
|
IsClose() bool
|
||||||
|
}
|
||||||
|
|
||||||
|
type BaseMsg struct {
|
||||||
|
isCloseMsg bool
|
||||||
|
}
|
||||||
|
|
||||||
|
func (msg BaseMsg) IsCloseMsg() bool {
|
||||||
|
return msg.isCloseMsg
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewBaseMsg(isCloseMsg bool) BaseMsg {
|
||||||
|
return BaseMsg{
|
||||||
|
isCloseMsg: isCloseMsg,
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// MsgStreamMsg is a wrapper of TsMsg in flowgraph
|
// MsgStreamMsg is a wrapper of TsMsg in flowgraph
|
||||||
type MsgStreamMsg struct {
|
type MsgStreamMsg struct {
|
||||||
|
BaseMsg
|
||||||
tsMessages []msgstream.TsMsg
|
tsMessages []msgstream.TsMsg
|
||||||
timestampMin Timestamp
|
timestampMin Timestamp
|
||||||
timestampMax Timestamp
|
timestampMax Timestamp
|
||||||
startPositions []*MsgPosition
|
startPositions []*MsgPosition
|
||||||
endPositions []*MsgPosition
|
endPositions []*MsgPosition
|
||||||
isCloseMsg bool
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// GenerateMsgStreamMsg is used to create a new MsgStreamMsg object
|
// GenerateMsgStreamMsg is used to create a new MsgStreamMsg object
|
||||||
@ -51,6 +66,10 @@ func (msMsg *MsgStreamMsg) TimeTick() Timestamp {
|
|||||||
return msMsg.timestampMax
|
return msMsg.timestampMax
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (msMsg *MsgStreamMsg) IsClose() bool {
|
||||||
|
return msMsg.isCloseMsg
|
||||||
|
}
|
||||||
|
|
||||||
// DownStreamNodeIdx returns 0
|
// DownStreamNodeIdx returns 0
|
||||||
func (msMsg *MsgStreamMsg) DownStreamNodeIdx() int {
|
func (msMsg *MsgStreamMsg) DownStreamNodeIdx() int {
|
||||||
return 0
|
return 0
|
||||||
|
|||||||
@ -38,6 +38,7 @@ type Node interface {
|
|||||||
Name() string
|
Name() string
|
||||||
MaxQueueLength() int32
|
MaxQueueLength() int32
|
||||||
MaxParallelism() int32
|
MaxParallelism() int32
|
||||||
|
IsValidInMsg(in []Msg) bool
|
||||||
Operate(in []Msg) []Msg
|
Operate(in []Msg) []Msg
|
||||||
IsInputNode() bool
|
IsInputNode() bool
|
||||||
Start()
|
Start()
|
||||||
@ -85,8 +86,7 @@ func (nodeCtx *nodeCtx) Unblock() {
|
|||||||
|
|
||||||
func isCloseMsg(msgs []Msg) bool {
|
func isCloseMsg(msgs []Msg) bool {
|
||||||
if len(msgs) == 1 {
|
if len(msgs) == 1 {
|
||||||
msg, ok := msgs[0].(*MsgStreamMsg)
|
return msgs[0].IsClose()
|
||||||
return ok && msg.isCloseMsg
|
|
||||||
}
|
}
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
@ -118,15 +118,14 @@ func (nodeCtx *nodeCtx) work() {
|
|||||||
input = <-nodeCtx.inputChannel
|
input = <-nodeCtx.inputChannel
|
||||||
}
|
}
|
||||||
// the input message decides whether the operate method is executed
|
// the input message decides whether the operate method is executed
|
||||||
if isCloseMsg(input) {
|
n := nodeCtx.node
|
||||||
output = input
|
nodeCtx.blockMutex.RLock()
|
||||||
}
|
if !n.IsValidInMsg(input) {
|
||||||
if len(output) == 0 {
|
|
||||||
n := nodeCtx.node
|
|
||||||
nodeCtx.blockMutex.RLock()
|
|
||||||
output = n.Operate(input)
|
|
||||||
nodeCtx.blockMutex.RUnlock()
|
nodeCtx.blockMutex.RUnlock()
|
||||||
|
continue
|
||||||
}
|
}
|
||||||
|
output = n.Operate(input)
|
||||||
|
nodeCtx.blockMutex.RUnlock()
|
||||||
// the output decide whether the node should be closed.
|
// the output decide whether the node should be closed.
|
||||||
if isCloseMsg(output) {
|
if isCloseMsg(output) {
|
||||||
close(nodeCtx.closeCh)
|
close(nodeCtx.closeCh)
|
||||||
@ -186,3 +185,24 @@ func (node *BaseNode) Start() {}
|
|||||||
|
|
||||||
// Close implementing Node, base node does nothing when stops
|
// Close implementing Node, base node does nothing when stops
|
||||||
func (node *BaseNode) Close() {}
|
func (node *BaseNode) Close() {}
|
||||||
|
|
||||||
|
func (node *BaseNode) Name() string {
|
||||||
|
return "BaseNode"
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *BaseNode) Operate(in []Msg) []Msg {
|
||||||
|
return in
|
||||||
|
}
|
||||||
|
|
||||||
|
func (node *BaseNode) IsValidInMsg(in []Msg) bool {
|
||||||
|
if in == nil {
|
||||||
|
log.Info("type assertion failed because it's nil")
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(in) != 1 {
|
||||||
|
log.Warn("Invalid operate message input", zap.Int("input length", len(in)))
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user