2021-10-15 18:01:38 +08:00
|
|
|
// Licensed to the LF AI & Data foundation under one
|
|
|
|
// or more contributor license agreements. See the NOTICE file
|
|
|
|
// distributed with this work for additional information
|
|
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
|
|
// to you under the Apache License, Version 2.0 (the
|
|
|
|
// "License"); you may not use this file except in compliance
|
2021-04-19 15:16:33 +08:00
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
2021-10-15 18:01:38 +08:00
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
2021-04-19 15:16:33 +08:00
|
|
|
//
|
2021-10-15 18:01:38 +08:00
|
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
// See the License for the specific language governing permissions and
|
|
|
|
// limitations under the License.
|
2021-04-19 15:16:33 +08:00
|
|
|
|
2021-01-19 11:37:16 +08:00
|
|
|
package datanode
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2021-09-09 15:36:01 +08:00
|
|
|
"errors"
|
2022-02-28 19:11:55 +08:00
|
|
|
"fmt"
|
2021-01-19 11:37:16 +08:00
|
|
|
|
2022-04-07 22:05:32 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
2022-02-28 19:11:55 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/metrics"
|
2022-03-03 21:57:56 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
2022-06-16 12:00:10 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
2021-05-25 15:35:37 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2022-03-17 18:03:23 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/storage"
|
2021-06-04 19:20:34 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
2022-07-07 14:32:21 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/concurrency"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
2021-01-19 11:37:16 +08:00
|
|
|
)
|
|
|
|
|
2021-09-15 21:25:49 +08:00
|
|
|
// dataSyncService controls a flowgraph for a specific collection
|
2021-01-19 11:37:16 +08:00
|
|
|
type dataSyncService struct {
|
2021-06-19 15:18:06 +08:00
|
|
|
ctx context.Context
|
|
|
|
cancelFn context.CancelFunc
|
2021-11-05 22:59:04 +08:00
|
|
|
fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages
|
|
|
|
flushCh chan flushMsg // chan to notify flush
|
2022-05-25 14:34:00 +08:00
|
|
|
resendTTCh chan resendTTMsg // chan to ask for resending DataNode time tick message.
|
2021-11-05 22:59:04 +08:00
|
|
|
replica Replica // segment replica stores meta
|
|
|
|
idAllocator allocatorInterface // id/timestamp allocator
|
2021-06-19 15:18:06 +08:00
|
|
|
msFactory msgstream.Factory
|
2021-11-25 09:43:15 +08:00
|
|
|
collectionID UniqueID // collection id of vchan for which this data sync service serves
|
|
|
|
vchannelName string
|
2021-11-05 22:59:04 +08:00
|
|
|
dataCoord types.DataCoord // DataCoord instance to interact with
|
2021-11-25 09:43:15 +08:00
|
|
|
clearSignal chan<- string // signal channel to notify flowgraph close for collection/partition drop msg consumed
|
2021-09-09 15:36:01 +08:00
|
|
|
|
2021-11-05 22:59:04 +08:00
|
|
|
flushingSegCache *Cache // a guarding cache stores currently flushing segment ids
|
|
|
|
flushManager flushManager // flush manager handles flush process
|
2022-03-17 18:03:23 +08:00
|
|
|
chunkManager storage.ChunkManager
|
2021-12-02 16:39:33 +08:00
|
|
|
compactor *compactionExecutor // reference to compaction executor
|
2022-07-07 14:32:21 +08:00
|
|
|
|
|
|
|
// concurrent add segments, reduce time to load delta log from oss
|
|
|
|
ioPool *concurrency.Pool
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2021-05-25 15:35:37 +08:00
|
|
|
func newDataSyncService(ctx context.Context,
|
2021-10-18 12:34:34 +08:00
|
|
|
flushCh chan flushMsg,
|
2022-05-25 14:34:00 +08:00
|
|
|
resendTTCh chan resendTTMsg,
|
2021-05-25 15:35:37 +08:00
|
|
|
replica Replica,
|
|
|
|
alloc allocatorInterface,
|
|
|
|
factory msgstream.Factory,
|
2021-06-07 11:25:37 +08:00
|
|
|
vchan *datapb.VchannelInfo,
|
2021-11-25 09:43:15 +08:00
|
|
|
clearSignal chan<- string,
|
2021-06-21 18:22:13 +08:00
|
|
|
dataCoord types.DataCoord,
|
2021-09-23 16:03:54 +08:00
|
|
|
flushingSegCache *Cache,
|
2022-03-17 18:03:23 +08:00
|
|
|
chunkManager storage.ChunkManager,
|
2021-12-02 16:39:33 +08:00
|
|
|
compactor *compactionExecutor,
|
2021-06-19 15:18:06 +08:00
|
|
|
) (*dataSyncService, error) {
|
2021-06-07 11:25:37 +08:00
|
|
|
|
2021-09-09 15:36:01 +08:00
|
|
|
if replica == nil {
|
|
|
|
return nil, errors.New("Nil input")
|
|
|
|
}
|
|
|
|
|
2022-07-07 14:32:21 +08:00
|
|
|
// Initialize io cocurrency pool
|
|
|
|
log.Info("initialize io concurrency pool", zap.String("vchannel", vchan.GetChannelName()), zap.Int("ioConcurrency", Params.DataNodeCfg.IOConcurrency))
|
|
|
|
ioPool, err := concurrency.NewPool(Params.DataNodeCfg.IOConcurrency)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("failed to create goroutine pool for dataSyncService", zap.Error(err))
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2021-06-07 11:25:37 +08:00
|
|
|
ctx1, cancel := context.WithCancel(ctx)
|
2021-05-25 15:35:37 +08:00
|
|
|
|
2021-02-03 17:30:10 +08:00
|
|
|
service := &dataSyncService{
|
2021-09-23 16:03:54 +08:00
|
|
|
ctx: ctx1,
|
|
|
|
cancelFn: cancel,
|
|
|
|
fg: nil,
|
2021-10-18 12:34:34 +08:00
|
|
|
flushCh: flushCh,
|
2022-05-25 14:34:00 +08:00
|
|
|
resendTTCh: resendTTCh,
|
2021-09-23 16:03:54 +08:00
|
|
|
replica: replica,
|
|
|
|
idAllocator: alloc,
|
|
|
|
msFactory: factory,
|
|
|
|
collectionID: vchan.GetCollectionID(),
|
2021-11-25 09:43:15 +08:00
|
|
|
vchannelName: vchan.GetChannelName(),
|
2021-09-23 16:03:54 +08:00
|
|
|
dataCoord: dataCoord,
|
|
|
|
clearSignal: clearSignal,
|
|
|
|
flushingSegCache: flushingSegCache,
|
2022-03-17 18:03:23 +08:00
|
|
|
chunkManager: chunkManager,
|
2021-12-02 16:39:33 +08:00
|
|
|
compactor: compactor,
|
2022-07-07 14:32:21 +08:00
|
|
|
ioPool: ioPool,
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
2021-05-25 15:35:37 +08:00
|
|
|
|
2021-06-19 15:18:06 +08:00
|
|
|
if err := service.initNodes(vchan); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return service, nil
|
2021-02-03 15:18:05 +08:00
|
|
|
}
|
|
|
|
|
2021-10-13 11:16:32 +08:00
|
|
|
type parallelConfig struct {
|
|
|
|
maxQueueLength int32
|
|
|
|
maxParallelism int32
|
|
|
|
}
|
|
|
|
|
|
|
|
type nodeConfig struct {
|
|
|
|
msFactory msgstream.Factory // msgStream factory
|
|
|
|
collectionID UniqueID
|
|
|
|
vChannelName string
|
|
|
|
replica Replica // Segment replica
|
|
|
|
allocator allocatorInterface
|
|
|
|
|
|
|
|
// defaults
|
|
|
|
parallelConfig
|
|
|
|
}
|
|
|
|
|
|
|
|
func newParallelConfig() parallelConfig {
|
2021-12-23 18:39:11 +08:00
|
|
|
return parallelConfig{Params.DataNodeCfg.FlowGraphMaxQueueLength, Params.DataNodeCfg.FlowGraphMaxParallelism}
|
2021-10-13 11:16:32 +08:00
|
|
|
}
|
|
|
|
|
2021-09-15 21:25:49 +08:00
|
|
|
// start starts the flowgraph in datasyncservice
|
2021-02-03 17:30:10 +08:00
|
|
|
func (dsService *dataSyncService) start() {
|
2021-05-08 14:07:04 +08:00
|
|
|
if dsService.fg != nil {
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Info("dataSyncService starting flowgraph", zap.Int64("collectionID", dsService.collectionID),
|
2022-01-04 15:47:18 +08:00
|
|
|
zap.String("vChanName", dsService.vchannelName))
|
2021-05-08 14:07:04 +08:00
|
|
|
dsService.fg.Start()
|
|
|
|
} else {
|
2022-01-04 15:47:18 +08:00
|
|
|
log.Warn("dataSyncService starting flowgraph is nil", zap.Int64("collectionID", dsService.collectionID),
|
|
|
|
zap.String("vChanName", dsService.vchannelName))
|
2021-05-08 14:07:04 +08:00
|
|
|
}
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (dsService *dataSyncService) close() {
|
|
|
|
if dsService.fg != nil {
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Info("dataSyncService closing flowgraph", zap.Int64("collectionID", dsService.collectionID),
|
2022-01-04 15:47:18 +08:00
|
|
|
zap.String("vChanName", dsService.vchannelName))
|
2021-01-19 11:37:16 +08:00
|
|
|
dsService.fg.Close()
|
2022-04-24 22:03:44 +08:00
|
|
|
metrics.DataNodeNumConsumers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Dec()
|
|
|
|
metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Sub(2) // timeTickChannel + deltaChannel
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
2021-06-07 11:25:37 +08:00
|
|
|
|
2022-06-22 10:16:14 +08:00
|
|
|
dsService.clearGlobalFlushingCache()
|
|
|
|
|
2021-06-07 11:25:37 +08:00
|
|
|
dsService.cancelFn()
|
2021-11-17 10:39:15 +08:00
|
|
|
dsService.flushManager.close()
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2022-06-22 10:16:14 +08:00
|
|
|
func (dsService *dataSyncService) clearGlobalFlushingCache() {
|
|
|
|
segments := dsService.replica.listAllSegmentIDs()
|
|
|
|
dsService.flushingSegCache.Remove(segments...)
|
|
|
|
}
|
|
|
|
|
2022-06-16 12:00:10 +08:00
|
|
|
// getSegmentInfos return the SegmentInfo details according to the given ids through RPC to datacoord
|
|
|
|
func (dsService *dataSyncService) getSegmentInfos(segmentIds []int64) ([]*datapb.SegmentInfo, error) {
|
|
|
|
var segmentInfos []*datapb.SegmentInfo
|
|
|
|
infoResp, err := dsService.dataCoord.GetSegmentInfo(dsService.ctx, &datapb.GetSegmentInfoRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_SegmentInfo,
|
|
|
|
MsgID: 0,
|
|
|
|
Timestamp: 0,
|
|
|
|
SourceID: Params.ProxyCfg.GetNodeID(),
|
|
|
|
},
|
2022-06-17 18:24:12 +08:00
|
|
|
SegmentIDs: segmentIds,
|
|
|
|
IncludeUnHealthy: true,
|
2022-06-16 12:00:10 +08:00
|
|
|
})
|
|
|
|
if err != nil {
|
|
|
|
log.Error("Fail to get datapb.SegmentInfo by ids from datacoord", zap.Error(err))
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
if infoResp.GetStatus().ErrorCode != commonpb.ErrorCode_Success {
|
|
|
|
err = errors.New(infoResp.GetStatus().Reason)
|
|
|
|
log.Error("Fail to get datapb.SegmentInfo by ids from datacoord", zap.Error(err))
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
segmentInfos = infoResp.Infos
|
|
|
|
return segmentInfos, nil
|
|
|
|
}
|
|
|
|
|
2021-09-15 21:25:49 +08:00
|
|
|
// initNodes inits a TimetickedFlowGraph
|
2021-06-19 15:18:06 +08:00
|
|
|
func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) error {
|
2021-01-19 11:37:16 +08:00
|
|
|
dsService.fg = flowgraph.NewTimeTickedFlowGraph(dsService.ctx)
|
2021-11-05 22:59:04 +08:00
|
|
|
// initialize flush manager for DataSync Service
|
2022-03-17 18:03:23 +08:00
|
|
|
dsService.flushManager = NewRendezvousFlushManager(dsService.idAllocator, dsService.chunkManager, dsService.replica,
|
2021-12-01 10:11:39 +08:00
|
|
|
flushNotifyFunc(dsService), dropVirtualChannelFunc(dsService))
|
2021-09-09 15:36:01 +08:00
|
|
|
|
2022-06-16 12:00:10 +08:00
|
|
|
var err error
|
2021-06-08 19:25:37 +08:00
|
|
|
// recover segment checkpoints
|
2022-06-16 12:00:10 +08:00
|
|
|
unflushedSegmentInfos, err := dsService.getSegmentInfos(vchanInfo.GetUnflushedSegmentIds())
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2022-07-07 14:32:21 +08:00
|
|
|
flushedSegmentInfos, err := dsService.getSegmentInfos(vchanInfo.GetFlushedSegmentIds())
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
futures := make([]*concurrency.Future, 0, len(unflushedSegmentInfos)+len(flushedSegmentInfos))
|
2022-06-16 12:00:10 +08:00
|
|
|
for _, us := range unflushedSegmentInfos {
|
2021-06-08 19:25:37 +08:00
|
|
|
if us.CollectionID != dsService.collectionID ||
|
|
|
|
us.GetInsertChannel() != vchanInfo.ChannelName {
|
2021-06-11 09:24:52 +08:00
|
|
|
log.Warn("Collection ID or ChannelName not compact",
|
|
|
|
zap.Int64("Wanted ID", dsService.collectionID),
|
|
|
|
zap.Int64("Actual ID", us.CollectionID),
|
|
|
|
zap.String("Wanted Channel Name", vchanInfo.ChannelName),
|
|
|
|
zap.String("Actual Channel Name", us.GetInsertChannel()),
|
|
|
|
)
|
2021-06-08 19:25:37 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2021-06-11 09:24:52 +08:00
|
|
|
log.Info("Recover Segment NumOfRows form checkpoints",
|
|
|
|
zap.String("InsertChannel", us.GetInsertChannel()),
|
|
|
|
zap.Int64("SegmentID", us.GetID()),
|
|
|
|
zap.Int64("NumOfRows", us.GetNumOfRows()),
|
|
|
|
)
|
2021-12-06 16:21:35 +08:00
|
|
|
var cp *segmentCheckPoint
|
|
|
|
if us.GetDmlPosition() != nil {
|
|
|
|
cp = &segmentCheckPoint{
|
|
|
|
numRows: us.GetNumOfRows(),
|
|
|
|
pos: *us.GetDmlPosition(),
|
|
|
|
}
|
|
|
|
}
|
2022-07-07 14:32:21 +08:00
|
|
|
// avoid closure capture iteration variable
|
|
|
|
segment := us
|
|
|
|
future := dsService.ioPool.Submit(func() (interface{}, error) {
|
|
|
|
if err := dsService.replica.addNormalSegment(segment.GetID(), segment.GetCollectionID(), segment.GetPartitionID(), segment.GetInsertChannel(),
|
|
|
|
segment.GetNumOfRows(), segment.GetStatslogs(), cp, vchanInfo.GetSeekPosition().GetTimestamp()); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return nil, nil
|
|
|
|
})
|
|
|
|
futures = append(futures, future)
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
|
|
|
|
2022-06-16 12:00:10 +08:00
|
|
|
for _, fs := range flushedSegmentInfos {
|
2021-10-08 19:09:12 +08:00
|
|
|
if fs.CollectionID != dsService.collectionID ||
|
|
|
|
fs.GetInsertChannel() != vchanInfo.ChannelName {
|
|
|
|
log.Warn("Collection ID or ChannelName not compact",
|
|
|
|
zap.Int64("Wanted ID", dsService.collectionID),
|
|
|
|
zap.Int64("Actual ID", fs.CollectionID),
|
|
|
|
zap.String("Wanted Channel Name", vchanInfo.ChannelName),
|
|
|
|
zap.String("Actual Channel Name", fs.GetInsertChannel()),
|
|
|
|
)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Info("Recover Segment NumOfRows form checkpoints",
|
|
|
|
zap.String("InsertChannel", fs.GetInsertChannel()),
|
|
|
|
zap.Int64("SegmentID", fs.GetID()),
|
|
|
|
zap.Int64("NumOfRows", fs.GetNumOfRows()),
|
|
|
|
)
|
2022-07-07 14:32:21 +08:00
|
|
|
// avoid closure capture iteration variable
|
|
|
|
segment := fs
|
|
|
|
future := dsService.ioPool.Submit(func() (interface{}, error) {
|
|
|
|
if err := dsService.replica.addFlushedSegment(segment.GetID(), segment.GetCollectionID(), segment.GetPartitionID(), segment.GetInsertChannel(),
|
|
|
|
segment.GetNumOfRows(), segment.GetStatslogs(), vchanInfo.GetSeekPosition().GetTimestamp()); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return nil, nil
|
|
|
|
})
|
|
|
|
futures = append(futures, future)
|
|
|
|
}
|
|
|
|
|
|
|
|
err = concurrency.AwaitAll(futures...)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
2021-10-08 19:09:12 +08:00
|
|
|
}
|
|
|
|
|
2021-11-04 15:56:28 +08:00
|
|
|
c := &nodeConfig{
|
|
|
|
msFactory: dsService.msFactory,
|
|
|
|
collectionID: vchanInfo.GetCollectionID(),
|
|
|
|
vChannelName: vchanInfo.GetChannelName(),
|
|
|
|
replica: dsService.replica,
|
|
|
|
allocator: dsService.idAllocator,
|
|
|
|
|
|
|
|
parallelConfig: newParallelConfig(),
|
|
|
|
}
|
|
|
|
|
|
|
|
var dmStreamNode Node
|
|
|
|
dmStreamNode, err = newDmInputNode(dsService.ctx, vchanInfo.GetSeekPosition(), c)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-06-16 12:00:10 +08:00
|
|
|
var ddNode Node = newDDNode(dsService.ctx, dsService.collectionID, vchanInfo, unflushedSegmentInfos, dsService.msFactory, dsService.compactor)
|
2021-11-04 15:56:28 +08:00
|
|
|
var insertBufferNode Node
|
|
|
|
insertBufferNode, err = newInsertBufferNode(
|
|
|
|
dsService.ctx,
|
2022-02-28 19:11:55 +08:00
|
|
|
dsService.collectionID,
|
2021-11-04 15:56:28 +08:00
|
|
|
dsService.flushCh,
|
2022-05-25 14:34:00 +08:00
|
|
|
dsService.resendTTCh,
|
2021-11-04 15:56:28 +08:00
|
|
|
dsService.flushManager,
|
|
|
|
dsService.flushingSegCache,
|
|
|
|
c,
|
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
var deleteNode Node
|
2021-11-11 20:56:49 +08:00
|
|
|
deleteNode, err = newDeleteNode(dsService.ctx, dsService.flushManager, dsService.clearSignal, c)
|
2021-11-04 15:56:28 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2021-02-25 17:35:36 +08:00
|
|
|
dsService.fg.AddNode(dmStreamNode)
|
|
|
|
dsService.fg.AddNode(ddNode)
|
|
|
|
dsService.fg.AddNode(insertBufferNode)
|
2021-08-25 11:41:52 +08:00
|
|
|
dsService.fg.AddNode(deleteNode)
|
2021-01-19 11:37:16 +08:00
|
|
|
|
|
|
|
// ddStreamNode
|
2021-06-02 15:58:33 +08:00
|
|
|
err = dsService.fg.SetEdges(dmStreamNode.Name(),
|
2021-01-19 11:37:16 +08:00
|
|
|
[]string{},
|
|
|
|
[]string{ddNode.Name()},
|
|
|
|
)
|
|
|
|
if err != nil {
|
2021-06-02 15:58:33 +08:00
|
|
|
log.Error("set edges failed in node", zap.String("name", dmStreamNode.Name()), zap.Error(err))
|
2021-06-19 15:18:06 +08:00
|
|
|
return err
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// ddNode
|
|
|
|
err = dsService.fg.SetEdges(ddNode.Name(),
|
2021-06-02 15:58:33 +08:00
|
|
|
[]string{dmStreamNode.Name()},
|
|
|
|
[]string{insertBufferNode.Name()},
|
2021-01-19 11:37:16 +08:00
|
|
|
)
|
|
|
|
if err != nil {
|
2021-02-26 10:13:36 +08:00
|
|
|
log.Error("set edges failed in node", zap.String("name", ddNode.Name()), zap.Error(err))
|
2021-06-19 15:18:06 +08:00
|
|
|
return err
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// insertBufferNode
|
|
|
|
err = dsService.fg.SetEdges(insertBufferNode.Name(),
|
2021-06-02 15:58:33 +08:00
|
|
|
[]string{ddNode.Name()},
|
2021-08-25 11:41:52 +08:00
|
|
|
[]string{deleteNode.Name()},
|
2021-01-19 11:37:16 +08:00
|
|
|
)
|
|
|
|
if err != nil {
|
2021-02-26 10:13:36 +08:00
|
|
|
log.Error("set edges failed in node", zap.String("name", insertBufferNode.Name()), zap.Error(err))
|
2021-06-19 15:18:06 +08:00
|
|
|
return err
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
2021-08-25 11:41:52 +08:00
|
|
|
|
|
|
|
//deleteNode
|
|
|
|
err = dsService.fg.SetEdges(deleteNode.Name(),
|
|
|
|
[]string{insertBufferNode.Name()},
|
|
|
|
[]string{},
|
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("set edges failed in node", zap.String("name", deleteNode.Name()), zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
2021-06-19 15:18:06 +08:00
|
|
|
return nil
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|