2022-01-13 14:21:34 +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
|
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
//
|
|
|
|
// 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.
|
|
|
|
|
|
|
|
package datanode
|
|
|
|
|
|
|
|
import (
|
|
|
|
"fmt"
|
2023-03-21 21:37:56 +08:00
|
|
|
"sort"
|
2022-01-13 14:21:34 +08:00
|
|
|
"sync"
|
2023-03-21 21:37:56 +08:00
|
|
|
"time"
|
|
|
|
|
|
|
|
"go.uber.org/zap"
|
2022-01-13 14:21:34 +08:00
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
2022-01-13 14:21:34 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
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/util/hardware"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
2023-07-24 10:21:07 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2022-01-13 14:21:34 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
type flowgraphManager struct {
|
2023-07-24 10:21:07 +08:00
|
|
|
flowgraphs *typeutil.ConcurrentMap[string, *dataSyncService]
|
2023-03-21 21:37:56 +08:00
|
|
|
|
|
|
|
closeCh chan struct{}
|
|
|
|
closeOnce sync.Once
|
2022-01-13 14:21:34 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func newFlowgraphManager() *flowgraphManager {
|
2023-03-21 21:37:56 +08:00
|
|
|
return &flowgraphManager{
|
2023-07-24 10:21:07 +08:00
|
|
|
flowgraphs: typeutil.NewConcurrentMap[string, *dataSyncService](),
|
|
|
|
closeCh: make(chan struct{}),
|
2023-03-21 21:37:56 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fm *flowgraphManager) start() {
|
|
|
|
ticker := time.NewTicker(3 * time.Second)
|
|
|
|
defer ticker.Stop()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-fm.closeCh:
|
|
|
|
return
|
|
|
|
case <-ticker.C:
|
|
|
|
fm.execute(hardware.GetMemoryCount())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-07-28 10:11:08 +08:00
|
|
|
func (fm *flowgraphManager) close() {
|
|
|
|
fm.dropAll()
|
2023-03-21 21:37:56 +08:00
|
|
|
fm.closeOnce.Do(func() {
|
|
|
|
close(fm.closeCh)
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fm *flowgraphManager) execute(totalMemory uint64) {
|
|
|
|
if !Params.DataNodeCfg.MemoryForceSyncEnable.GetAsBool() {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
var total int64
|
|
|
|
channels := make([]struct {
|
|
|
|
channel string
|
|
|
|
bufferSize int64
|
|
|
|
}, 0)
|
2023-07-24 10:21:07 +08:00
|
|
|
fm.flowgraphs.Range(func(key string, value *dataSyncService) bool {
|
|
|
|
size := value.channel.getTotalMemorySize()
|
2023-03-21 21:37:56 +08:00
|
|
|
channels = append(channels, struct {
|
|
|
|
channel string
|
|
|
|
bufferSize int64
|
2023-07-24 10:21:07 +08:00
|
|
|
}{key, size})
|
2023-03-21 21:37:56 +08:00
|
|
|
total += size
|
|
|
|
return true
|
|
|
|
})
|
|
|
|
if len(channels) == 0 {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2023-08-30 17:13:00 +08:00
|
|
|
toMB := func(mem float64) float64 {
|
|
|
|
return mem / 1024 / 1024
|
|
|
|
}
|
|
|
|
|
2023-08-08 16:49:08 +08:00
|
|
|
memoryWatermark := float64(totalMemory) * Params.DataNodeCfg.MemoryWatermark.GetAsFloat()
|
|
|
|
if float64(total) < memoryWatermark {
|
|
|
|
log.RatedDebug(5, "skip force sync because memory level is not high enough",
|
2023-08-30 17:13:00 +08:00
|
|
|
zap.Float64("current_total_memory_usage", toMB(float64(total))),
|
|
|
|
zap.Float64("current_memory_watermark", toMB(memoryWatermark)),
|
2023-08-08 16:49:08 +08:00
|
|
|
zap.Any("channel_memory_usages", channels))
|
2023-03-21 21:37:56 +08:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
sort.Slice(channels, func(i, j int) bool {
|
|
|
|
return channels[i].bufferSize > channels[j].bufferSize
|
|
|
|
})
|
2023-07-24 10:21:07 +08:00
|
|
|
if fg, ok := fm.flowgraphs.Get(channels[0].channel); ok { // sync the first channel with the largest memory usage
|
2023-09-12 21:07:19 +08:00
|
|
|
fg.channel.setIsHighMemory(true)
|
2023-03-21 21:37:56 +08:00
|
|
|
log.Info("notify flowgraph to sync",
|
|
|
|
zap.String("channel", channels[0].channel), zap.Int64("bufferSize", channels[0].bufferSize))
|
|
|
|
}
|
2022-01-13 14:21:34 +08:00
|
|
|
}
|
|
|
|
|
2023-02-15 16:20:34 +08:00
|
|
|
func (fm *flowgraphManager) addAndStart(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *tickler) error {
|
2023-03-23 19:43:57 +08:00
|
|
|
log := log.With(zap.String("channel", vchan.GetChannelName()))
|
2023-07-24 10:21:07 +08:00
|
|
|
if fm.flowgraphs.Contain(vchan.GetChannelName()) {
|
2023-03-23 19:43:57 +08:00
|
|
|
log.Warn("try to add an existed DataSyncService")
|
2022-01-13 14:21:34 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2022-10-18 15:33:26 +08:00
|
|
|
channel := newChannel(vchan.GetChannelName(), vchan.GetCollectionID(), schema, dn.rootCoord, dn.chunkManager)
|
2022-01-13 14:21:34 +08:00
|
|
|
|
2022-10-18 15:33:26 +08:00
|
|
|
dataSyncService, err := newDataSyncService(dn.ctx, make(chan flushMsg, 100), make(chan resendTTMsg, 100), channel,
|
2023-06-14 14:16:38 +08:00
|
|
|
dn.allocator, dn.dispClient, dn.factory, vchan, dn.clearSignal, dn.dataCoord, dn.segmentCache, dn.chunkManager, dn.compactionExecutor, tickler, dn.GetSession().ServerID, dn.timeTickSender)
|
2022-01-13 14:21:34 +08:00
|
|
|
if err != nil {
|
2023-03-23 19:43:57 +08:00
|
|
|
log.Warn("fail to create new datasyncservice", zap.Error(err))
|
2022-01-13 14:21:34 +08:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
dataSyncService.start()
|
2023-07-24 10:21:07 +08:00
|
|
|
fm.flowgraphs.Insert(vchan.GetChannelName(), dataSyncService)
|
2022-02-28 19:11:55 +08:00
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
2022-01-13 14:21:34 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fm *flowgraphManager) release(vchanName string) {
|
2023-07-24 10:21:07 +08:00
|
|
|
if fg, loaded := fm.flowgraphs.GetAndRemove(vchanName); loaded {
|
|
|
|
fg.close()
|
2022-11-04 14:25:38 +08:00
|
|
|
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec()
|
2022-01-13 14:21:34 +08:00
|
|
|
}
|
2022-09-16 09:56:47 +08:00
|
|
|
rateCol.removeFlowGraphChannel(vchanName)
|
2022-01-13 14:21:34 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (fm *flowgraphManager) getFlushCh(segID UniqueID) (chan<- flushMsg, error) {
|
2022-09-23 10:22:52 +08:00
|
|
|
var flushCh chan flushMsg
|
2022-01-13 14:21:34 +08:00
|
|
|
|
2023-07-24 10:21:07 +08:00
|
|
|
fm.flowgraphs.Range(func(key string, fg *dataSyncService) bool {
|
2022-10-18 15:33:26 +08:00
|
|
|
if fg.channel.hasSegment(segID, true) {
|
2022-01-13 14:21:34 +08:00
|
|
|
flushCh = fg.flushCh
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
return true
|
|
|
|
})
|
|
|
|
|
2022-06-23 09:58:14 +08:00
|
|
|
if flushCh != nil {
|
2022-01-13 14:21:34 +08:00
|
|
|
return flushCh, nil
|
|
|
|
}
|
|
|
|
|
2023-03-28 19:04:00 +08:00
|
|
|
return nil, merr.WrapErrSegmentNotFound(segID, "failed to get flush channel has this segment")
|
2022-01-13 14:21:34 +08:00
|
|
|
}
|
|
|
|
|
2022-10-18 15:33:26 +08:00
|
|
|
func (fm *flowgraphManager) getChannel(segID UniqueID) (Channel, error) {
|
2022-09-23 10:22:52 +08:00
|
|
|
var (
|
2022-10-18 15:33:26 +08:00
|
|
|
rep Channel
|
2022-09-23 10:22:52 +08:00
|
|
|
exists = false
|
|
|
|
)
|
2023-07-24 10:21:07 +08:00
|
|
|
fm.flowgraphs.Range(func(key string, fg *dataSyncService) bool {
|
2022-10-18 15:33:26 +08:00
|
|
|
if fg.channel.hasSegment(segID, true) {
|
2022-09-23 10:22:52 +08:00
|
|
|
exists = true
|
2022-10-18 15:33:26 +08:00
|
|
|
rep = fg.channel
|
2022-09-23 10:22:52 +08:00
|
|
|
return false
|
|
|
|
}
|
|
|
|
return true
|
|
|
|
})
|
|
|
|
|
|
|
|
if exists {
|
|
|
|
return rep, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil, fmt.Errorf("cannot find segment %d in all flowgraphs", segID)
|
|
|
|
}
|
|
|
|
|
2022-11-03 14:05:36 +08:00
|
|
|
// resendTT loops through flow graphs, looks for segments that are not flushed,
|
|
|
|
// and sends them to that flow graph's `resendTTCh` channel so stats of
|
|
|
|
// these segments will be resent.
|
2022-05-25 14:34:00 +08:00
|
|
|
func (fm *flowgraphManager) resendTT() []UniqueID {
|
|
|
|
var unFlushedSegments []UniqueID
|
2023-07-24 10:21:07 +08:00
|
|
|
fm.flowgraphs.Range(func(key string, fg *dataSyncService) bool {
|
2022-10-18 15:33:26 +08:00
|
|
|
segIDs := fg.channel.listNotFlushedSegmentIDs()
|
2022-05-25 14:34:00 +08:00
|
|
|
if len(segIDs) > 0 {
|
|
|
|
log.Info("un-flushed segments found, stats will be resend",
|
|
|
|
zap.Int64s("segment IDs", segIDs))
|
|
|
|
unFlushedSegments = append(unFlushedSegments, segIDs...)
|
|
|
|
fg.resendTTCh <- resendTTMsg{
|
|
|
|
segmentIDs: segIDs,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return true
|
|
|
|
})
|
|
|
|
return unFlushedSegments
|
|
|
|
}
|
|
|
|
|
2022-01-13 14:21:34 +08:00
|
|
|
func (fm *flowgraphManager) getFlowgraphService(vchan string) (*dataSyncService, bool) {
|
2023-07-24 10:21:07 +08:00
|
|
|
return fm.flowgraphs.Get(vchan)
|
2022-01-13 14:21:34 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (fm *flowgraphManager) exist(vchan string) bool {
|
|
|
|
_, exist := fm.getFlowgraphService(vchan)
|
|
|
|
return exist
|
|
|
|
}
|
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
// getFlowGraphNum returns number of flow graphs.
|
|
|
|
func (fm *flowgraphManager) getFlowGraphNum() int {
|
2023-07-24 10:21:07 +08:00
|
|
|
return fm.flowgraphs.Len()
|
2022-09-16 09:56:47 +08:00
|
|
|
}
|
|
|
|
|
2022-01-13 14:21:34 +08:00
|
|
|
func (fm *flowgraphManager) dropAll() {
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Info("start drop all flowgraph resources in DataNode")
|
2023-07-24 10:21:07 +08:00
|
|
|
fm.flowgraphs.Range(func(key string, value *dataSyncService) bool {
|
2023-08-21 11:16:20 +08:00
|
|
|
value.GracefullyClose()
|
2023-07-24 10:21:07 +08:00
|
|
|
fm.flowgraphs.GetAndRemove(key)
|
2022-01-13 14:21:34 +08:00
|
|
|
|
2023-07-24 10:21:07 +08:00
|
|
|
log.Info("successfully dropped flowgraph", zap.String("vChannelName", key))
|
2022-01-13 14:21:34 +08:00
|
|
|
return true
|
|
|
|
})
|
|
|
|
}
|