2021-10-15 18:07:09 +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:07:09 +08:00
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
2021-04-19 15:16:33 +08:00
|
|
|
//
|
2021-10-15 18:07:09 +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 (
|
2021-11-05 11:59:02 +08:00
|
|
|
"context"
|
2021-12-30 10:33:46 +08:00
|
|
|
"fmt"
|
2022-05-24 21:11:59 +08:00
|
|
|
"reflect"
|
2022-09-16 09:56:47 +08:00
|
|
|
"strconv"
|
2021-11-11 20:56:49 +08:00
|
|
|
"sync/atomic"
|
2021-06-07 13:58:37 +08:00
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
"github.com/golang/protobuf/proto"
|
2022-05-24 21:11:59 +08:00
|
|
|
"github.com/opentracing/opentracing-go"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2022-09-16 16:56:49 +08:00
|
|
|
"github.com/milvus-io/milvus/api/commonpb"
|
2022-06-02 10:12:03 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/common"
|
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"
|
2021-06-07 13:58:37 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2021-06-02 15:58:33 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
2021-10-07 19:54:56 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
2022-03-15 21:51:21 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
2022-09-16 09:56:47 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
2022-05-24 21:11:59 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/retry"
|
2021-06-30 16:18:13 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/trace"
|
2022-06-30 17:44:18 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
2021-01-19 11:37:16 +08:00
|
|
|
)
|
|
|
|
|
2021-10-07 19:54:56 +08:00
|
|
|
// make sure ddNode implements flowgraph.Node
|
|
|
|
var _ flowgraph.Node = (*ddNode)(nil)
|
|
|
|
|
2021-12-02 18:59:33 +08:00
|
|
|
// ddNode filters messages from message streams.
|
2021-10-06 22:51:58 +08:00
|
|
|
//
|
|
|
|
// ddNode recives all the messages from message stream dml channels, including insert messages,
|
2022-07-07 18:26:20 +08:00
|
|
|
// delete messages and ddl messages like CreateCollectionMsg and DropCollectionMsg.
|
2021-10-06 22:51:58 +08:00
|
|
|
//
|
2022-07-07 18:26:20 +08:00
|
|
|
// ddNode filters insert messages according to the `sealedSegment`.
|
|
|
|
// ddNode will filter out the insert message for those who belong to `sealedSegment`
|
2021-10-06 22:51:58 +08:00
|
|
|
//
|
|
|
|
// When receiving a `DropCollection` message, ddNode will send a signal to DataNode `BackgroundGC`
|
2022-07-07 18:26:20 +08:00
|
|
|
// goroutinue, telling DataNode to release the resources of this particular flow graph.
|
2021-10-06 22:51:58 +08:00
|
|
|
//
|
|
|
|
// After the filtering process, ddNode passes all the valid insert messages and delete message
|
|
|
|
// to the following flow graph node, which in DataNode is `insertBufferNode`
|
2021-01-19 11:37:16 +08:00
|
|
|
type ddNode struct {
|
|
|
|
BaseNode
|
2021-06-07 11:25:37 +08:00
|
|
|
|
2022-05-24 21:11:59 +08:00
|
|
|
ctx context.Context
|
2021-06-07 11:25:37 +08:00
|
|
|
collectionID UniqueID
|
2022-07-07 18:26:20 +08:00
|
|
|
vChannelName string
|
2021-11-05 11:59:02 +08:00
|
|
|
|
2021-12-02 16:39:33 +08:00
|
|
|
deltaMsgStream msgstream.MsgStream
|
|
|
|
dropMode atomic.Value
|
|
|
|
compactionExecutor *compactionExecutor
|
2022-07-07 18:26:20 +08:00
|
|
|
|
|
|
|
// for recovery
|
|
|
|
growingSegInfo map[UniqueID]*datapb.SegmentInfo // segmentID
|
|
|
|
sealedSegInfo map[UniqueID]*datapb.SegmentInfo // segmentID
|
|
|
|
droppedSegmentIDs []int64
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2021-10-07 19:54:56 +08:00
|
|
|
// Name returns node name, implementing flowgraph.Node
|
2021-06-02 15:58:33 +08:00
|
|
|
func (ddn *ddNode) Name() string {
|
2022-07-07 18:26:20 +08:00
|
|
|
return fmt.Sprintf("ddNode-%d-%s", ddn.collectionID, ddn.vChannelName)
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2021-10-07 19:54:56 +08:00
|
|
|
// Operate handles input messages, implementing flowgrpah.Node
|
2021-09-07 15:41:59 +08:00
|
|
|
func (ddn *ddNode) Operate(in []Msg) []Msg {
|
2022-10-10 22:15:22 +08:00
|
|
|
if in == nil {
|
|
|
|
log.Debug("type assertion failed for MsgStreamMsg because it's nil")
|
|
|
|
return []Msg{}
|
|
|
|
}
|
|
|
|
|
2021-01-19 11:37:16 +08:00
|
|
|
if len(in) != 1 {
|
2021-09-07 15:41:59 +08:00
|
|
|
log.Warn("Invalid operate message input in ddNode", zap.Int("input length", len(in)))
|
|
|
|
return []Msg{}
|
2021-05-31 15:28:30 +08:00
|
|
|
}
|
|
|
|
|
2021-02-25 17:35:36 +08:00
|
|
|
msMsg, ok := in[0].(*MsgStreamMsg)
|
2021-01-19 11:37:16 +08:00
|
|
|
if !ok {
|
2022-10-10 22:15:22 +08:00
|
|
|
log.Warn("type assertion failed for MsgStreamMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
2021-09-07 15:41:59 +08:00
|
|
|
return []Msg{}
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2021-06-30 16:18:13 +08:00
|
|
|
var spans []opentracing.Span
|
|
|
|
for _, msg := range msMsg.TsMessages() {
|
|
|
|
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
|
|
|
spans = append(spans, sp)
|
|
|
|
msg.SetTraceCtx(ctx)
|
|
|
|
}
|
|
|
|
|
2021-11-11 20:56:49 +08:00
|
|
|
if load := ddn.dropMode.Load(); load != nil && load.(bool) {
|
2021-11-15 15:25:09 +08:00
|
|
|
log.Debug("ddNode in dropMode",
|
2022-07-07 18:26:20 +08:00
|
|
|
zap.String("vChannelName", ddn.vChannelName),
|
2021-11-15 15:25:09 +08:00
|
|
|
zap.Int64("collection ID", ddn.collectionID))
|
2021-11-11 20:56:49 +08:00
|
|
|
return []Msg{}
|
|
|
|
}
|
|
|
|
|
2021-09-26 10:43:57 +08:00
|
|
|
var fgMsg = flowGraphMsg{
|
2021-06-02 15:58:33 +08:00
|
|
|
insertMessages: make([]*msgstream.InsertMsg, 0),
|
2021-01-19 11:37:16 +08:00
|
|
|
timeRange: TimeRange{
|
|
|
|
timestampMin: msMsg.TimestampMin(),
|
|
|
|
timestampMax: msMsg.TimestampMax(),
|
|
|
|
},
|
2021-06-02 15:58:33 +08:00
|
|
|
startPositions: make([]*internalpb.MsgPosition, 0),
|
|
|
|
endPositions: make([]*internalpb.MsgPosition, 0),
|
2021-11-11 20:56:49 +08:00
|
|
|
dropCollection: false,
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2021-12-02 16:39:33 +08:00
|
|
|
var forwardMsgs []msgstream.TsMsg
|
2021-06-02 15:58:33 +08:00
|
|
|
for _, msg := range msMsg.TsMessages() {
|
|
|
|
switch msg.Type() {
|
|
|
|
case commonpb.MsgType_DropCollection:
|
2021-06-07 11:25:37 +08:00
|
|
|
if msg.(*msgstream.DropCollectionMsg).GetCollectionID() == ddn.collectionID {
|
2021-11-25 09:43:15 +08:00
|
|
|
log.Info("Receiving DropCollection msg",
|
|
|
|
zap.Any("collectionID", ddn.collectionID),
|
2022-07-07 18:26:20 +08:00
|
|
|
zap.String("vChannelName", ddn.vChannelName))
|
2021-11-11 20:56:49 +08:00
|
|
|
ddn.dropMode.Store(true)
|
2021-12-02 16:39:33 +08:00
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
log.Info("Stop compaction of vChannel", zap.String("vChannelName", ddn.vChannelName))
|
|
|
|
ddn.compactionExecutor.stopExecutingtaskByVChannelName(ddn.vChannelName)
|
2021-11-11 20:56:49 +08:00
|
|
|
fgMsg.dropCollection = true
|
2021-06-07 11:25:37 +08:00
|
|
|
}
|
2022-07-07 18:26:20 +08:00
|
|
|
|
2022-07-15 17:12:27 +08:00
|
|
|
case commonpb.MsgType_DropPartition:
|
|
|
|
dpMsg := msg.(*msgstream.DropPartitionMsg)
|
|
|
|
if dpMsg.GetCollectionID() == ddn.collectionID {
|
|
|
|
log.Info("drop partition msg received",
|
|
|
|
zap.Int64("collectionID", dpMsg.GetCollectionID()),
|
|
|
|
zap.Int64("partitionID", dpMsg.GetPartitionID()),
|
|
|
|
zap.String("vChanneName", ddn.vChannelName))
|
|
|
|
fgMsg.dropPartitions = append(fgMsg.dropPartitions, dpMsg.PartitionID)
|
|
|
|
}
|
|
|
|
|
2021-06-02 15:58:33 +08:00
|
|
|
case commonpb.MsgType_Insert:
|
2021-08-31 18:35:58 +08:00
|
|
|
imsg := msg.(*msgstream.InsertMsg)
|
2021-09-07 15:41:59 +08:00
|
|
|
if imsg.CollectionID != ddn.collectionID {
|
2022-07-07 18:26:20 +08:00
|
|
|
log.Warn("filter invalid insert message, collection mis-match",
|
2021-11-25 09:43:15 +08:00
|
|
|
zap.Int64("Get collID", imsg.CollectionID),
|
|
|
|
zap.Int64("Expected collID", ddn.collectionID))
|
2021-09-07 15:41:59 +08:00
|
|
|
continue
|
|
|
|
}
|
2022-07-07 18:26:20 +08:00
|
|
|
|
|
|
|
if ddn.tryToFilterSegmentInsertMessages(imsg) {
|
|
|
|
log.Info("filter insert messages",
|
|
|
|
zap.Int64("filter segment ID", imsg.GetSegmentID()),
|
|
|
|
zap.Uint64("message timestamp", msg.EndTs()),
|
2022-08-23 19:12:55 +08:00
|
|
|
zap.String("segment's vChannel", imsg.GetShardName()),
|
|
|
|
zap.String("current vChannel", ddn.vChannelName))
|
2022-07-07 18:26:20 +08:00
|
|
|
continue
|
2021-05-31 15:28:30 +08:00
|
|
|
}
|
2022-07-07 18:26:20 +08:00
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
rateCol.Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(&imsg.InsertRequest)))
|
|
|
|
metrics.DataNodeConsumeCounter.WithLabelValues(strconv.FormatInt(Params.DataNodeCfg.GetNodeID(), 10), metrics.InsertLabel).Add(float64(proto.Size(&imsg.InsertRequest)))
|
|
|
|
|
2021-11-25 09:43:15 +08:00
|
|
|
log.Debug("DDNode receive insert messages",
|
|
|
|
zap.Int("numRows", len(imsg.GetRowIDs())),
|
2022-07-07 18:26:20 +08:00
|
|
|
zap.String("vChannelName", ddn.vChannelName))
|
2021-09-26 10:43:57 +08:00
|
|
|
fgMsg.insertMessages = append(fgMsg.insertMessages, imsg)
|
2022-07-07 18:26:20 +08:00
|
|
|
|
2021-10-11 16:31:44 +08:00
|
|
|
case commonpb.MsgType_Delete:
|
|
|
|
dmsg := msg.(*msgstream.DeleteMsg)
|
2021-12-24 10:50:07 +08:00
|
|
|
log.Debug("DDNode receive delete messages",
|
2022-04-02 17:43:29 +08:00
|
|
|
zap.Int64("num", dmsg.NumRows),
|
2022-07-07 18:26:20 +08:00
|
|
|
zap.String("vChannelName", ddn.vChannelName))
|
2022-04-02 17:43:29 +08:00
|
|
|
for i := int64(0); i < dmsg.NumRows; i++ {
|
2021-11-09 09:27:04 +08:00
|
|
|
dmsg.HashValues = append(dmsg.HashValues, uint32(0))
|
|
|
|
}
|
|
|
|
forwardMsgs = append(forwardMsgs, dmsg)
|
2021-10-11 16:31:44 +08:00
|
|
|
if dmsg.CollectionID != ddn.collectionID {
|
2021-11-25 09:43:15 +08:00
|
|
|
log.Warn("filter invalid DeleteMsg, collection mis-match",
|
|
|
|
zap.Int64("Get collID", dmsg.CollectionID),
|
|
|
|
zap.Int64("Expected collID", ddn.collectionID))
|
2021-10-11 16:31:44 +08:00
|
|
|
continue
|
|
|
|
}
|
2022-09-16 09:56:47 +08:00
|
|
|
rateCol.Add(metricsinfo.DeleteConsumeThroughput, float64(proto.Size(&dmsg.DeleteRequest)))
|
|
|
|
metrics.DataNodeConsumeCounter.WithLabelValues(strconv.FormatInt(Params.DataNodeCfg.GetNodeID(), 10), metrics.DeleteLabel).Add(float64(proto.Size(&dmsg.DeleteRequest)))
|
2021-10-11 16:31:44 +08:00
|
|
|
fgMsg.deleteMessages = append(fgMsg.deleteMessages, dmsg)
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
}
|
2022-05-24 21:11:59 +08:00
|
|
|
err := retry.Do(ddn.ctx, func() error {
|
|
|
|
return ddn.forwardDeleteMsg(forwardMsgs, msMsg.TimestampMin(), msMsg.TimestampMax())
|
2022-09-13 14:12:31 +08:00
|
|
|
}, getFlowGraphRetryOpt())
|
2021-11-05 11:59:02 +08:00
|
|
|
if err != nil {
|
2022-07-07 18:26:20 +08:00
|
|
|
err = fmt.Errorf("DDNode forward delete msg failed, vChannel = %s, err = %s", ddn.vChannelName, err)
|
2022-05-24 21:11:59 +08:00
|
|
|
log.Error(err.Error())
|
2022-06-02 10:12:03 +08:00
|
|
|
if !common.IsIgnorableError(err) {
|
|
|
|
panic(err)
|
|
|
|
}
|
2021-11-05 11:59:02 +08:00
|
|
|
}
|
2021-01-19 11:37:16 +08:00
|
|
|
|
2021-09-26 10:43:57 +08:00
|
|
|
fgMsg.startPositions = append(fgMsg.startPositions, msMsg.StartPositions()...)
|
|
|
|
fgMsg.endPositions = append(fgMsg.endPositions, msMsg.EndPositions()...)
|
2021-01-22 19:36:09 +08:00
|
|
|
|
2021-06-30 16:18:13 +08:00
|
|
|
for _, sp := range spans {
|
|
|
|
sp.Finish()
|
|
|
|
}
|
|
|
|
|
2021-10-11 16:31:44 +08:00
|
|
|
return []Msg{&fgMsg}
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
func (ddn *ddNode) tryToFilterSegmentInsertMessages(msg *msgstream.InsertMsg) bool {
|
2022-08-23 19:12:55 +08:00
|
|
|
if msg.GetShardName() != ddn.vChannelName {
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
// Filter all dropped segments
|
|
|
|
if ddn.isDropped(msg.GetSegmentID()) {
|
2021-06-11 09:24:52 +08:00
|
|
|
return true
|
2021-06-07 13:58:37 +08:00
|
|
|
}
|
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
// Filter all sealed segments until current Ts > Sealed segment cp
|
|
|
|
for segID, segInfo := range ddn.sealedSegInfo {
|
|
|
|
if msg.EndTs() > segInfo.GetDmlPosition().GetTimestamp() {
|
|
|
|
delete(ddn.sealedSegInfo, segID)
|
2021-06-07 13:58:37 +08:00
|
|
|
}
|
2021-09-07 15:41:59 +08:00
|
|
|
}
|
2022-07-07 18:26:20 +08:00
|
|
|
if _, ok := ddn.sealedSegInfo[msg.GetSegmentID()]; ok {
|
|
|
|
return true
|
|
|
|
}
|
2021-05-19 19:42:07 +08:00
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
// Filter all growing segments until current Ts > growing segment dmlPosition
|
|
|
|
if si, ok := ddn.growingSegInfo[msg.GetSegmentID()]; ok {
|
|
|
|
if msg.EndTs() <= si.GetDmlPosition().GetTimestamp() {
|
2021-06-07 13:58:37 +08:00
|
|
|
return true
|
|
|
|
}
|
2022-07-07 18:26:20 +08:00
|
|
|
|
|
|
|
delete(ddn.growingSegInfo, msg.GetSegmentID())
|
2021-06-07 13:58:37 +08:00
|
|
|
}
|
2022-07-07 18:26:20 +08:00
|
|
|
|
2021-06-07 13:58:37 +08:00
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2021-11-24 14:11:15 +08:00
|
|
|
func (ddn *ddNode) isDropped(segID UniqueID) bool {
|
2022-06-16 12:00:10 +08:00
|
|
|
for _, droppedSegmentID := range ddn.droppedSegmentIDs {
|
|
|
|
if droppedSegmentID == segID {
|
2021-11-24 14:11:15 +08:00
|
|
|
return true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2021-11-05 11:59:02 +08:00
|
|
|
func (ddn *ddNode) forwardDeleteMsg(msgs []msgstream.TsMsg, minTs Timestamp, maxTs Timestamp) error {
|
|
|
|
if len(msgs) != 0 {
|
|
|
|
var msgPack = msgstream.MsgPack{
|
|
|
|
Msgs: msgs,
|
|
|
|
BeginTs: minTs,
|
|
|
|
EndTs: maxTs,
|
|
|
|
}
|
|
|
|
if err := ddn.deltaMsgStream.Produce(&msgPack); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if err := ddn.sendDeltaTimeTick(maxTs); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ddn *ddNode) sendDeltaTimeTick(ts Timestamp) error {
|
|
|
|
msgPack := msgstream.MsgPack{}
|
|
|
|
baseMsg := msgstream.BaseMsg{
|
|
|
|
BeginTimestamp: ts,
|
|
|
|
EndTimestamp: ts,
|
|
|
|
HashValues: []uint32{0},
|
|
|
|
}
|
|
|
|
timeTickResult := internalpb.TimeTickMsg{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_TimeTick,
|
|
|
|
MsgID: 0,
|
|
|
|
Timestamp: ts,
|
2022-04-24 22:03:44 +08:00
|
|
|
SourceID: Params.DataNodeCfg.GetNodeID(),
|
2021-11-05 11:59:02 +08:00
|
|
|
},
|
|
|
|
}
|
|
|
|
timeTickMsg := &msgstream.TimeTickMsg{
|
|
|
|
BaseMsg: baseMsg,
|
|
|
|
TimeTickMsg: timeTickResult,
|
|
|
|
}
|
|
|
|
msgPack.Msgs = append(msgPack.Msgs, timeTickMsg)
|
|
|
|
|
|
|
|
if err := ddn.deltaMsgStream.Produce(&msgPack); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2022-06-30 17:44:18 +08:00
|
|
|
p, _ := tsoutil.ParseTS(ts)
|
|
|
|
log.RatedDebug(10.0, "DDNode sent delta timeTick",
|
|
|
|
zap.Any("collectionID", ddn.collectionID),
|
|
|
|
zap.Any("ts", ts),
|
|
|
|
zap.Any("ts_p", p),
|
2022-07-07 18:26:20 +08:00
|
|
|
zap.Any("channel", ddn.vChannelName),
|
2022-06-30 17:44:18 +08:00
|
|
|
)
|
2021-11-05 11:59:02 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ddn *ddNode) Close() {
|
|
|
|
if ddn.deltaMsgStream != nil {
|
|
|
|
ddn.deltaMsgStream.Close()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
func newDDNode(ctx context.Context, collID UniqueID, vChannelName string, droppedSegmentIDs []UniqueID,
|
|
|
|
sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo,
|
|
|
|
msFactory msgstream.Factory, compactor *compactionExecutor) (*ddNode, error) {
|
|
|
|
|
2021-01-19 11:37:16 +08:00
|
|
|
baseNode := BaseNode{}
|
2021-12-23 18:39:11 +08:00
|
|
|
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength)
|
|
|
|
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism)
|
2021-01-19 11:37:16 +08:00
|
|
|
|
2021-11-05 11:59:02 +08:00
|
|
|
deltaStream, err := msFactory.NewMsgStream(ctx)
|
|
|
|
if err != nil {
|
2022-07-07 18:26:20 +08:00
|
|
|
return nil, err
|
2021-11-05 11:59:02 +08:00
|
|
|
}
|
2022-07-07 18:26:20 +08:00
|
|
|
pChannelName := funcutil.ToPhysicalChannel(vChannelName)
|
|
|
|
log.Info("ddNode convert vChannel to pChannel",
|
|
|
|
zap.String("vChannelName", vChannelName),
|
2022-04-20 14:03:40 +08:00
|
|
|
zap.String("pChannelName", pChannelName),
|
|
|
|
)
|
2022-07-07 18:26:20 +08:00
|
|
|
|
2022-03-15 21:51:21 +08:00
|
|
|
deltaChannelName, err := funcutil.ConvertChannelName(pChannelName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta)
|
2021-11-05 11:59:02 +08:00
|
|
|
if err != nil {
|
2022-07-07 18:26:20 +08:00
|
|
|
return nil, err
|
2021-11-05 11:59:02 +08:00
|
|
|
}
|
2021-11-09 09:27:04 +08:00
|
|
|
deltaStream.SetRepackFunc(msgstream.DefaultRepackFunc)
|
2021-11-05 11:59:02 +08:00
|
|
|
deltaStream.AsProducer([]string{deltaChannelName})
|
2022-04-24 22:03:44 +08:00
|
|
|
metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Inc()
|
2021-11-09 09:27:04 +08:00
|
|
|
log.Debug("datanode AsProducer", zap.String("DeltaChannelName", deltaChannelName))
|
2021-11-05 11:59:02 +08:00
|
|
|
var deltaMsgStream msgstream.MsgStream = deltaStream
|
|
|
|
deltaMsgStream.Start()
|
|
|
|
|
2021-09-07 15:41:59 +08:00
|
|
|
dd := &ddNode{
|
2022-05-24 21:11:59 +08:00
|
|
|
ctx: ctx,
|
2021-12-02 16:39:33 +08:00
|
|
|
BaseNode: baseNode,
|
|
|
|
collectionID: collID,
|
2022-07-07 18:26:20 +08:00
|
|
|
sealedSegInfo: make(map[UniqueID]*datapb.SegmentInfo, len(sealedSegments)),
|
|
|
|
growingSegInfo: make(map[UniqueID]*datapb.SegmentInfo, len(growingSegments)),
|
|
|
|
droppedSegmentIDs: droppedSegmentIDs,
|
|
|
|
vChannelName: vChannelName,
|
2021-12-02 16:39:33 +08:00
|
|
|
deltaMsgStream: deltaMsgStream,
|
|
|
|
compactionExecutor: compactor,
|
2021-06-07 13:58:37 +08:00
|
|
|
}
|
|
|
|
|
2021-11-11 20:56:49 +08:00
|
|
|
dd.dropMode.Store(false)
|
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
for _, s := range sealedSegments {
|
|
|
|
dd.sealedSegInfo[s.GetID()] = s
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
2021-09-07 15:41:59 +08:00
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
for _, s := range growingSegments {
|
|
|
|
dd.growingSegInfo[s.GetID()] = s
|
|
|
|
}
|
|
|
|
log.Info("ddNode add sealed and growing segments",
|
2021-09-07 15:41:59 +08:00
|
|
|
zap.Int64("collectionID", collID),
|
2022-07-07 18:26:20 +08:00
|
|
|
zap.Int("No. sealed segments", len(sealedSegments)),
|
|
|
|
zap.Int("No. growing segments", len(growingSegments)),
|
2021-09-07 15:41:59 +08:00
|
|
|
)
|
|
|
|
|
2022-07-07 18:26:20 +08:00
|
|
|
return dd, nil
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|