2023-03-27 00:42:00 +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 pipeline
|
|
|
|
|
|
|
|
import (
|
|
|
|
"fmt"
|
|
|
|
"sync"
|
|
|
|
|
2023-04-06 19:14:32 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/delegator"
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/segments"
|
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/msgdispatcher"
|
2023-04-26 17:06:34 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2023-03-27 00:42:00 +08:00
|
|
|
)
|
|
|
|
|
2023-04-26 17:06:34 +08:00
|
|
|
// Manager manage pipeline in querynode
|
2023-03-27 00:42:00 +08:00
|
|
|
type Manager interface {
|
|
|
|
Num() int
|
|
|
|
Add(collectionID UniqueID, channel string) (Pipeline, error)
|
|
|
|
Get(channel string) Pipeline
|
|
|
|
Remove(channels ...string)
|
|
|
|
Start(channels ...string) error
|
|
|
|
Close()
|
|
|
|
}
|
2023-04-06 19:14:32 +08:00
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
type manager struct {
|
|
|
|
channel2Pipeline map[string]Pipeline
|
|
|
|
dataManager *DataManager
|
|
|
|
delegators *typeutil.ConcurrentMap[string, delegator.ShardDelegator]
|
|
|
|
|
|
|
|
tSafeManager TSafeManager
|
|
|
|
dispatcher msgdispatcher.Client
|
2023-05-16 15:17:23 +08:00
|
|
|
mu sync.RWMutex
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (m *manager) Num() int {
|
2023-05-16 15:17:23 +08:00
|
|
|
m.mu.RLock()
|
|
|
|
defer m.mu.RUnlock()
|
2023-03-27 00:42:00 +08:00
|
|
|
return len(m.channel2Pipeline)
|
|
|
|
}
|
|
|
|
|
2023-04-26 17:06:34 +08:00
|
|
|
// Add pipeline for each channel of collection
|
2023-03-27 00:42:00 +08:00
|
|
|
func (m *manager) Add(collectionID UniqueID, channel string) (Pipeline, error) {
|
|
|
|
m.mu.Lock()
|
|
|
|
defer m.mu.Unlock()
|
|
|
|
|
|
|
|
log.Debug("start create pipeine",
|
|
|
|
zap.Int64("collectionID", collectionID),
|
|
|
|
zap.String("channel", channel),
|
|
|
|
)
|
|
|
|
collection := m.dataManager.Collection.Get(collectionID)
|
|
|
|
if collection == nil {
|
|
|
|
return nil, segments.WrapCollectionNotFound(collectionID)
|
|
|
|
}
|
|
|
|
|
|
|
|
if pipeline, ok := m.channel2Pipeline[channel]; ok {
|
|
|
|
return pipeline, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
//get shard delegator for add growing in pipeline
|
|
|
|
delegator, ok := m.delegators.Get(channel)
|
|
|
|
if !ok {
|
2023-04-26 17:06:34 +08:00
|
|
|
return nil, merr.WrapErrShardDelegatorNotFound(channel)
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
newPipeLine, err := NewPipeLine(collectionID, channel, m.dataManager, m.tSafeManager, m.dispatcher, delegator)
|
|
|
|
if err != nil {
|
2023-04-26 17:06:34 +08:00
|
|
|
return nil, merr.WrapErrServiceUnavailable(err.Error(), "failed to create new pipeline")
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
m.channel2Pipeline[channel] = newPipeLine
|
|
|
|
metrics.QueryNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
|
|
|
metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
|
|
|
return newPipeLine, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (m *manager) Get(channel string) Pipeline {
|
|
|
|
m.mu.Lock()
|
|
|
|
defer m.mu.Unlock()
|
|
|
|
|
|
|
|
pipeline, ok := m.channel2Pipeline[channel]
|
|
|
|
if !ok {
|
|
|
|
log.Warn("pipeline not existed",
|
|
|
|
zap.String("channel", channel),
|
|
|
|
)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
return pipeline
|
|
|
|
}
|
|
|
|
|
2023-04-26 17:06:34 +08:00
|
|
|
// Remove pipeline from Manager by channel
|
2023-03-27 00:42:00 +08:00
|
|
|
func (m *manager) Remove(channels ...string) {
|
|
|
|
m.mu.Lock()
|
|
|
|
defer m.mu.Unlock()
|
|
|
|
|
|
|
|
for _, channel := range channels {
|
|
|
|
if pipeline, ok := m.channel2Pipeline[channel]; ok {
|
|
|
|
pipeline.Close()
|
|
|
|
delete(m.channel2Pipeline, channel)
|
|
|
|
} else {
|
|
|
|
log.Warn("pipeline to be removed doesn't existed", zap.Any("channel", channel))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
metrics.QueryNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec()
|
|
|
|
metrics.QueryNodeNumDmlChannels.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec()
|
|
|
|
}
|
|
|
|
|
2023-04-26 17:06:34 +08:00
|
|
|
// Start pipeline by channel
|
2023-03-27 00:42:00 +08:00
|
|
|
func (m *manager) Start(channels ...string) error {
|
|
|
|
m.mu.Lock()
|
|
|
|
defer m.mu.Unlock()
|
|
|
|
|
|
|
|
//check pipelie all exist before start
|
|
|
|
for _, channel := range channels {
|
|
|
|
if _, ok := m.channel2Pipeline[channel]; !ok {
|
2023-04-26 17:06:34 +08:00
|
|
|
reason := fmt.Sprintf("pipeline with channel %s not exist", channel)
|
|
|
|
return merr.WrapErrServiceUnavailable(reason, "pipine start failed")
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, channel := range channels {
|
|
|
|
m.channel2Pipeline[channel].Start()
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2023-04-26 17:06:34 +08:00
|
|
|
// Close all pipeline of Manager
|
2023-03-27 00:42:00 +08:00
|
|
|
func (m *manager) Close() {
|
|
|
|
m.mu.Lock()
|
|
|
|
defer m.mu.Unlock()
|
|
|
|
for _, pipeline := range m.channel2Pipeline {
|
|
|
|
pipeline.Close()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func NewManager(dataManager *DataManager,
|
|
|
|
tSafeManager TSafeManager,
|
|
|
|
dispatcher msgdispatcher.Client,
|
|
|
|
delegators *typeutil.ConcurrentMap[string, delegator.ShardDelegator],
|
|
|
|
) Manager {
|
|
|
|
return &manager{
|
|
|
|
channel2Pipeline: make(map[string]Pipeline),
|
|
|
|
dataManager: dataManager,
|
|
|
|
delegators: delegators,
|
|
|
|
tSafeManager: tSafeManager,
|
|
|
|
dispatcher: dispatcher,
|
|
|
|
}
|
|
|
|
}
|