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"
|
2021-06-04 19:20:34 +08:00
|
|
|
"fmt"
|
2021-01-19 11:37:16 +08:00
|
|
|
|
2021-10-19 11:04:34 +08:00
|
|
|
miniokv "github.com/milvus-io/milvus/internal/kv/minio"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/msgstream"
|
2021-06-04 19:20:34 +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"
|
2021-06-04 19:20:34 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
2021-02-26 10:13:36 +08:00
|
|
|
|
|
|
|
"go.uber.org/zap"
|
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
|
|
|
|
fg *flowgraph.TimeTickedFlowGraph
|
2021-10-18 12:34:34 +08:00
|
|
|
flushCh chan flushMsg
|
2021-06-19 15:18:06 +08:00
|
|
|
replica Replica
|
|
|
|
idAllocator allocatorInterface
|
|
|
|
msFactory msgstream.Factory
|
|
|
|
collectionID UniqueID
|
2021-06-21 18:22:13 +08:00
|
|
|
dataCoord types.DataCoord
|
2021-06-19 15:18:06 +08:00
|
|
|
clearSignal chan<- UniqueID
|
2021-09-09 15:36:01 +08:00
|
|
|
|
2021-09-23 16:03:54 +08:00
|
|
|
flushingSegCache *Cache
|
2021-10-19 11:04:34 +08:00
|
|
|
flushManager flushManager
|
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,
|
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,
|
|
|
|
clearSignal chan<- UniqueID,
|
2021-06-21 18:22:13 +08:00
|
|
|
dataCoord types.DataCoord,
|
2021-09-23 16:03:54 +08:00
|
|
|
flushingSegCache *Cache,
|
2021-06-07 11:25:37 +08:00
|
|
|
|
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")
|
|
|
|
}
|
|
|
|
|
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,
|
2021-09-23 16:03:54 +08:00
|
|
|
replica: replica,
|
|
|
|
idAllocator: alloc,
|
|
|
|
msFactory: factory,
|
|
|
|
collectionID: vchan.GetCollectionID(),
|
|
|
|
dataCoord: dataCoord,
|
|
|
|
clearSignal: clearSignal,
|
|
|
|
flushingSegCache: flushingSegCache,
|
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 {
|
|
|
|
return parallelConfig{Params.FlowGraphMaxQueueLength, Params.FlowGraphMaxParallelism}
|
|
|
|
}
|
|
|
|
|
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 {
|
2021-06-07 11:25:37 +08:00
|
|
|
log.Debug("Data Sync Service starting flowgraph")
|
2021-05-08 14:07:04 +08:00
|
|
|
dsService.fg.Start()
|
|
|
|
} else {
|
|
|
|
log.Debug("Data Sync Service flowgraph nil")
|
|
|
|
}
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (dsService *dataSyncService) close() {
|
|
|
|
if dsService.fg != nil {
|
2021-06-07 11:25:37 +08:00
|
|
|
log.Debug("Data Sync Service closing flowgraph")
|
2021-01-19 11:37:16 +08:00
|
|
|
dsService.fg.Close()
|
|
|
|
}
|
2021-06-07 11:25:37 +08:00
|
|
|
|
|
|
|
dsService.cancelFn()
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
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-02-08 14:30:54 +08:00
|
|
|
m := map[string]interface{}{
|
|
|
|
"PulsarAddress": Params.PulsarAddress,
|
|
|
|
"ReceiveBufSize": 1024,
|
2021-05-25 15:35:37 +08:00
|
|
|
"PulsarBufSize": 1024,
|
|
|
|
}
|
|
|
|
|
2021-05-28 14:54:31 +08:00
|
|
|
err := dsService.msFactory.SetParams(m)
|
2021-02-08 14:30:54 +08:00
|
|
|
if err != nil {
|
2021-06-19 15:18:06 +08:00
|
|
|
return err
|
2021-02-08 14:30:54 +08:00
|
|
|
}
|
|
|
|
|
2021-10-19 11:04:34 +08:00
|
|
|
// MinIO
|
|
|
|
option := &miniokv.Option{
|
|
|
|
Address: Params.MinioAddress,
|
|
|
|
AccessKeyID: Params.MinioAccessKeyID,
|
|
|
|
SecretAccessKeyID: Params.MinioSecretAccessKey,
|
|
|
|
UseSSL: Params.MinioUseSSL,
|
|
|
|
CreateBucket: true,
|
|
|
|
BucketName: Params.MinioBucketName,
|
|
|
|
}
|
|
|
|
|
|
|
|
minIOKV, err := miniokv.NewMinIOKV(dsService.ctx, option)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
dsService.flushManager = NewRendezvousFlushManager(dsService.idAllocator, minIOKV, dsService.replica, func(pack *segmentFlushPack) error {
|
|
|
|
fieldInsert := []*datapb.FieldBinlog{}
|
|
|
|
fieldStats := []*datapb.FieldBinlog{}
|
2021-10-25 18:49:20 +08:00
|
|
|
deltaInfos := []*datapb.DeltaLogInfo{}
|
2021-06-04 19:20:34 +08:00
|
|
|
checkPoints := []*datapb.CheckPoint{}
|
2021-10-19 11:04:34 +08:00
|
|
|
for k, v := range pack.insertLogs {
|
|
|
|
fieldInsert = append(fieldInsert, &datapb.FieldBinlog{FieldID: k, Binlogs: []string{v}})
|
2021-06-04 19:20:34 +08:00
|
|
|
}
|
2021-10-19 11:04:34 +08:00
|
|
|
for k, v := range pack.statsLogs {
|
|
|
|
fieldStats = append(fieldStats, &datapb.FieldBinlog{FieldID: k, Binlogs: []string{v}})
|
2021-06-04 19:20:34 +08:00
|
|
|
}
|
2021-10-19 14:34:35 +08:00
|
|
|
for _, delData := range pack.deltaLogs {
|
2021-10-25 18:49:20 +08:00
|
|
|
deltaInfos = append(deltaInfos, &datapb.DeltaLogInfo{RecordEntries: uint64(delData.size), TimestampFrom: delData.tsFrom, TimestampTo: delData.tsTo, DeltaLogPath: delData.filePath, DeltaLogSize: delData.fileSize})
|
2021-10-19 14:34:35 +08:00
|
|
|
}
|
|
|
|
|
2021-10-19 11:04:34 +08:00
|
|
|
// only current segment checkpoint info,
|
|
|
|
updates, _ := dsService.replica.getSegmentStatisticsUpdates(pack.segmentID)
|
|
|
|
checkPoints = append(checkPoints, &datapb.CheckPoint{
|
|
|
|
SegmentID: pack.segmentID,
|
|
|
|
NumOfRows: updates.GetNumRows(),
|
|
|
|
Position: pack.pos,
|
|
|
|
})
|
|
|
|
|
2021-06-08 19:25:37 +08:00
|
|
|
log.Debug("SaveBinlogPath",
|
2021-10-19 11:04:34 +08:00
|
|
|
zap.Int64("SegmentID", pack.segmentID),
|
|
|
|
zap.Int64("CollectionID", dsService.collectionID),
|
|
|
|
zap.Int("Length of Field2BinlogPaths", len(fieldInsert)),
|
2021-10-25 18:49:20 +08:00
|
|
|
zap.Int("Length of Field2Stats", len(fieldStats)),
|
|
|
|
zap.Int("Length of Field2Deltalogs", len(deltaInfos)),
|
2021-06-08 19:25:37 +08:00
|
|
|
)
|
2021-06-04 19:20:34 +08:00
|
|
|
|
|
|
|
req := &datapb.SaveBinlogPathsRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
2021-06-08 19:25:37 +08:00
|
|
|
MsgType: 0, //TODO msg type
|
|
|
|
MsgID: 0, //TODO msg id
|
|
|
|
Timestamp: 0, //TODO time stamp
|
2021-06-04 19:20:34 +08:00
|
|
|
SourceID: Params.NodeID,
|
|
|
|
},
|
2021-10-19 14:34:35 +08:00
|
|
|
SegmentID: pack.segmentID,
|
|
|
|
CollectionID: dsService.collectionID,
|
|
|
|
Field2BinlogPaths: fieldInsert,
|
|
|
|
Field2StatslogPaths: fieldStats,
|
|
|
|
Deltalogs: deltaInfos,
|
|
|
|
|
2021-10-19 11:04:34 +08:00
|
|
|
CheckPoints: checkPoints,
|
|
|
|
|
|
|
|
StartPositions: dsService.replica.listNewSegmentsStartPositions(),
|
|
|
|
Flushed: pack.flushed,
|
2021-06-04 19:20:34 +08:00
|
|
|
}
|
2021-06-21 18:22:13 +08:00
|
|
|
rsp, err := dsService.dataCoord.SaveBinlogPaths(dsService.ctx, req)
|
2021-06-04 19:20:34 +08:00
|
|
|
if err != nil {
|
2021-06-08 19:25:37 +08:00
|
|
|
return fmt.Errorf(err.Error())
|
2021-06-04 19:20:34 +08:00
|
|
|
}
|
|
|
|
if rsp.ErrorCode != commonpb.ErrorCode_Success {
|
|
|
|
return fmt.Errorf("data service save bin log path failed, reason = %s", rsp.Reason)
|
|
|
|
}
|
2021-10-27 16:06:24 +08:00
|
|
|
dsService.flushingSegCache.Remove(req.GetSegmentID())
|
2021-06-04 19:20:34 +08:00
|
|
|
return nil
|
2021-10-19 11:04:34 +08:00
|
|
|
})
|
2021-09-09 15:36:01 +08:00
|
|
|
|
2021-06-08 19:25:37 +08:00
|
|
|
// recover segment checkpoints
|
|
|
|
for _, us := range vchanInfo.GetUnflushedSegments() {
|
|
|
|
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-06-21 16:00:22 +08:00
|
|
|
|
2021-10-22 14:31:13 +08:00
|
|
|
if err := dsService.replica.addNormalSegment(us.GetID(), us.CollectionID, us.PartitionID, us.GetInsertChannel(), us.GetNumOfRows(), us.Statslogs, &segmentCheckPoint{us.GetNumOfRows(), *us.GetDmlPosition()}); err != nil {
|
2021-10-09 15:38:57 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
|
|
|
|
2021-10-08 19:09:12 +08:00
|
|
|
for _, fs := range vchanInfo.GetFlushedSegments() {
|
|
|
|
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()),
|
|
|
|
)
|
2021-10-14 17:40:42 +08:00
|
|
|
if err := dsService.replica.addFlushedSegment(fs.GetID(), fs.CollectionID,
|
2021-10-22 14:31:13 +08:00
|
|
|
fs.PartitionID, fs.GetInsertChannel(), fs.GetNumOfRows(), fs.Statslogs); err != nil {
|
2021-10-14 17:40:42 +08:00
|
|
|
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
|
|
|
|
}
|
|
|
|
|
2021-11-05 11:59:02 +08:00
|
|
|
var ddNode Node = newDDNode(dsService.ctx, dsService.clearSignal, dsService.collectionID, vchanInfo, dsService.msFactory)
|
2021-11-04 15:56:28 +08:00
|
|
|
var insertBufferNode Node
|
|
|
|
insertBufferNode, err = newInsertBufferNode(
|
|
|
|
dsService.ctx,
|
|
|
|
dsService.flushCh,
|
|
|
|
dsService.flushManager,
|
|
|
|
dsService.flushingSegCache,
|
|
|
|
c,
|
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
var deleteNode Node
|
|
|
|
deleteNode, err = newDeleteNode(dsService.ctx, dsService.flushManager, c)
|
|
|
|
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
|
|
|
}
|