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
|
|
|
|
2024-07-01 14:46:07 +08:00
|
|
|
package pipeline
|
2021-01-19 11:37:16 +08:00
|
|
|
|
|
|
|
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"
|
2021-11-11 20:56:49 +08:00
|
|
|
"sync/atomic"
|
2021-06-07 13:58:37 +08:00
|
|
|
|
2023-03-04 23:21:50 +08:00
|
|
|
"go.opentelemetry.io/otel/trace"
|
2022-05-24 21:11:59 +08:00
|
|
|
"go.uber.org/zap"
|
2024-07-29 11:31:51 +08:00
|
|
|
"google.golang.org/protobuf/proto"
|
2022-05-24 21:11:59 +08:00
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
2024-06-14 14:34:01 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/compaction"
|
2024-08-02 18:30:23 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/flushcommon/util"
|
2021-06-07 13:58:37 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2024-08-02 18:30:23 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/streamingnode/server/flusher"
|
2021-10-07 19:54:56 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
2024-08-29 10:03:08 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/streamingutil"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
|
|
|
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
2024-08-29 10:03:08 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
2024-08-02 18:30:23 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
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-10-21 15:57:28 +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-10-21 15:57:28 +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
|
2022-10-21 15:57:28 +08:00
|
|
|
//
|
|
|
|
// 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
|
2024-08-02 18:30:23 +08:00
|
|
|
collectionID typeutil.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
|
|
|
dropMode atomic.Value
|
2024-06-14 14:34:01 +08:00
|
|
|
compactionExecutor compaction.Executor
|
2024-11-08 10:16:34 +08:00
|
|
|
msgHandler flusher.MsgHandler
|
2022-07-07 18:26:20 +08:00
|
|
|
|
|
|
|
// for recovery
|
2024-08-02 18:30:23 +08:00
|
|
|
growingSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo // segmentID
|
|
|
|
sealedSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo // segmentID
|
2022-07-07 18:26:20 +08:00
|
|
|
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 {
|
2024-07-04 16:34:17 +08:00
|
|
|
return fmt.Sprintf("ddNode-%s", ddn.vChannelName)
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2023-01-06 14:49:36 +08:00
|
|
|
func (ddn *ddNode) IsValidInMsg(in []Msg) bool {
|
|
|
|
if !ddn.BaseNode.IsValidInMsg(in) {
|
|
|
|
return false
|
2022-10-10 22:15:22 +08:00
|
|
|
}
|
2023-01-06 14:49:36 +08:00
|
|
|
_, ok := in[0].(*MsgStreamMsg)
|
|
|
|
if !ok {
|
|
|
|
log.Warn("type assertion failed for MsgStreamMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
|
|
|
return false
|
2021-05-31 15:28:30 +08:00
|
|
|
}
|
2023-01-06 14:49:36 +08:00
|
|
|
return true
|
|
|
|
}
|
2021-05-31 15:28:30 +08:00
|
|
|
|
2023-01-06 14:49:36 +08:00
|
|
|
// Operate handles input messages, implementing flowgrpah.Node
|
|
|
|
func (ddn *ddNode) Operate(in []Msg) []Msg {
|
2021-02-25 17:35:36 +08:00
|
|
|
msMsg, ok := in[0].(*MsgStreamMsg)
|
2021-01-19 11:37:16 +08:00
|
|
|
if !ok {
|
2024-05-25 04:43:41 +08:00
|
|
|
log.Warn("type assertion failed for MsgStreamMsg", zap.String("channel", ddn.vChannelName), 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
|
|
|
}
|
|
|
|
|
2023-01-06 14:49:36 +08:00
|
|
|
if msMsg.IsCloseMsg() {
|
2024-07-01 14:46:07 +08:00
|
|
|
fgMsg := FlowGraphMsg{
|
2023-01-06 14:49:36 +08:00
|
|
|
BaseMsg: flowgraph.NewBaseMsg(true),
|
2024-07-01 14:46:07 +08:00
|
|
|
InsertMessages: make([]*msgstream.InsertMsg, 0),
|
|
|
|
TimeRange: util.TimeRange{
|
|
|
|
TimestampMin: msMsg.TimestampMin(),
|
|
|
|
TimestampMax: msMsg.TimestampMax(),
|
2023-01-06 14:49:36 +08:00
|
|
|
},
|
2024-07-01 14:46:07 +08:00
|
|
|
StartPositions: msMsg.StartPositions(),
|
|
|
|
EndPositions: msMsg.EndPositions(),
|
2023-01-06 14:49:36 +08:00
|
|
|
dropCollection: false,
|
|
|
|
}
|
2024-05-25 04:43:41 +08:00
|
|
|
log.Warn("MsgStream closed", zap.Any("ddNode node", ddn.Name()), zap.String("channel", ddn.vChannelName), zap.Int64("collection", ddn.collectionID))
|
2023-01-06 14:49:36 +08:00
|
|
|
return []Msg{&fgMsg}
|
2021-06-30 16:18:13 +08:00
|
|
|
}
|
|
|
|
|
2021-11-11 20:56:49 +08:00
|
|
|
if load := ddn.dropMode.Load(); load != nil && load.(bool) {
|
2024-05-25 04:43:41 +08:00
|
|
|
log.RatedInfo(1.0, "ddNode in dropMode", zap.String("channel", ddn.vChannelName))
|
2021-11-11 20:56:49 +08:00
|
|
|
return []Msg{}
|
|
|
|
}
|
|
|
|
|
2023-01-12 16:09:39 +08:00
|
|
|
var spans []trace.Span
|
2023-01-06 14:49:36 +08:00
|
|
|
for _, msg := range msMsg.TsMessages() {
|
2024-07-01 14:46:07 +08:00
|
|
|
ctx, sp := util.StartTracer(msg, "DDNode-Operate")
|
2023-01-06 14:49:36 +08:00
|
|
|
spans = append(spans, sp)
|
|
|
|
msg.SetTraceCtx(ctx)
|
|
|
|
}
|
|
|
|
defer func() {
|
|
|
|
for _, sp := range spans {
|
2023-01-12 16:09:39 +08:00
|
|
|
sp.End()
|
2023-01-06 14:49:36 +08:00
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
2024-07-01 14:46:07 +08:00
|
|
|
fgMsg := FlowGraphMsg{
|
|
|
|
InsertMessages: make([]*msgstream.InsertMsg, 0),
|
|
|
|
TimeRange: util.TimeRange{
|
|
|
|
TimestampMin: msMsg.TimestampMin(),
|
|
|
|
TimestampMax: msMsg.TimestampMax(),
|
2021-01-19 11:37:16 +08:00
|
|
|
},
|
2024-07-01 14:46:07 +08:00
|
|
|
StartPositions: make([]*msgpb.MsgPosition, 0),
|
|
|
|
EndPositions: make([]*msgpb.MsgPosition, 0),
|
2021-11-11 20:56:49 +08:00
|
|
|
dropCollection: false,
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
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 {
|
2024-05-25 04:43:41 +08:00
|
|
|
log.Info("Receiving DropCollection msg", zap.String("channel", ddn.vChannelName))
|
2021-11-11 20:56:49 +08:00
|
|
|
ddn.dropMode.Store(true)
|
2021-12-02 16:39:33 +08:00
|
|
|
|
2024-05-25 04:43:41 +08:00
|
|
|
log.Info("Stop compaction for dropped channel", zap.String("channel", ddn.vChannelName))
|
2024-08-29 10:03:08 +08:00
|
|
|
if !streamingutil.IsStreamingServiceEnabled() {
|
|
|
|
ddn.compactionExecutor.DiscardByDroppedChannel(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 {
|
2024-05-25 04:43:41 +08:00
|
|
|
log.Info("drop partition msg received", zap.String("channel", ddn.vChannelName), zap.Int64("partitionID", dpMsg.GetPartitionID()))
|
2022-07-15 17:12:27 +08:00
|
|
|
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 {
|
2024-05-06 20:51:29 +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),
|
2024-05-25 04:43:41 +08:00
|
|
|
zap.String("channel", ddn.vChannelName),
|
2021-11-25 09:43:15 +08:00
|
|
|
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) {
|
2024-05-06 20:51:29 +08:00
|
|
|
log.Debug("filter insert messages",
|
2023-07-14 15:56:31 +08:00
|
|
|
zap.Int64("filter segmentID", imsg.GetSegmentID()),
|
2024-05-25 04:43:41 +08:00
|
|
|
zap.String("channel", ddn.vChannelName),
|
2022-07-07 18:26:20 +08:00
|
|
|
zap.Uint64("message timestamp", msg.EndTs()),
|
2024-05-13 18:21:32 +08:00
|
|
|
)
|
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
|
|
|
|
2024-08-02 18:30:23 +08:00
|
|
|
util.GetRateCollector().Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(imsg.InsertRequest)))
|
2022-11-07 10:15:02 +08:00
|
|
|
|
|
|
|
metrics.DataNodeConsumeBytesCount.
|
|
|
|
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel).
|
2024-07-30 15:53:51 +08:00
|
|
|
Add(float64(proto.Size(imsg.InsertRequest)))
|
2022-11-07 10:15:02 +08:00
|
|
|
|
|
|
|
metrics.DataNodeConsumeMsgCount.
|
|
|
|
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel, fmt.Sprint(ddn.collectionID)).
|
|
|
|
Inc()
|
2022-09-16 09:56:47 +08:00
|
|
|
|
2024-01-19 11:34:54 +08:00
|
|
|
metrics.DataNodeConsumeMsgRowsCount.
|
|
|
|
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel).
|
|
|
|
Add(float64(imsg.GetNumRows()))
|
|
|
|
|
2021-11-25 09:43:15 +08:00
|
|
|
log.Debug("DDNode receive insert messages",
|
2024-05-13 18:21:32 +08:00
|
|
|
zap.Int64("segmentID", imsg.GetSegmentID()),
|
2024-05-25 04:43:41 +08:00
|
|
|
zap.String("channel", ddn.vChannelName),
|
2024-07-06 08:50:11 +08:00
|
|
|
zap.Int("numRows", len(imsg.GetRowIDs())),
|
|
|
|
zap.Uint64("startPosTs", msMsg.StartPositions()[0].GetTimestamp()),
|
|
|
|
zap.Uint64("endPosTs", msMsg.EndPositions()[0].GetTimestamp()))
|
2024-07-01 14:46:07 +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)
|
2023-08-28 11:30:26 +08:00
|
|
|
|
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),
|
2024-05-25 04:43:41 +08:00
|
|
|
zap.String("channel", ddn.vChannelName),
|
2021-11-25 09:43:15 +08:00
|
|
|
zap.Int64("Expected collID", ddn.collectionID))
|
2021-10-11 16:31:44 +08:00
|
|
|
continue
|
|
|
|
}
|
2024-05-13 18:21:32 +08:00
|
|
|
|
2024-05-25 04:43:41 +08:00
|
|
|
log.Debug("DDNode receive delete messages", zap.String("channel", ddn.vChannelName), zap.Int64("numRows", dmsg.NumRows))
|
2024-08-02 18:30:23 +08:00
|
|
|
util.GetRateCollector().Add(metricsinfo.DeleteConsumeThroughput, float64(proto.Size(dmsg.DeleteRequest)))
|
2022-11-07 10:15:02 +08:00
|
|
|
|
|
|
|
metrics.DataNodeConsumeBytesCount.
|
|
|
|
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel).
|
2024-07-30 15:53:51 +08:00
|
|
|
Add(float64(proto.Size(dmsg.DeleteRequest)))
|
2022-11-07 10:15:02 +08:00
|
|
|
|
|
|
|
metrics.DataNodeConsumeMsgCount.
|
|
|
|
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel, fmt.Sprint(ddn.collectionID)).
|
|
|
|
Inc()
|
2024-01-19 11:34:54 +08:00
|
|
|
|
|
|
|
metrics.DataNodeConsumeMsgRowsCount.
|
|
|
|
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel).
|
|
|
|
Add(float64(dmsg.GetNumRows()))
|
2024-07-01 14:46:07 +08:00
|
|
|
fgMsg.DeleteMessages = append(fgMsg.DeleteMessages, dmsg)
|
2024-11-08 10:16:34 +08:00
|
|
|
case commonpb.MsgType_CreateSegment:
|
|
|
|
createSegment := msg.(*adaptor.CreateSegmentMessageBody)
|
|
|
|
logger := log.With(
|
|
|
|
zap.String("vchannel", ddn.Name()),
|
|
|
|
zap.Int32("msgType", int32(msg.Type())),
|
|
|
|
zap.Uint64("timetick", createSegment.CreateSegmentMessage.TimeTick()),
|
|
|
|
)
|
|
|
|
logger.Info("receive create segment message")
|
|
|
|
if err := ddn.msgHandler.HandleCreateSegment(context.Background(), ddn.vChannelName, createSegment.CreateSegmentMessage); err != nil {
|
|
|
|
logger.Warn("handle create segment message failed", zap.Error(err))
|
|
|
|
} else {
|
|
|
|
logger.Info("handle create segment message success")
|
|
|
|
}
|
2024-08-29 10:03:08 +08:00
|
|
|
case commonpb.MsgType_FlushSegment:
|
|
|
|
flushMsg := msg.(*adaptor.FlushMessageBody)
|
|
|
|
logger := log.With(
|
|
|
|
zap.String("vchannel", ddn.Name()),
|
|
|
|
zap.Int32("msgType", int32(msg.Type())),
|
|
|
|
zap.Uint64("timetick", flushMsg.FlushMessage.TimeTick()),
|
|
|
|
)
|
|
|
|
logger.Info("receive flush message")
|
2024-11-08 10:16:34 +08:00
|
|
|
if err := ddn.msgHandler.HandleFlush(ddn.vChannelName, flushMsg.FlushMessage); err != nil {
|
2024-08-29 10:03:08 +08:00
|
|
|
logger.Warn("handle flush message failed", zap.Error(err))
|
|
|
|
} else {
|
|
|
|
logger.Info("handle flush message success")
|
|
|
|
}
|
|
|
|
case commonpb.MsgType_ManualFlush:
|
|
|
|
manualFlushMsg := msg.(*adaptor.ManualFlushMessageBody)
|
|
|
|
logger := log.With(
|
|
|
|
zap.String("vchannel", ddn.Name()),
|
|
|
|
zap.Int32("msgType", int32(msg.Type())),
|
|
|
|
zap.Uint64("timetick", manualFlushMsg.ManualFlushMessage.TimeTick()),
|
|
|
|
zap.Uint64("flushTs", manualFlushMsg.ManualFlushMessage.Header().FlushTs),
|
|
|
|
)
|
|
|
|
logger.Info("receive manual flush message")
|
2024-11-08 10:16:34 +08:00
|
|
|
if err := ddn.msgHandler.HandleManualFlush(ddn.vChannelName, manualFlushMsg.ManualFlushMessage); err != nil {
|
2024-08-29 10:03:08 +08:00
|
|
|
logger.Warn("handle manual flush message failed", zap.Error(err))
|
|
|
|
} else {
|
|
|
|
logger.Info("handle manual flush message success")
|
2024-08-02 18:30:23 +08:00
|
|
|
}
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-07-01 14:46:07 +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-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
|
|
|
|
}
|
|
|
|
|
2024-08-02 18:30:23 +08:00
|
|
|
func (ddn *ddNode) isDropped(segID typeutil.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
|
|
|
|
}
|
|
|
|
|
2023-11-13 10:50:17 +08:00
|
|
|
func (ddn *ddNode) Close() {
|
|
|
|
log.Info("Flowgraph DD Node closing")
|
|
|
|
}
|
2021-11-05 11:59:02 +08:00
|
|
|
|
2024-08-02 18:30:23 +08:00
|
|
|
func newDDNode(ctx context.Context, collID typeutil.UniqueID, vChannelName string, droppedSegmentIDs []typeutil.UniqueID,
|
2024-11-08 10:16:34 +08:00
|
|
|
sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor, handler flusher.MsgHandler,
|
2024-11-14 11:00:31 +08:00
|
|
|
) *ddNode {
|
2021-01-19 11:37:16 +08:00
|
|
|
baseNode := BaseNode{}
|
2024-07-01 14:46:07 +08:00
|
|
|
baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
|
|
|
|
baseNode.SetMaxParallelism(paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
|
2021-01-19 11:37:16 +08:00
|
|
|
|
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,
|
2024-08-02 18:30:23 +08:00
|
|
|
sealedSegInfo: make(map[typeutil.UniqueID]*datapb.SegmentInfo, len(sealedSegments)),
|
|
|
|
growingSegInfo: make(map[typeutil.UniqueID]*datapb.SegmentInfo, len(growingSegments)),
|
2022-07-07 18:26:20 +08:00
|
|
|
droppedSegmentIDs: droppedSegmentIDs,
|
|
|
|
vChannelName: vChannelName,
|
2024-06-14 14:34:01 +08:00
|
|
|
compactionExecutor: executor,
|
2024-11-08 10:16:34 +08:00
|
|
|
msgHandler: handler,
|
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
|
|
|
)
|
|
|
|
|
2024-11-14 11:00:31 +08:00
|
|
|
return dd
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|