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 querynodev2
|
|
|
|
|
|
|
|
/*
|
|
|
|
#cgo pkg-config: milvus_segcore milvus_common
|
|
|
|
|
|
|
|
#include "segcore/collection_c.h"
|
|
|
|
#include "segcore/segment_c.h"
|
|
|
|
#include "segcore/segcore_init_c.h"
|
|
|
|
#include "common/init_c.h"
|
|
|
|
|
|
|
|
*/
|
|
|
|
import "C"
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"fmt"
|
|
|
|
"os"
|
|
|
|
"path"
|
2023-05-26 15:07:26 +08:00
|
|
|
"path/filepath"
|
2023-04-06 14:32:29 +08:00
|
|
|
"plugin"
|
2023-03-27 00:42:00 +08:00
|
|
|
"runtime/debug"
|
2023-07-03 15:18:24 +08:00
|
|
|
"strings"
|
2023-03-27 00:42:00 +08:00
|
|
|
"sync"
|
|
|
|
"time"
|
|
|
|
"unsafe"
|
|
|
|
|
2023-09-21 09:45:27 +08:00
|
|
|
"github.com/samber/lo"
|
2023-03-27 00:42:00 +08:00
|
|
|
clientv3 "go.etcd.io/etcd/client/v3"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
2023-03-27 00:42:00 +08:00
|
|
|
grpcquerynodeclient "github.com/milvus-io/milvus/internal/distributed/querynode/client"
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/cluster"
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/delegator"
|
2023-09-28 10:01:26 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/optimizers"
|
2023-03-27 00:42:00 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/pipeline"
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/segments"
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/tasks"
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/tsafe"
|
2023-10-19 11:10:07 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/registry"
|
2023-03-27 00:42:00 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/storage"
|
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/dependency"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/initcore"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/config"
|
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
2023-06-26 17:52:44 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
2023-12-22 18:36:44 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/expr"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/gc"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/hardware"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/lifetime"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2023-03-27 00:42:00 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
// make sure QueryNode implements types.QueryNode
|
|
|
|
var _ types.QueryNode = (*QueryNode)(nil)
|
|
|
|
|
|
|
|
// make sure QueryNode implements types.QueryNodeComponent
|
|
|
|
var _ types.QueryNodeComponent = (*QueryNode)(nil)
|
|
|
|
|
|
|
|
// QueryNode communicates with outside services and union all
|
|
|
|
// services in querynode package.
|
|
|
|
//
|
|
|
|
// QueryNode implements `types.Component`, `types.QueryNode` interfaces.
|
|
|
|
//
|
|
|
|
// `rootCoord` is a grpc client of root coordinator.
|
|
|
|
// `indexCoord` is a grpc client of index coordinator.
|
|
|
|
// `stateCode` is current statement of this query node, indicating whether it's healthy.
|
|
|
|
type QueryNode struct {
|
|
|
|
ctx context.Context
|
|
|
|
cancel context.CancelFunc
|
|
|
|
|
|
|
|
lifetime lifetime.Lifetime[commonpb.StateCode]
|
|
|
|
|
|
|
|
// call once
|
2023-05-12 09:51:25 +08:00
|
|
|
initOnce sync.Once
|
|
|
|
startOnce sync.Once
|
|
|
|
stopOnce sync.Once
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
// internal components
|
2023-08-23 10:10:22 +08:00
|
|
|
manager *segments.Manager
|
|
|
|
clusterManager cluster.Manager
|
|
|
|
tSafeManager tsafe.Manager
|
|
|
|
pipelineManager pipeline.Manager
|
|
|
|
subscribingChannels *typeutil.ConcurrentSet[string]
|
|
|
|
unsubscribingChannels *typeutil.ConcurrentSet[string]
|
|
|
|
delegators *typeutil.ConcurrentMap[string, delegator.ShardDelegator]
|
2024-02-21 11:54:53 +08:00
|
|
|
serverID int64
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
// segment loader
|
|
|
|
loader segments.Loader
|
|
|
|
|
|
|
|
// Search/Query
|
2023-07-03 18:24:25 +08:00
|
|
|
scheduler tasks.Scheduler
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
// etcd client
|
|
|
|
etcdCli *clientv3.Client
|
|
|
|
address string
|
|
|
|
|
|
|
|
dispClient msgdispatcher.Client
|
|
|
|
factory dependency.Factory
|
|
|
|
|
|
|
|
session *sessionutil.Session
|
|
|
|
eventCh <-chan *sessionutil.SessionEvent
|
|
|
|
|
2023-11-30 18:00:33 +08:00
|
|
|
chunkManager storage.ChunkManager
|
2023-03-27 00:42:00 +08:00
|
|
|
|
2023-04-19 23:56:31 +08:00
|
|
|
/*
|
|
|
|
// Pool for search/query
|
|
|
|
knnPool *conc.Pool*/
|
2023-04-06 14:32:29 +08:00
|
|
|
|
|
|
|
// parameter turning hook
|
2023-09-28 10:01:26 +08:00
|
|
|
queryHook optimizers.QueryHook
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// NewQueryNode will return a QueryNode with abnormal state.
|
|
|
|
func NewQueryNode(ctx context.Context, factory dependency.Factory) *QueryNode {
|
2023-04-06 14:32:29 +08:00
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
2023-03-27 00:42:00 +08:00
|
|
|
node := &QueryNode{
|
2023-04-06 14:32:29 +08:00
|
|
|
ctx: ctx,
|
2023-03-27 00:42:00 +08:00
|
|
|
cancel: cancel,
|
|
|
|
factory: factory,
|
|
|
|
lifetime: lifetime.NewLifetime(commonpb.StateCode_Abnormal),
|
|
|
|
}
|
|
|
|
|
|
|
|
node.tSafeManager = tsafe.NewTSafeReplica()
|
2023-12-22 18:36:44 +08:00
|
|
|
expr.Register("querynode", node)
|
2023-03-27 00:42:00 +08:00
|
|
|
return node
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) initSession() error {
|
2023-09-25 21:39:27 +08:00
|
|
|
minimalIndexVersion, currentIndexVersion := getIndexEngineVersion()
|
2023-10-27 07:36:12 +08:00
|
|
|
node.session = sessionutil.NewSession(node.ctx, sessionutil.WithIndexEngineVersion(minimalIndexVersion, currentIndexVersion))
|
2023-03-27 00:42:00 +08:00
|
|
|
if node.session == nil {
|
|
|
|
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
|
|
|
}
|
|
|
|
node.session.Init(typeutil.QueryNodeRole, node.address, false, true)
|
2023-10-16 10:24:10 +08:00
|
|
|
sessionutil.SaveServerInfo(typeutil.QueryNodeRole, node.session.ServerID)
|
2023-03-27 00:42:00 +08:00
|
|
|
paramtable.SetNodeID(node.session.ServerID)
|
2024-02-21 11:54:53 +08:00
|
|
|
node.serverID = node.session.ServerID
|
|
|
|
log.Info("QueryNode init session", zap.Int64("nodeID", node.GetNodeID()), zap.String("node address", node.session.Address))
|
2023-03-27 00:42:00 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Register register query node at etcd
|
|
|
|
func (node *QueryNode) Register() error {
|
|
|
|
node.session.Register()
|
|
|
|
// start liveness check
|
2024-02-21 11:54:53 +08:00
|
|
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(node.GetNodeID()), typeutil.QueryNodeRole).Inc()
|
2023-04-12 20:12:28 +08:00
|
|
|
node.session.LivenessCheck(node.ctx, func() {
|
2024-02-29 17:01:50 +08:00
|
|
|
log.Error("Query Node disconnected from etcd, process will exit", zap.Int64("Server Id", paramtable.GetNodeID()))
|
|
|
|
os.Exit(1)
|
2023-03-27 00:42:00 +08:00
|
|
|
})
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// InitSegcore set init params of segCore, such as chunckRows, SIMD type...
|
2023-06-25 14:38:44 +08:00
|
|
|
func (node *QueryNode) InitSegcore() error {
|
2023-09-05 10:31:48 +08:00
|
|
|
cGlogConf := C.CString(path.Join(paramtable.GetBaseTable().GetConfigDir(), paramtable.DefaultGlogConf))
|
2023-07-27 19:49:02 +08:00
|
|
|
C.SegcoreInit(cGlogConf)
|
|
|
|
C.free(unsafe.Pointer(cGlogConf))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
// override segcore chunk size
|
|
|
|
cChunkRows := C.int64_t(paramtable.Get().QueryNodeCfg.ChunkRows.GetAsInt64())
|
|
|
|
C.SegcoreSetChunkRows(cChunkRows)
|
|
|
|
|
2023-04-17 10:44:29 +08:00
|
|
|
cKnowhereThreadPoolSize := C.uint32_t(paramtable.Get().QueryNodeCfg.KnowhereThreadPoolSize.GetAsUint32())
|
2023-08-18 14:00:19 +08:00
|
|
|
C.SegcoreSetKnowhereSearchThreadPoolNum(cKnowhereThreadPoolSize)
|
2023-04-17 10:44:29 +08:00
|
|
|
|
2023-11-01 02:20:15 +08:00
|
|
|
enableGrowingIndex := C.bool(paramtable.Get().QueryNodeCfg.EnableTempSegmentIndex.GetAsBool())
|
|
|
|
C.SegcoreSetEnableTempSegmentIndex(enableGrowingIndex)
|
2023-04-26 10:14:41 +08:00
|
|
|
|
2023-11-01 02:20:15 +08:00
|
|
|
nlist := C.int64_t(paramtable.Get().QueryNodeCfg.InterimIndexNlist.GetAsInt64())
|
2023-03-27 00:42:00 +08:00
|
|
|
C.SegcoreSetNlist(nlist)
|
|
|
|
|
2023-11-01 02:20:15 +08:00
|
|
|
nprobe := C.int64_t(paramtable.Get().QueryNodeCfg.InterimIndexNProbe.GetAsInt64())
|
2023-03-27 00:42:00 +08:00
|
|
|
C.SegcoreSetNprobe(nprobe)
|
|
|
|
|
|
|
|
// override segcore SIMD type
|
|
|
|
cSimdType := C.CString(paramtable.Get().CommonCfg.SimdType.GetValue())
|
|
|
|
C.SegcoreSetSimdType(cSimdType)
|
|
|
|
C.free(unsafe.Pointer(cSimdType))
|
|
|
|
|
|
|
|
// override segcore index slice size
|
|
|
|
cIndexSliceSize := C.int64_t(paramtable.Get().CommonCfg.IndexSliceSize.GetAsInt64())
|
|
|
|
C.InitIndexSliceSize(cIndexSliceSize)
|
|
|
|
|
2023-09-21 09:45:27 +08:00
|
|
|
// set up thread pool for different priorities
|
2023-08-03 09:31:07 +08:00
|
|
|
cHighPriorityThreadCoreCoefficient := C.int64_t(paramtable.Get().CommonCfg.HighPriorityThreadCoreCoefficient.GetAsInt64())
|
|
|
|
C.InitHighPriorityThreadCoreCoefficient(cHighPriorityThreadCoreCoefficient)
|
|
|
|
cMiddlePriorityThreadCoreCoefficient := C.int64_t(paramtable.Get().CommonCfg.MiddlePriorityThreadCoreCoefficient.GetAsInt64())
|
|
|
|
C.InitMiddlePriorityThreadCoreCoefficient(cMiddlePriorityThreadCoreCoefficient)
|
|
|
|
cLowPriorityThreadCoreCoefficient := C.int64_t(paramtable.Get().CommonCfg.LowPriorityThreadCoreCoefficient.GetAsInt64())
|
|
|
|
C.InitLowPriorityThreadCoreCoefficient(cLowPriorityThreadCoreCoefficient)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
cCPUNum := C.int(hardware.GetCPUNum())
|
|
|
|
C.InitCpuNum(cCPUNum)
|
|
|
|
|
2023-12-18 12:04:42 +08:00
|
|
|
cExprBatchSize := C.int64_t(paramtable.Get().QueryNodeCfg.ExprEvalBatchSize.GetAsInt64())
|
|
|
|
C.InitDefaultExprEvalBatchSize(cExprBatchSize)
|
|
|
|
|
2023-12-26 11:04:47 +08:00
|
|
|
cGpuMemoryPoolInitSize := C.uint32_t(paramtable.Get().GpuConfig.InitSize.GetAsUint32())
|
|
|
|
cGpuMemoryPoolMaxSize := C.uint32_t(paramtable.Get().GpuConfig.MaxSize.GetAsUint32())
|
|
|
|
C.SegcoreSetKnowhereGpuMemoryPoolSize(cGpuMemoryPoolInitSize, cGpuMemoryPoolMaxSize)
|
|
|
|
|
2023-05-26 15:07:26 +08:00
|
|
|
localDataRootPath := filepath.Join(paramtable.Get().LocalStorageCfg.Path.GetValue(), typeutil.QueryNodeRole)
|
2023-06-25 14:38:44 +08:00
|
|
|
initcore.InitLocalChunkManager(localDataRootPath)
|
2023-03-28 21:30:05 +08:00
|
|
|
|
2023-09-15 10:21:20 +08:00
|
|
|
err := initcore.InitRemoteChunkManager(paramtable.Get())
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
mmapDirPath := paramtable.Get().QueryNodeCfg.MmapDirPath.GetValue()
|
|
|
|
if len(mmapDirPath) == 0 {
|
2023-12-21 18:12:45 +08:00
|
|
|
paramtable.Get().Save(
|
|
|
|
paramtable.Get().QueryNodeCfg.MmapDirPath.Key,
|
|
|
|
path.Join(paramtable.Get().LocalStorageCfg.Path.GetValue(), "mmap"),
|
|
|
|
)
|
|
|
|
mmapDirPath = paramtable.Get().QueryNodeCfg.MmapDirPath.GetValue()
|
2023-09-15 10:21:20 +08:00
|
|
|
}
|
2023-09-30 17:05:27 +08:00
|
|
|
chunkCachePath := path.Join(mmapDirPath, "chunk_cache")
|
2023-09-28 15:47:27 +08:00
|
|
|
policy := paramtable.Get().QueryNodeCfg.ReadAheadPolicy.GetValue()
|
2023-09-30 17:05:27 +08:00
|
|
|
err = initcore.InitChunkCache(chunkCachePath, policy)
|
2023-09-15 10:21:20 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2023-09-30 17:05:27 +08:00
|
|
|
log.Info("InitChunkCache done", zap.String("dir", chunkCachePath), zap.String("policy", policy))
|
2023-09-15 10:21:20 +08:00
|
|
|
|
2023-03-29 18:10:02 +08:00
|
|
|
initcore.InitTraceConfig(paramtable.Get())
|
2023-09-15 10:21:20 +08:00
|
|
|
return nil
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
|
2023-09-25 21:39:27 +08:00
|
|
|
func getIndexEngineVersion() (minimal, current int32) {
|
|
|
|
cMinimal, cCurrent := C.GetMinimalIndexVersion(), C.GetCurrentIndexVersion()
|
|
|
|
return int32(cMinimal), int32(cCurrent)
|
|
|
|
}
|
|
|
|
|
2024-02-21 11:54:53 +08:00
|
|
|
func (node *QueryNode) GetNodeID() int64 {
|
|
|
|
return node.serverID
|
|
|
|
}
|
|
|
|
|
2023-07-27 19:49:02 +08:00
|
|
|
func (node *QueryNode) CloseSegcore() {
|
|
|
|
// safe stop
|
|
|
|
initcore.CleanRemoteChunkManager()
|
|
|
|
initcore.CleanGlogManager()
|
|
|
|
}
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
// Init function init historical and streaming module to manage segments
|
|
|
|
func (node *QueryNode) Init() error {
|
|
|
|
var initError error
|
|
|
|
node.initOnce.Do(func() {
|
|
|
|
// ctx := context.Background()
|
|
|
|
log.Info("QueryNode session info", zap.String("metaPath", paramtable.Get().EtcdCfg.MetaRootPath.GetValue()))
|
|
|
|
err := node.initSession()
|
|
|
|
if err != nil {
|
|
|
|
log.Error("QueryNode init session failed", zap.Error(err))
|
|
|
|
initError = err
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2023-04-06 14:32:29 +08:00
|
|
|
err = node.initHook()
|
|
|
|
if err != nil {
|
|
|
|
// auto index cannot work if hook init failed
|
|
|
|
if paramtable.Get().AutoIndexConfig.Enable.GetAsBool() {
|
2023-06-08 15:40:35 +08:00
|
|
|
log.Error("QueryNode init hook failed", zap.Error(err))
|
2023-04-06 14:32:29 +08:00
|
|
|
initError = err
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
node.factory.Init(paramtable.Get())
|
|
|
|
|
2023-08-11 10:35:42 +08:00
|
|
|
localRootPath := paramtable.Get().LocalStorageCfg.Path.GetValue()
|
2023-12-27 16:10:47 +08:00
|
|
|
localUsedSize, err := segments.GetLocalUsedSize(node.ctx, localRootPath)
|
2023-08-11 10:35:42 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Warn("get local used size failed", zap.Error(err))
|
|
|
|
initError = err
|
|
|
|
return
|
|
|
|
}
|
2024-02-21 11:54:53 +08:00
|
|
|
metrics.QueryNodeDiskUsedSize.WithLabelValues(fmt.Sprint(node.GetNodeID())).Set(float64(localUsedSize / 1024 / 1024))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
2023-11-30 18:00:33 +08:00
|
|
|
node.chunkManager, err = node.factory.NewPersistentStorageChunkManager(node.ctx)
|
2023-03-27 00:42:00 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Error("QueryNode init vector storage failed", zap.Error(err))
|
|
|
|
initError = err
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2023-07-03 18:24:25 +08:00
|
|
|
schedulePolicy := paramtable.Get().QueryNodeCfg.SchedulePolicyName.GetValue()
|
|
|
|
node.scheduler = tasks.NewScheduler(
|
|
|
|
schedulePolicy,
|
|
|
|
)
|
|
|
|
log.Info("queryNode init scheduler", zap.String("policy", schedulePolicy))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
2023-09-05 10:05:48 +08:00
|
|
|
node.clusterManager = cluster.NewWorkerManager(func(ctx context.Context, nodeID int64) (cluster.Worker, error) {
|
2024-02-21 11:54:53 +08:00
|
|
|
if nodeID == node.GetNodeID() {
|
2023-03-27 00:42:00 +08:00
|
|
|
return NewLocalWorker(node), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
sessions, _, err := node.session.GetSessions(typeutil.QueryNodeRole)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
addr := ""
|
|
|
|
for _, session := range sessions {
|
|
|
|
if session.ServerID == nodeID {
|
|
|
|
addr = session.Address
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-02-28 10:40:09 +08:00
|
|
|
client, err := grpcquerynodeclient.NewClient(node.ctx, addr, nodeID)
|
2023-03-27 00:42:00 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return cluster.NewRemoteWorker(client), nil
|
|
|
|
})
|
|
|
|
node.delegators = typeutil.NewConcurrentMap[string, delegator.ShardDelegator]()
|
|
|
|
node.subscribingChannels = typeutil.NewConcurrentSet[string]()
|
2023-08-23 10:10:22 +08:00
|
|
|
node.unsubscribingChannels = typeutil.NewConcurrentSet[string]()
|
2023-03-27 00:42:00 +08:00
|
|
|
node.manager = segments.NewManager()
|
2024-01-12 18:10:51 +08:00
|
|
|
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
|
|
|
node.loader = segments.NewLoaderV2(node.manager, node.chunkManager)
|
|
|
|
} else {
|
|
|
|
node.loader = segments.NewLoader(node.manager, node.chunkManager)
|
|
|
|
}
|
2024-02-21 11:54:53 +08:00
|
|
|
node.dispClient = msgdispatcher.NewClient(node.factory, typeutil.QueryNodeRole, node.GetNodeID())
|
2023-03-27 00:42:00 +08:00
|
|
|
// init pipeline manager
|
|
|
|
node.pipelineManager = pipeline.NewManager(node.manager, node.tSafeManager, node.dispClient, node.delegators)
|
|
|
|
|
2023-06-25 14:38:44 +08:00
|
|
|
err = node.InitSegcore()
|
|
|
|
if err != nil {
|
|
|
|
log.Error("QueryNode init segcore failed", zap.Error(err))
|
|
|
|
initError = err
|
|
|
|
return
|
|
|
|
}
|
2023-03-27 00:42:00 +08:00
|
|
|
if paramtable.Get().QueryNodeCfg.GCEnabled.GetAsBool() {
|
|
|
|
if paramtable.Get().QueryNodeCfg.GCHelperEnabled.GetAsBool() {
|
|
|
|
action := func(GOGC uint32) {
|
|
|
|
debug.SetGCPercent(int(GOGC))
|
|
|
|
}
|
|
|
|
gc.NewTuner(paramtable.Get().QueryNodeCfg.OverloadedMemoryThresholdPercentage.GetAsFloat(), uint32(paramtable.Get().QueryNodeCfg.MinimumGOGCConfig.GetAsInt()), uint32(paramtable.Get().QueryNodeCfg.MaximumGOGCConfig.GetAsInt()), action)
|
|
|
|
} else {
|
|
|
|
action := func(uint32) {}
|
|
|
|
gc.NewTuner(paramtable.Get().QueryNodeCfg.OverloadedMemoryThresholdPercentage.GetAsFloat(), uint32(paramtable.Get().QueryNodeCfg.MinimumGOGCConfig.GetAsInt()), uint32(paramtable.Get().QueryNodeCfg.MaximumGOGCConfig.GetAsInt()), action)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Info("query node init successfully",
|
2024-02-21 11:54:53 +08:00
|
|
|
zap.Int64("queryNodeID", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
zap.String("Address", node.address),
|
|
|
|
)
|
|
|
|
})
|
|
|
|
|
|
|
|
return initError
|
|
|
|
}
|
|
|
|
|
|
|
|
// Start mainly start QueryNode's query service.
|
|
|
|
func (node *QueryNode) Start() error {
|
2023-05-12 09:51:25 +08:00
|
|
|
node.startOnce.Do(func() {
|
2023-09-28 10:21:26 +08:00
|
|
|
node.scheduler.Start()
|
2023-05-12 09:51:25 +08:00
|
|
|
|
|
|
|
paramtable.SetCreateTime(time.Now())
|
|
|
|
paramtable.SetUpdateTime(time.Now())
|
2023-07-19 18:48:58 +08:00
|
|
|
mmapDirPath := paramtable.Get().QueryNodeCfg.MmapDirPath.GetValue()
|
|
|
|
mmapEnabled := len(mmapDirPath) > 0
|
2023-05-12 09:51:25 +08:00
|
|
|
node.UpdateStateCode(commonpb.StateCode_Healthy)
|
2023-10-19 11:10:07 +08:00
|
|
|
|
2024-02-21 11:54:53 +08:00
|
|
|
registry.GetInMemoryResolver().RegisterQueryNode(node.GetNodeID(), node)
|
2023-05-12 09:51:25 +08:00
|
|
|
log.Info("query node start successfully",
|
2024-02-21 11:54:53 +08:00
|
|
|
zap.Int64("queryNodeID", node.GetNodeID()),
|
2023-05-12 09:51:25 +08:00
|
|
|
zap.String("Address", node.address),
|
2023-07-19 18:48:58 +08:00
|
|
|
zap.Bool("mmapEnabled", mmapEnabled),
|
2023-05-12 09:51:25 +08:00
|
|
|
)
|
|
|
|
})
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Stop mainly stop QueryNode's query service, historical loop and streaming loop.
|
|
|
|
func (node *QueryNode) Stop() error {
|
2023-04-18 17:56:30 +08:00
|
|
|
node.stopOnce.Do(func() {
|
|
|
|
log.Info("Query node stop...")
|
2023-06-30 17:28:23 +08:00
|
|
|
err := node.session.GoingStop()
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("session fail to go stopping state", zap.Error(err))
|
|
|
|
} else {
|
2023-12-21 10:14:52 +08:00
|
|
|
metrics.StoppingBalanceNodeNum.WithLabelValues().Set(1)
|
2024-02-29 17:01:50 +08:00
|
|
|
// TODO: Redundant timeout control, graceful stop timeout is controlled by outside by `component`.
|
|
|
|
// Integration test is still using it, Remove it in future.
|
2023-06-30 17:28:23 +08:00
|
|
|
timeoutCh := time.After(paramtable.Get().QueryNodeCfg.GracefulStopTimeout.GetAsDuration(time.Second))
|
|
|
|
|
|
|
|
outer:
|
2023-07-07 17:28:25 +08:00
|
|
|
for (node.manager != nil && !node.manager.Segment.Empty()) ||
|
|
|
|
(node.pipelineManager != nil && node.pipelineManager.Num() != 0) {
|
2023-12-21 10:14:52 +08:00
|
|
|
var (
|
|
|
|
sealedSegments = []segments.Segment{}
|
|
|
|
growingSegments = []segments.Segment{}
|
|
|
|
channelNum = 0
|
|
|
|
)
|
|
|
|
if node.manager != nil {
|
|
|
|
sealedSegments = node.manager.Segment.GetBy(segments.WithType(segments.SegmentTypeSealed))
|
|
|
|
growingSegments = node.manager.Segment.GetBy(segments.WithType(segments.SegmentTypeGrowing))
|
|
|
|
}
|
|
|
|
if node.pipelineManager != nil {
|
|
|
|
channelNum = node.pipelineManager.Num()
|
|
|
|
}
|
|
|
|
|
2023-06-30 17:28:23 +08:00
|
|
|
select {
|
|
|
|
case <-timeoutCh:
|
2024-02-29 17:01:50 +08:00
|
|
|
log.Warn("migrate data timed out", zap.Int64("ServerID", paramtable.GetNodeID()),
|
2023-12-21 10:14:52 +08:00
|
|
|
zap.Int64s("sealedSegments", lo.Map(sealedSegments, func(s segments.Segment, i int) int64 {
|
2023-06-30 17:28:23 +08:00
|
|
|
return s.ID()
|
|
|
|
})),
|
2023-12-21 10:14:52 +08:00
|
|
|
zap.Int64s("growingSegments", lo.Map(growingSegments, func(t segments.Segment, i int) int64 {
|
2023-06-30 17:28:23 +08:00
|
|
|
return t.ID()
|
|
|
|
})),
|
2023-07-06 10:04:25 +08:00
|
|
|
zap.Int("channelNum", channelNum),
|
2023-06-30 17:28:23 +08:00
|
|
|
)
|
|
|
|
break outer
|
|
|
|
case <-time.After(time.Second):
|
2024-02-21 11:54:53 +08:00
|
|
|
metrics.StoppingBalanceSegmentNum.WithLabelValues(fmt.Sprint(node.GetNodeID())).Set(float64(len(sealedSegments)))
|
|
|
|
metrics.StoppingBalanceChannelNum.WithLabelValues(fmt.Sprint(node.GetNodeID())).Set(float64(channelNum))
|
2024-02-29 17:01:50 +08:00
|
|
|
log.Info("migrate data...", zap.Int64("ServerID", paramtable.GetNodeID()),
|
|
|
|
zap.Int64s("sealedSegments", lo.Map(sealedSegments, func(s segments.Segment, i int) int64 {
|
|
|
|
return s.ID()
|
|
|
|
})),
|
|
|
|
zap.Int64s("growingSegments", lo.Map(growingSegments, func(t segments.Segment, i int) int64 {
|
|
|
|
return t.ID()
|
|
|
|
})),
|
|
|
|
zap.Int("channelNum", channelNum),
|
|
|
|
)
|
2023-06-30 17:28:23 +08:00
|
|
|
}
|
|
|
|
}
|
2023-12-21 10:14:52 +08:00
|
|
|
|
|
|
|
metrics.StoppingBalanceNodeNum.WithLabelValues().Set(0)
|
2024-02-21 11:54:53 +08:00
|
|
|
metrics.StoppingBalanceSegmentNum.WithLabelValues(fmt.Sprint(node.GetNodeID())).Set(0)
|
|
|
|
metrics.StoppingBalanceChannelNum.WithLabelValues(fmt.Sprint(node.GetNodeID())).Set(0)
|
2023-06-30 17:28:23 +08:00
|
|
|
}
|
|
|
|
|
2023-04-18 17:56:30 +08:00
|
|
|
node.UpdateStateCode(commonpb.StateCode_Abnormal)
|
|
|
|
node.lifetime.Wait()
|
2023-09-28 10:21:26 +08:00
|
|
|
if node.scheduler != nil {
|
|
|
|
node.scheduler.Stop()
|
|
|
|
}
|
2023-04-18 17:56:30 +08:00
|
|
|
if node.pipelineManager != nil {
|
|
|
|
node.pipelineManager.Close()
|
|
|
|
}
|
2023-11-08 03:20:17 +08:00
|
|
|
// Delay the cancellation of ctx to ensure that the session is automatically recycled after closed the pipeline
|
|
|
|
node.cancel()
|
2023-04-18 17:56:30 +08:00
|
|
|
if node.session != nil {
|
|
|
|
node.session.Stop()
|
|
|
|
}
|
|
|
|
if node.dispClient != nil {
|
|
|
|
node.dispClient.Close()
|
|
|
|
}
|
2023-06-30 17:28:23 +08:00
|
|
|
if node.manager != nil {
|
|
|
|
node.manager.Segment.Clear()
|
|
|
|
}
|
2023-06-25 14:38:44 +08:00
|
|
|
|
2023-07-27 19:49:02 +08:00
|
|
|
node.CloseSegcore()
|
2023-04-18 17:56:30 +08:00
|
|
|
})
|
2023-03-27 00:42:00 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// UpdateStateCode updata the state of query node, which can be initializing, healthy, and abnormal
|
|
|
|
func (node *QueryNode) UpdateStateCode(code commonpb.StateCode) {
|
|
|
|
node.lifetime.SetState(code)
|
|
|
|
}
|
|
|
|
|
|
|
|
// SetEtcdClient assigns parameter client to its member etcdCli
|
|
|
|
func (node *QueryNode) SetEtcdClient(client *clientv3.Client) {
|
|
|
|
node.etcdCli = client
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) GetAddress() string {
|
|
|
|
return node.address
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) SetAddress(address string) {
|
|
|
|
node.address = address
|
|
|
|
}
|
2023-04-06 14:32:29 +08:00
|
|
|
|
|
|
|
// initHook initializes parameter tuning hook.
|
|
|
|
func (node *QueryNode) initHook() error {
|
|
|
|
path := paramtable.Get().QueryNodeCfg.SoPath.GetValue()
|
|
|
|
if path == "" {
|
|
|
|
return fmt.Errorf("fail to set the plugin path")
|
|
|
|
}
|
2023-08-14 18:57:32 +08:00
|
|
|
log.Info("start to load plugin", zap.String("path", path))
|
2023-04-06 14:32:29 +08:00
|
|
|
|
|
|
|
p, err := plugin.Open(path)
|
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("fail to open the plugin, error: %s", err.Error())
|
|
|
|
}
|
2023-08-14 18:57:32 +08:00
|
|
|
log.Info("plugin open")
|
2023-04-06 14:32:29 +08:00
|
|
|
|
|
|
|
h, err := p.Lookup("QueryNodePlugin")
|
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("fail to find the 'QueryNodePlugin' object in the plugin, error: %s", err.Error())
|
|
|
|
}
|
|
|
|
|
2023-09-28 10:01:26 +08:00
|
|
|
hoo, ok := h.(optimizers.QueryHook)
|
2023-04-06 14:32:29 +08:00
|
|
|
if !ok {
|
|
|
|
return fmt.Errorf("fail to convert the `Hook` interface")
|
|
|
|
}
|
2023-08-31 10:25:01 +08:00
|
|
|
if err = hoo.Init(paramtable.Get().AutoIndexConfig.AutoIndexSearchConfig.GetValue()); err != nil {
|
2023-04-06 14:32:29 +08:00
|
|
|
return fmt.Errorf("fail to init configs for the hook, error: %s", err.Error())
|
|
|
|
}
|
2023-08-31 10:25:01 +08:00
|
|
|
if err = hoo.InitTuningConfig(paramtable.Get().AutoIndexConfig.AutoIndexTuningConfig.GetValue()); err != nil {
|
2023-07-03 15:18:24 +08:00
|
|
|
return fmt.Errorf("fail to init tuning configs for the hook, error: %s", err.Error())
|
|
|
|
}
|
2023-04-06 14:32:29 +08:00
|
|
|
|
|
|
|
node.queryHook = hoo
|
2023-07-03 15:18:24 +08:00
|
|
|
node.handleQueryHookEvent()
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *QueryNode) handleQueryHookEvent() {
|
2023-04-06 14:32:29 +08:00
|
|
|
onEvent := func(event *config.Event) {
|
|
|
|
if node.queryHook != nil {
|
|
|
|
if err := node.queryHook.Init(event.Value); err != nil {
|
|
|
|
log.Error("failed to refresh hook config", zap.Error(err))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2023-07-03 15:18:24 +08:00
|
|
|
onEvent2 := func(event *config.Event) {
|
2023-08-31 10:25:01 +08:00
|
|
|
if node.queryHook != nil && strings.HasPrefix(event.Key, paramtable.Get().AutoIndexConfig.AutoIndexTuningConfig.KeyPrefix) {
|
|
|
|
realKey := strings.TrimPrefix(event.Key, paramtable.Get().AutoIndexConfig.AutoIndexTuningConfig.KeyPrefix)
|
2023-07-03 15:18:24 +08:00
|
|
|
if event.EventType == config.CreateType || event.EventType == config.UpdateType {
|
|
|
|
if err := node.queryHook.InitTuningConfig(map[string]string{realKey: event.Value}); err != nil {
|
2023-08-14 18:57:32 +08:00
|
|
|
log.Warn("failed to refresh hook tuning config", zap.Error(err))
|
2023-07-03 15:18:24 +08:00
|
|
|
}
|
|
|
|
} else if event.EventType == config.DeleteType {
|
|
|
|
if err := node.queryHook.DeleteTuningConfig(realKey); err != nil {
|
2023-08-14 18:57:32 +08:00
|
|
|
log.Warn("failed to delete hook tuning config", zap.Error(err))
|
2023-07-03 15:18:24 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2023-08-31 10:25:01 +08:00
|
|
|
paramtable.Get().Watch(paramtable.Get().AutoIndexConfig.AutoIndexSearchConfig.Key, config.NewHandler("queryHook", onEvent))
|
2023-04-06 14:32:29 +08:00
|
|
|
|
2023-08-31 10:25:01 +08:00
|
|
|
paramtable.Get().WatchKeyPrefix(paramtable.Get().AutoIndexConfig.AutoIndexTuningConfig.KeyPrefix, config.NewHandler("queryHook2", onEvent2))
|
2023-04-06 14:32:29 +08:00
|
|
|
}
|