mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
Use GlobalParamTable for all components (#13885)
Signed-off-by: yudong.cai <yudong.cai@zilliz.com>
This commit is contained in:
parent
6c3f1a9162
commit
891c202b73
@ -94,7 +94,7 @@ func (mr *MilvusRoles) runRootCoord(ctx context.Context, localMsg bool) *compone
|
||||
rootcoord.Params.InitOnce()
|
||||
|
||||
f := setLoggerFunc()
|
||||
rootcoord.Params.SetLogConfig(f)
|
||||
rootcoord.Params.BaseParams.SetLogConfig(f)
|
||||
factory := newMsgFactory(localMsg)
|
||||
var err error
|
||||
rc, err = components.NewRootCoord(ctx, factory)
|
||||
@ -119,11 +119,11 @@ func (mr *MilvusRoles) runProxy(ctx context.Context, localMsg bool, alias string
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
proxy.Params.InitAlias(alias)
|
||||
proxy.Params.ProxyCfg.InitAlias(alias)
|
||||
proxy.Params.InitOnce()
|
||||
|
||||
f := setLoggerFunc()
|
||||
proxy.Params.SetLogConfig(f)
|
||||
proxy.Params.BaseParams.SetLogConfig(f)
|
||||
factory := newMsgFactory(localMsg)
|
||||
var err error
|
||||
pn, err = components.NewProxy(ctx, factory)
|
||||
@ -151,7 +151,7 @@ func (mr *MilvusRoles) runQueryCoord(ctx context.Context, localMsg bool) *compon
|
||||
querycoord.Params.InitOnce()
|
||||
|
||||
f := setLoggerFunc()
|
||||
querycoord.Params.SetLogConfig(f)
|
||||
querycoord.Params.BaseParams.SetLogConfig(f)
|
||||
factory := newMsgFactory(localMsg)
|
||||
var err error
|
||||
qs, err = components.NewQueryCoord(ctx, factory)
|
||||
@ -176,11 +176,11 @@ func (mr *MilvusRoles) runQueryNode(ctx context.Context, localMsg bool, alias st
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
querynode.Params.InitAlias(alias)
|
||||
querynode.Params.QueryNodeCfg.InitAlias(alias)
|
||||
querynode.Params.InitOnce()
|
||||
|
||||
f := setLoggerFunc()
|
||||
querynode.Params.SetLogConfig(f)
|
||||
querynode.Params.BaseParams.SetLogConfig(f)
|
||||
factory := newMsgFactory(localMsg)
|
||||
var err error
|
||||
qn, err = components.NewQueryNode(ctx, factory)
|
||||
@ -208,7 +208,7 @@ func (mr *MilvusRoles) runDataCoord(ctx context.Context, localMsg bool) *compone
|
||||
datacoord.Params.InitOnce()
|
||||
|
||||
f := setLoggerFunc()
|
||||
datacoord.Params.SetLogConfig(f)
|
||||
datacoord.Params.BaseParams.SetLogConfig(f)
|
||||
factory := newMsgFactory(localMsg)
|
||||
var err error
|
||||
ds, err = components.NewDataCoord(ctx, factory)
|
||||
@ -233,10 +233,10 @@ func (mr *MilvusRoles) runDataNode(ctx context.Context, localMsg bool, alias str
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
datanode.Params.InitAlias(alias)
|
||||
datanode.Params.DataNodeCfg.InitAlias(alias)
|
||||
datanode.Params.InitOnce()
|
||||
f := setLoggerFunc()
|
||||
datanode.Params.SetLogConfig(f)
|
||||
datanode.Params.BaseParams.SetLogConfig(f)
|
||||
factory := newMsgFactory(localMsg)
|
||||
var err error
|
||||
dn, err = components.NewDataNode(ctx, factory)
|
||||
@ -264,7 +264,7 @@ func (mr *MilvusRoles) runIndexCoord(ctx context.Context, localMsg bool) *compon
|
||||
indexcoord.Params.InitOnce()
|
||||
|
||||
f := setLoggerFunc()
|
||||
indexcoord.Params.SetLogConfig(f)
|
||||
indexcoord.Params.BaseParams.SetLogConfig(f)
|
||||
var err error
|
||||
is, err = components.NewIndexCoord(ctx)
|
||||
if err != nil {
|
||||
@ -288,11 +288,11 @@ func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool, alias st
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
indexnode.Params.InitAlias(alias)
|
||||
indexnode.Params.IndexNodeCfg.InitAlias(alias)
|
||||
indexnode.Params.InitOnce()
|
||||
|
||||
f := setLoggerFunc()
|
||||
indexnode.Params.SetLogConfig(f)
|
||||
indexnode.Params.BaseParams.SetLogConfig(f)
|
||||
var err error
|
||||
in, err = components.NewIndexNode(ctx)
|
||||
if err != nil {
|
||||
|
||||
@ -53,7 +53,7 @@ func (alloc *rootCoordAllocator) allocTimestamp(ctx context.Context) (Timestamp,
|
||||
MsgType: commonpb.MsgType_RequestTSO,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataCoordCfg.NodeID,
|
||||
},
|
||||
Count: 1,
|
||||
})
|
||||
@ -70,7 +70,7 @@ func (alloc *rootCoordAllocator) allocID(ctx context.Context) (UniqueID, error)
|
||||
MsgType: commonpb.MsgType_RequestID,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataCoordCfg.NodeID,
|
||||
},
|
||||
Count: 1,
|
||||
})
|
||||
|
||||
@ -132,7 +132,7 @@ func NewChannelStore(kv kv.TxnKV) *ChannelStore {
|
||||
|
||||
// Reload restores the buffer channels and node-channels mapping from kv
|
||||
func (c *ChannelStore) Reload() error {
|
||||
keys, values, err := c.store.LoadWithPrefix(Params.ChannelWatchSubPath)
|
||||
keys, values, err := c.store.LoadWithPrefix(Params.DataCoordCfg.ChannelWatchSubPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -341,11 +341,11 @@ func (c *ChannelStore) txn(opSet ChannelOpSet) error {
|
||||
}
|
||||
|
||||
func buildChannelKey(nodeID int64, channel string) string {
|
||||
return fmt.Sprintf("%s%s%d%s%s", Params.ChannelWatchSubPath, delimeter, nodeID, delimeter, channel)
|
||||
return fmt.Sprintf("%s%s%d%s%s", Params.DataCoordCfg.ChannelWatchSubPath, delimeter, nodeID, delimeter, channel)
|
||||
}
|
||||
|
||||
func buildNodeKey(nodeID int64) string {
|
||||
return fmt.Sprintf("%s%s%d", Params.ChannelWatchSubPath, delimeter, nodeID)
|
||||
return fmt.Sprintf("%s%s%d", Params.DataCoordCfg.ChannelWatchSubPath, delimeter, nodeID)
|
||||
}
|
||||
|
||||
func parseNodeID(key string) (int64, error) {
|
||||
|
||||
@ -112,7 +112,7 @@ func (c *Cluster) Flush(ctx context.Context, segments []*datapb.SegmentInfo, mar
|
||||
req := &datapb.FlushSegmentsRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Flush,
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataCoordCfg.NodeID,
|
||||
},
|
||||
SegmentIDs: segments,
|
||||
MarkSegmentIDs: marks,
|
||||
|
||||
@ -63,7 +63,7 @@ func TestClusterCreate(t *testing.T) {
|
||||
}
|
||||
info1Data, err := proto.Marshal(info1)
|
||||
assert.Nil(t, err)
|
||||
err = kv.Save(Params.ChannelWatchSubPath+"/1/channel1", string(info1Data))
|
||||
err = kv.Save(Params.DataCoordCfg.ChannelWatchSubPath+"/1/channel1", string(info1Data))
|
||||
assert.Nil(t, err)
|
||||
|
||||
sessionManager := NewSessionManager()
|
||||
|
||||
@ -108,7 +108,7 @@ func (t *compactionTrigger) startGlobalCompactionLoop() {
|
||||
defer t.wg.Done()
|
||||
|
||||
// If AutoCompaction disabled, global loop will not start
|
||||
if !Params.EnableAutoCompaction {
|
||||
if !Params.DataCoordCfg.EnableAutoCompaction {
|
||||
return
|
||||
}
|
||||
|
||||
@ -156,7 +156,7 @@ func (t *compactionTrigger) triggerCompaction(timetravel *timetravel) error {
|
||||
// triggerSingleCompaction triger a compaction bundled with collection-partiiton-channel-segment
|
||||
func (t *compactionTrigger) triggerSingleCompaction(collectionID, partitionID, segmentID int64, channel string, timetravel *timetravel) error {
|
||||
// If AutoCompaction diabled, flush request will not trigger compaction
|
||||
if !Params.EnableAutoCompaction {
|
||||
if !Params.DataCoordCfg.EnableAutoCompaction {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
@ -647,7 +647,7 @@ func Test_compactionTrigger_triggerCompaction(t *testing.T) {
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
Params.EnableAutoCompaction = tt.fields.autoCompactionEnabled
|
||||
Params.DataCoordCfg.EnableAutoCompaction = tt.fields.autoCompactionEnabled
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
@ -1040,7 +1040,7 @@ func Test_compactionTrigger_singleTriggerCompaction(t *testing.T) {
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
Params.EnableAutoCompaction = tt.fields.enableAutoCompaction
|
||||
Params.DataCoordCfg.EnableAutoCompaction = tt.fields.enableAutoCompaction
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
|
||||
@ -204,9 +204,9 @@ func Test_garbageCollector_scan(t *testing.T) {
|
||||
// initialize unit test sso env
|
||||
func initUtOSSEnv(bucket, root string, n int) (cli *minio.Client, inserts []string, stats []string, delta []string, other []string, err error) {
|
||||
Params.Init()
|
||||
cli, err = minio.New(Params.MinioAddress, &minio.Options{
|
||||
Creds: credentials.NewStaticV4(Params.MinioAccessKeyID, Params.MinioSecretAccessKey, ""),
|
||||
Secure: Params.MinioUseSSL,
|
||||
cli, err = minio.New(Params.DataCoordCfg.MinioAddress, &minio.Options{
|
||||
Creds: credentials.NewStaticV4(Params.DataCoordCfg.MinioAccessKeyID, Params.DataCoordCfg.MinioSecretAccessKey, ""),
|
||||
Secure: Params.DataCoordCfg.MinioUseSSL,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, nil, err
|
||||
|
||||
@ -59,7 +59,7 @@ func (s *Server) getSystemInfoMetrics(
|
||||
coordTopology := metricsinfo.DataCoordTopology{
|
||||
Cluster: clusterTopology,
|
||||
Connections: metricsinfo.ConnTopology{
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, Params.NodeID),
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, Params.DataCoordCfg.NodeID),
|
||||
// TODO(dragondriver): fill ConnectedComponents if necessary
|
||||
ConnectedComponents: []metricsinfo.ConnectionInfo{},
|
||||
},
|
||||
@ -70,7 +70,7 @@ func (s *Server) getSystemInfoMetrics(
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
},
|
||||
Response: "",
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, Params.NodeID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, Params.DataCoordCfg.NodeID),
|
||||
}
|
||||
var err error
|
||||
resp.Response, err = metricsinfo.MarshalTopology(coordTopology)
|
||||
@ -87,7 +87,7 @@ func (s *Server) getSystemInfoMetrics(
|
||||
func (s *Server) getDataCoordMetrics() metricsinfo.DataCoordInfos {
|
||||
ret := metricsinfo.DataCoordInfos{
|
||||
BaseComponentInfos: metricsinfo.BaseComponentInfos{
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, Params.NodeID),
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, Params.DataCoordCfg.NodeID),
|
||||
HardwareInfos: metricsinfo.HardwareMetrics{
|
||||
IP: s.session.Address,
|
||||
CPUCoreCount: metricsinfo.GetCPUCoreCount(false),
|
||||
@ -98,13 +98,13 @@ func (s *Server) getDataCoordMetrics() metricsinfo.DataCoordInfos {
|
||||
DiskUsage: metricsinfo.GetDiskUsage(),
|
||||
},
|
||||
SystemInfo: metricsinfo.DeployMetrics{},
|
||||
CreatedTime: Params.CreatedTime.String(),
|
||||
UpdatedTime: Params.UpdatedTime.String(),
|
||||
CreatedTime: Params.DataCoordCfg.CreatedTime.String(),
|
||||
UpdatedTime: Params.DataCoordCfg.UpdatedTime.String(),
|
||||
Type: typeutil.DataCoordRole,
|
||||
ID: s.session.ServerID,
|
||||
},
|
||||
SystemConfigurations: metricsinfo.DataCoordConfiguration{
|
||||
SegmentMaxSize: Params.SegmentMaxSize,
|
||||
SegmentMaxSize: Params.DataCoordCfg.SegmentMaxSize,
|
||||
},
|
||||
}
|
||||
|
||||
|
||||
@ -1,375 +0,0 @@
|
||||
// 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 datacoord
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
)
|
||||
|
||||
// ParamTable is a derived struct of paramtable.BaseTable. It achieves Composition by
|
||||
// embedding paramtable.BaseTable. It is used to quickly and easily access the system configuration.
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
|
||||
NodeID int64
|
||||
|
||||
IP string
|
||||
Port int
|
||||
Address string
|
||||
|
||||
// --- ETCD ---
|
||||
EtcdEndpoints []string
|
||||
MetaRootPath string
|
||||
KvRootPath string
|
||||
SegmentBinlogSubPath string
|
||||
CollectionBinlogSubPath string
|
||||
ChannelWatchSubPath string
|
||||
|
||||
// --- MinIO ---
|
||||
MinioAddress string
|
||||
MinioAccessKeyID string
|
||||
MinioSecretAccessKey string
|
||||
MinioUseSSL bool
|
||||
MinioBucketName string
|
||||
MinioRootPath string
|
||||
|
||||
// --- Pulsar ---
|
||||
PulsarAddress string
|
||||
|
||||
// --- Rocksmq ---
|
||||
RocksmqPath string
|
||||
|
||||
FlushStreamPosSubPath string
|
||||
StatsStreamPosSubPath string
|
||||
|
||||
// --- SEGMENTS ---
|
||||
SegmentMaxSize float64
|
||||
SegmentSealProportion float64
|
||||
SegAssignmentExpiration int64
|
||||
|
||||
// --- Channels ---
|
||||
ClusterChannelPrefix string
|
||||
InsertChannelPrefixName string
|
||||
TimeTickChannelName string
|
||||
SegmentInfoChannelName string
|
||||
DataCoordSubscriptionName string
|
||||
|
||||
CreatedTime time.Time
|
||||
UpdatedTime time.Time
|
||||
|
||||
EnableCompaction bool
|
||||
EnableGarbageCollection bool
|
||||
|
||||
CompactionRetentionDuration int64
|
||||
EnableAutoCompaction bool
|
||||
|
||||
// Garbage Collection
|
||||
GCInterval time.Duration
|
||||
GCMissingTolerance time.Duration
|
||||
GCDropTolerance time.Duration
|
||||
}
|
||||
|
||||
// Params is a package scoped variable of type ParamTable.
|
||||
var Params ParamTable
|
||||
var once sync.Once
|
||||
|
||||
// Init is an override method of BaseTable's Init. It mainly calls the
|
||||
// Init of BaseTable and do some other initialization.
|
||||
func (p *ParamTable) Init() {
|
||||
// load yaml
|
||||
p.BaseTable.Init()
|
||||
|
||||
// set members
|
||||
p.initEtcdEndpoints()
|
||||
p.initMetaRootPath()
|
||||
p.initKvRootPath()
|
||||
p.initSegmentBinlogSubPath()
|
||||
p.initCollectionBinlogSubPath()
|
||||
p.initChannelWatchPrefix()
|
||||
|
||||
p.initPulsarAddress()
|
||||
p.initRocksmqPath()
|
||||
|
||||
p.initSegmentMaxSize()
|
||||
p.initSegmentSealProportion()
|
||||
p.initSegAssignmentExpiration()
|
||||
|
||||
// Has to init global msgchannel prefix before other channel names
|
||||
p.initClusterMsgChannelPrefix()
|
||||
p.initInsertChannelPrefixName()
|
||||
p.initTimeTickChannelName()
|
||||
p.initSegmentInfoChannelName()
|
||||
p.initDataCoordSubscriptionName()
|
||||
p.initRoleName()
|
||||
|
||||
p.initFlushStreamPosSubPath()
|
||||
p.initStatsStreamPosSubPath()
|
||||
|
||||
p.initEnableCompaction()
|
||||
|
||||
p.initMinioAddress()
|
||||
p.initMinioAccessKeyID()
|
||||
p.initMinioSecretAccessKey()
|
||||
p.initMinioUseSSL()
|
||||
p.initMinioBucketName()
|
||||
p.initMinioRootPath()
|
||||
|
||||
p.initCompactionRetentionDuration()
|
||||
p.initEnableAutoCompaction()
|
||||
|
||||
p.initEnableGarbageCollection()
|
||||
p.initGCInterval()
|
||||
p.initGCMissingTolerance()
|
||||
p.initGCDropTolerance()
|
||||
}
|
||||
|
||||
// InitOnce ensures param table is a singleton
|
||||
func (p *ParamTable) InitOnce() {
|
||||
once.Do(func() {
|
||||
p.Init()
|
||||
})
|
||||
}
|
||||
|
||||
func (p *ParamTable) initEtcdEndpoints() {
|
||||
endpoints, err := p.Load("_EtcdEndpoints")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initPulsarAddress() {
|
||||
addr, err := p.Load("_PulsarAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.PulsarAddress = addr
|
||||
}
|
||||
|
||||
func (p *ParamTable) initRocksmqPath() {
|
||||
path, err := p.Load("_RocksmqPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.RocksmqPath = path
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMetaRootPath() {
|
||||
rootPath, err := p.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := p.Load("etcd.metaSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MetaRootPath = rootPath + "/" + subPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) initKvRootPath() {
|
||||
rootPath, err := p.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := p.Load("etcd.kvSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.KvRootPath = rootPath + "/" + subPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) initSegmentBinlogSubPath() {
|
||||
subPath, err := p.Load("etcd.segmentBinlogSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.SegmentBinlogSubPath = subPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) initCollectionBinlogSubPath() {
|
||||
subPath, err := p.Load("etcd.collectionBinlogSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.CollectionBinlogSubPath = subPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) initSegmentMaxSize() {
|
||||
p.SegmentMaxSize = p.ParseFloatWithDefault("dataCoord.segment.maxSize", 512.0)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initSegmentSealProportion() {
|
||||
p.SegmentSealProportion = p.ParseFloatWithDefault("dataCoord.segment.sealProportion", 0.75)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initSegAssignmentExpiration() {
|
||||
p.SegAssignmentExpiration = p.ParseInt64WithDefault("dataCoord.segment.assignmentExpiration", 2000)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initClusterMsgChannelPrefix() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.cluster")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.ClusterChannelPrefix = config
|
||||
}
|
||||
|
||||
func (p *ParamTable) initInsertChannelPrefixName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.dataCoordInsertChannel")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.InsertChannelPrefixName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initTimeTickChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.dataCoordTimeTick")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.TimeTickChannelName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initSegmentInfoChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.dataCoordSegmentInfo")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.SegmentInfoChannelName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDataCoordSubscriptionName() {
|
||||
config, err := p.Load("msgChannel.subNamePrefix.dataCoordSubNamePrefix")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.DataCoordSubscriptionName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initRoleName() {
|
||||
p.RoleName = "datacoord"
|
||||
}
|
||||
|
||||
func (p *ParamTable) initFlushStreamPosSubPath() {
|
||||
subPath, err := p.Load("etcd.flushStreamPosSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.FlushStreamPosSubPath = subPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) initStatsStreamPosSubPath() {
|
||||
subPath, err := p.Load("etcd.statsStreamPosSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.StatsStreamPosSubPath = subPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) initChannelWatchPrefix() {
|
||||
// WARN: this value should not be put to milvus.yaml. It's a default value for channel watch path.
|
||||
// This will be removed after we reconstruct our config module.
|
||||
p.ChannelWatchSubPath = "channelwatch"
|
||||
}
|
||||
|
||||
func (p *ParamTable) initEnableCompaction() {
|
||||
p.EnableCompaction = p.ParseBool("dataCoord.enableCompaction", false)
|
||||
}
|
||||
|
||||
// -- GC --
|
||||
|
||||
func (p *ParamTable) initEnableGarbageCollection() {
|
||||
p.EnableGarbageCollection = p.ParseBool("dataCoord.enableGarbageCollection", false)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initGCInterval() {
|
||||
p.GCInterval = time.Duration(p.ParseInt64WithDefault("dataCoord.gc.interval", 60*60)) * time.Second
|
||||
}
|
||||
|
||||
func (p *ParamTable) initGCMissingTolerance() {
|
||||
p.GCMissingTolerance = time.Duration(p.ParseInt64WithDefault("dataCoord.gc.missingTolerance", 24*60*60)) * time.Second
|
||||
}
|
||||
|
||||
func (p *ParamTable) initGCDropTolerance() {
|
||||
p.GCDropTolerance = time.Duration(p.ParseInt64WithDefault("dataCoord.gc.dropTolerance", 24*60*60)) * time.Second
|
||||
}
|
||||
|
||||
// --- MinIO ---
|
||||
func (p *ParamTable) initMinioAddress() {
|
||||
endpoint, err := p.Load("_MinioAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioAddress = endpoint
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioAccessKeyID() {
|
||||
keyID, err := p.Load("_MinioAccessKeyID")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioAccessKeyID = keyID
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioSecretAccessKey() {
|
||||
key, err := p.Load("_MinioSecretAccessKey")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioSecretAccessKey = key
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioUseSSL() {
|
||||
usessl, err := p.Load("_MinioUseSSL")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioUseSSL, _ = strconv.ParseBool(usessl)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioBucketName() {
|
||||
bucketName, err := p.Load("_MinioBucketName")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioBucketName = bucketName
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioRootPath() {
|
||||
rootPath, err := p.Load("minio.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioRootPath = rootPath
|
||||
}
|
||||
|
||||
func (p *ParamTable) initCompactionRetentionDuration() {
|
||||
p.CompactionRetentionDuration = p.ParseInt64WithDefault("dataCoord.compaction.retentionDuration", 432000)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initEnableAutoCompaction() {
|
||||
p.EnableAutoCompaction = p.ParseBool("dataCoord.compaction.enableAutoCompaction", false)
|
||||
}
|
||||
@ -1,41 +0,0 @@
|
||||
// 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 datacoord
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
//TODO add more test for other parameters
|
||||
func TestParamTable(t *testing.T) {
|
||||
Params.Init()
|
||||
|
||||
assert.Equal(t, Params.InsertChannelPrefixName, "by-dev-insert-channel-")
|
||||
t.Logf("DataCoord insert channel = %s", Params.InsertChannelPrefixName)
|
||||
|
||||
assert.Equal(t, Params.TimeTickChannelName, "by-dev-datacoord-timetick-channel")
|
||||
t.Logf("DataCoord timetick channel = %s", Params.TimeTickChannelName)
|
||||
|
||||
assert.Equal(t, Params.SegmentInfoChannelName, "by-dev-segment-info-channel")
|
||||
t.Logf("DataCoord segment info channel = %s", Params.SegmentInfoChannelName)
|
||||
|
||||
assert.Equal(t, Params.DataCoordSubscriptionName, "by-dev-dataCoord")
|
||||
t.Logf("DataCoord subscription channel = %s", Params.DataCoordSubscriptionName)
|
||||
|
||||
}
|
||||
@ -41,7 +41,7 @@ func calBySchemaPolicy(schema *schemapb.CollectionSchema) (int, error) {
|
||||
if sizePerRecord == 0 {
|
||||
return -1, errors.New("zero size record schema found")
|
||||
}
|
||||
threshold := Params.SegmentMaxSize * 1024 * 1024
|
||||
threshold := Params.DataCoordCfg.SegmentMaxSize * 1024 * 1024
|
||||
return int(threshold / float64(sizePerRecord)), nil
|
||||
}
|
||||
|
||||
|
||||
@ -78,7 +78,7 @@ func TestUpperLimitCalBySchema(t *testing.T) {
|
||||
},
|
||||
},
|
||||
},
|
||||
expected: int(Params.SegmentMaxSize * 1024 * 1024 / float64(524)),
|
||||
expected: int(Params.DataCoordCfg.SegmentMaxSize * 1024 * 1024 / float64(524)),
|
||||
expectErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
@ -177,7 +177,7 @@ func defaultAllocatePolicy() AllocatePolicy {
|
||||
func defaultSegmentSealPolicy() []segmentSealPolicy {
|
||||
return []segmentSealPolicy{
|
||||
sealByLifetimePolicy(segmentMaxLifetime),
|
||||
getSegmentCapacityPolicy(Params.SegmentSealProportion),
|
||||
getSegmentCapacityPolicy(Params.DataCoordCfg.SegmentSealProportion),
|
||||
}
|
||||
}
|
||||
|
||||
@ -288,7 +288,7 @@ func (s *SegmentManager) genExpireTs(ctx context.Context) (Timestamp, error) {
|
||||
return 0, err
|
||||
}
|
||||
physicalTs, logicalTs := tsoutil.ParseTS(ts)
|
||||
expirePhysicalTs := physicalTs.Add(time.Duration(Params.SegAssignmentExpiration) * time.Millisecond)
|
||||
expirePhysicalTs := physicalTs.Add(time.Duration(Params.DataCoordCfg.SegAssignmentExpiration) * time.Millisecond)
|
||||
expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs))
|
||||
return expireTs, nil
|
||||
}
|
||||
|
||||
@ -38,6 +38,7 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/mqclient"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
@ -86,6 +87,8 @@ type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdEnd
|
||||
// makes sure Server implements `DataCoord`
|
||||
var _ types.DataCoord = (*Server)(nil)
|
||||
|
||||
var Params paramtable.GlobalParamTable
|
||||
|
||||
// Server implements `types.Datacoord`
|
||||
// handles Data Cooridinator related jobs
|
||||
type Server struct {
|
||||
@ -222,13 +225,13 @@ func (s *Server) Register() error {
|
||||
}
|
||||
|
||||
func (s *Server) initSession() error {
|
||||
s.session = sessionutil.NewSession(s.ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
s.session = sessionutil.NewSession(s.ctx, Params.DataCoordCfg.MetaRootPath, Params.DataCoordCfg.EtcdEndpoints)
|
||||
if s.session == nil {
|
||||
return errors.New("failed to initialize session")
|
||||
}
|
||||
s.session.Init(typeutil.DataCoordRole, Params.Address, true)
|
||||
Params.NodeID = s.session.ServerID
|
||||
Params.SetLogger(Params.NodeID)
|
||||
s.session.Init(typeutil.DataCoordRole, Params.DataCoordCfg.Address, true)
|
||||
Params.DataCoordCfg.NodeID = s.session.ServerID
|
||||
Params.BaseParams.SetLogger(Params.DataCoordCfg.NodeID)
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -248,7 +251,7 @@ func (s *Server) Init() error {
|
||||
func (s *Server) Start() error {
|
||||
var err error
|
||||
m := map[string]interface{}{
|
||||
"PulsarAddress": Params.PulsarAddress,
|
||||
"PulsarAddress": Params.DataCoordCfg.PulsarAddress,
|
||||
"ReceiveBufSize": 1024,
|
||||
"PulsarBufSize": 1024}
|
||||
err = s.msFactory.SetParams(m)
|
||||
@ -270,7 +273,7 @@ func (s *Server) Start() error {
|
||||
}
|
||||
|
||||
s.allocator = newRootCoordAllocator(s.rootCoordClient)
|
||||
if Params.EnableCompaction {
|
||||
if Params.DataCoordCfg.EnableCompaction {
|
||||
s.createCompactionHandler()
|
||||
s.createCompactionTrigger()
|
||||
}
|
||||
@ -285,8 +288,8 @@ func (s *Server) Start() error {
|
||||
}
|
||||
|
||||
s.startServerLoop()
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
Params.DataCoordCfg.CreatedTime = time.Now()
|
||||
Params.DataCoordCfg.UpdatedTime = time.Now()
|
||||
atomic.StoreInt64(&s.isServing, ServerStateHealthy)
|
||||
log.Debug("DataCoord startup success")
|
||||
|
||||
@ -329,20 +332,20 @@ func (s *Server) stopCompactionTrigger() {
|
||||
func (s *Server) initGarbageCollection() error {
|
||||
var cli *minio.Client
|
||||
var err error
|
||||
if Params.EnableGarbageCollection {
|
||||
cli, err = minio.New(Params.MinioAddress, &minio.Options{
|
||||
Creds: credentials.NewStaticV4(Params.MinioAccessKeyID, Params.MinioSecretAccessKey, ""),
|
||||
Secure: Params.MinioUseSSL,
|
||||
if Params.DataCoordCfg.EnableGarbageCollection {
|
||||
cli, err = minio.New(Params.DataCoordCfg.MinioAddress, &minio.Options{
|
||||
Creds: credentials.NewStaticV4(Params.DataCoordCfg.MinioAccessKeyID, Params.DataCoordCfg.MinioSecretAccessKey, ""),
|
||||
Secure: Params.DataCoordCfg.MinioUseSSL,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
has, err := cli.BucketExists(context.TODO(), Params.MinioBucketName)
|
||||
has, err := cli.BucketExists(context.TODO(), Params.DataCoordCfg.MinioBucketName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if !has {
|
||||
err = cli.MakeBucket(context.TODO(), Params.MinioBucketName, minio.MakeBucketOptions{})
|
||||
err = cli.MakeBucket(context.TODO(), Params.DataCoordCfg.MinioBucketName, minio.MakeBucketOptions{})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -351,13 +354,13 @@ func (s *Server) initGarbageCollection() error {
|
||||
|
||||
s.garbageCollector = newGarbageCollector(s.meta, GcOption{
|
||||
cli: cli,
|
||||
enabled: Params.EnableGarbageCollection,
|
||||
bucketName: Params.MinioBucketName,
|
||||
rootPath: Params.MinioRootPath,
|
||||
enabled: Params.DataCoordCfg.EnableGarbageCollection,
|
||||
bucketName: Params.DataCoordCfg.MinioBucketName,
|
||||
rootPath: Params.DataCoordCfg.MinioRootPath,
|
||||
|
||||
checkInterval: Params.GCInterval,
|
||||
missingTolerance: Params.GCMissingTolerance,
|
||||
dropTolerance: Params.GCDropTolerance,
|
||||
checkInterval: Params.DataCoordCfg.GCInterval,
|
||||
missingTolerance: Params.DataCoordCfg.GCMissingTolerance,
|
||||
dropTolerance: Params.DataCoordCfg.GCDropTolerance,
|
||||
})
|
||||
return nil
|
||||
}
|
||||
@ -393,7 +396,7 @@ func (s *Server) startSegmentManager() {
|
||||
|
||||
func (s *Server) initMeta() error {
|
||||
connectEtcdFn := func() error {
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.DataCoordCfg.EtcdEndpoints, Params.DataCoordCfg.MetaRootPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -425,11 +428,11 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) {
|
||||
log.Error("DataCoord failed to create timetick channel", zap.Error(err))
|
||||
return
|
||||
}
|
||||
ttMsgStream.AsConsumerWithPosition([]string{Params.TimeTickChannelName},
|
||||
Params.DataCoordSubscriptionName, mqclient.SubscriptionPositionLatest)
|
||||
ttMsgStream.AsConsumerWithPosition([]string{Params.DataCoordCfg.TimeTickChannelName},
|
||||
Params.DataCoordCfg.DataCoordSubscriptionName, mqclient.SubscriptionPositionLatest)
|
||||
log.Debug("DataCoord creates the timetick channel consumer",
|
||||
zap.String("timeTickChannel", Params.TimeTickChannelName),
|
||||
zap.String("subscription", Params.DataCoordSubscriptionName))
|
||||
zap.String("timeTickChannel", Params.DataCoordCfg.TimeTickChannelName),
|
||||
zap.String("subscription", Params.DataCoordCfg.DataCoordSubscriptionName))
|
||||
ttMsgStream.Start()
|
||||
|
||||
go func() {
|
||||
@ -709,7 +712,7 @@ func (s *Server) handleFlushingSegments(ctx context.Context) {
|
||||
|
||||
func (s *Server) initRootCoordClient() error {
|
||||
var err error
|
||||
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.MetaRootPath, Params.EtcdEndpoints); err != nil {
|
||||
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.DataCoordCfg.MetaRootPath, Params.DataCoordCfg.EtcdEndpoints); err != nil {
|
||||
return err
|
||||
}
|
||||
if err = s.rootCoordClient.Init(); err != nil {
|
||||
@ -732,7 +735,7 @@ func (s *Server) Stop() error {
|
||||
s.stopServerLoop()
|
||||
s.session.Revoke(time.Second)
|
||||
|
||||
if Params.EnableCompaction {
|
||||
if Params.DataCoordCfg.EnableCompaction {
|
||||
s.stopCompactionTrigger()
|
||||
s.stopCompactionHandler()
|
||||
}
|
||||
@ -769,7 +772,7 @@ func (s *Server) loadCollectionFromRootCoord(ctx context.Context, collectionID i
|
||||
resp, err := s.rootCoordClient.DescribeCollection(ctx, &milvuspb.DescribeCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_DescribeCollection,
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataCoordCfg.NodeID,
|
||||
},
|
||||
DbName: "",
|
||||
CollectionID: collectionID,
|
||||
@ -782,7 +785,7 @@ func (s *Server) loadCollectionFromRootCoord(ctx context.Context, collectionID i
|
||||
MsgType: commonpb.MsgType_ShowPartitions,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataCoordCfg.NodeID,
|
||||
},
|
||||
DbName: "",
|
||||
CollectionName: resp.Schema.Name,
|
||||
|
||||
@ -58,7 +58,7 @@ func TestGetSegmentInfoChannel(t *testing.T) {
|
||||
resp, err := svr.GetSegmentInfoChannel(context.TODO())
|
||||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
|
||||
assert.EqualValues(t, Params.SegmentInfoChannelName, resp.Value)
|
||||
assert.EqualValues(t, Params.DataCoordCfg.SegmentInfoChannelName, resp.Value)
|
||||
})
|
||||
}
|
||||
|
||||
@ -245,7 +245,7 @@ func TestGetTimeTickChannel(t *testing.T) {
|
||||
resp, err := svr.GetTimeTickChannel(context.TODO())
|
||||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
|
||||
assert.EqualValues(t, Params.TimeTickChannelName, resp.Value)
|
||||
assert.EqualValues(t, Params.DataCoordCfg.TimeTickChannelName, resp.Value)
|
||||
}
|
||||
|
||||
func TestGetSegmentStates(t *testing.T) {
|
||||
@ -1064,7 +1064,7 @@ func TestDataNodeTtChannel(t *testing.T) {
|
||||
|
||||
ttMsgStream, err := svr.msFactory.NewMsgStream(context.TODO())
|
||||
assert.Nil(t, err)
|
||||
ttMsgStream.AsProducer([]string{Params.TimeTickChannelName})
|
||||
ttMsgStream.AsProducer([]string{Params.DataCoordCfg.TimeTickChannelName})
|
||||
ttMsgStream.Start()
|
||||
defer ttMsgStream.Close()
|
||||
info := &NodeInfo{
|
||||
@ -1132,7 +1132,7 @@ func TestDataNodeTtChannel(t *testing.T) {
|
||||
})
|
||||
ttMsgStream, err := svr.msFactory.NewMsgStream(context.TODO())
|
||||
assert.Nil(t, err)
|
||||
ttMsgStream.AsProducer([]string{Params.TimeTickChannelName})
|
||||
ttMsgStream.AsProducer([]string{Params.DataCoordCfg.TimeTickChannelName})
|
||||
ttMsgStream.Start()
|
||||
defer ttMsgStream.Close()
|
||||
info := &NodeInfo{
|
||||
@ -1214,7 +1214,7 @@ func TestDataNodeTtChannel(t *testing.T) {
|
||||
|
||||
ttMsgStream, err := svr.msFactory.NewMsgStream(context.TODO())
|
||||
assert.Nil(t, err)
|
||||
ttMsgStream.AsProducer([]string{Params.TimeTickChannelName})
|
||||
ttMsgStream.AsProducer([]string{Params.DataCoordCfg.TimeTickChannelName})
|
||||
ttMsgStream.Start()
|
||||
defer ttMsgStream.Close()
|
||||
node := &NodeInfo{
|
||||
@ -1736,7 +1736,7 @@ func TestGetRecoveryInfo(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestGetCompactionState(t *testing.T) {
|
||||
Params.EnableCompaction = true
|
||||
Params.DataCoordCfg.EnableCompaction = true
|
||||
t.Run("test get compaction state with new compactionhandler", func(t *testing.T) {
|
||||
svr := &Server{}
|
||||
svr.isServing = ServerStateHealthy
|
||||
@ -1791,12 +1791,12 @@ func TestGetCompactionState(t *testing.T) {
|
||||
resp, err := svr.GetCompactionState(context.Background(), &milvuspb.GetCompactionStateRequest{})
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetStatus().GetErrorCode())
|
||||
assert.Equal(t, msgDataCoordIsUnhealthy(Params.NodeID), resp.GetStatus().GetReason())
|
||||
assert.Equal(t, msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID), resp.GetStatus().GetReason())
|
||||
})
|
||||
}
|
||||
|
||||
func TestCompleteCompaction(t *testing.T) {
|
||||
Params.EnableCompaction = true
|
||||
Params.DataCoordCfg.EnableCompaction = true
|
||||
t.Run("test complete compaction successfully", func(t *testing.T) {
|
||||
svr := &Server{}
|
||||
svr.isServing = ServerStateHealthy
|
||||
@ -1835,12 +1835,12 @@ func TestCompleteCompaction(t *testing.T) {
|
||||
resp, err := svr.CompleteCompaction(context.Background(), &datapb.CompactionResult{})
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode())
|
||||
assert.Equal(t, msgDataCoordIsUnhealthy(Params.NodeID), resp.GetReason())
|
||||
assert.Equal(t, msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID), resp.GetReason())
|
||||
})
|
||||
}
|
||||
|
||||
func TestManualCompaction(t *testing.T) {
|
||||
Params.EnableCompaction = true
|
||||
Params.DataCoordCfg.EnableCompaction = true
|
||||
t.Run("test manual compaction successfully", func(t *testing.T) {
|
||||
svr := &Server{allocator: &MockAllocator{}}
|
||||
svr.isServing = ServerStateHealthy
|
||||
@ -1896,7 +1896,7 @@ func TestManualCompaction(t *testing.T) {
|
||||
})
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.ErrorCode)
|
||||
assert.Equal(t, msgDataCoordIsUnhealthy(Params.NodeID), resp.Status.Reason)
|
||||
assert.Equal(t, msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID), resp.Status.Reason)
|
||||
})
|
||||
}
|
||||
|
||||
@ -1946,7 +1946,7 @@ func TestGetCompactionStateWithPlans(t *testing.T) {
|
||||
})
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.ErrorCode)
|
||||
assert.Equal(t, msgDataCoordIsUnhealthy(Params.NodeID), resp.Status.Reason)
|
||||
assert.Equal(t, msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID), resp.Status.Reason)
|
||||
})
|
||||
}
|
||||
|
||||
@ -2235,20 +2235,20 @@ func TestGetFlushState(t *testing.T) {
|
||||
|
||||
func newTestServer(t *testing.T, receiveCh chan interface{}, opts ...Option) *Server {
|
||||
Params.Init()
|
||||
Params.TimeTickChannelName = Params.TimeTickChannelName + strconv.Itoa(rand.Int())
|
||||
Params.DataCoordCfg.TimeTickChannelName = Params.DataCoordCfg.TimeTickChannelName + strconv.Itoa(rand.Int())
|
||||
var err error
|
||||
factory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
"pulsarAddress": Params.DataCoordCfg.PulsarAddress,
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarBufSize": 1024,
|
||||
}
|
||||
err = factory.SetParams(m)
|
||||
assert.Nil(t, err)
|
||||
|
||||
etcdCli, err := initEtcd(Params.EtcdEndpoints)
|
||||
etcdCli, err := initEtcd(Params.DataCoordCfg.EtcdEndpoints)
|
||||
assert.Nil(t, err)
|
||||
sessKey := path.Join(Params.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||
sessKey := path.Join(Params.DataCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
|
||||
assert.Nil(t, err)
|
||||
|
||||
|
||||
@ -48,7 +48,7 @@ func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringRespon
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
},
|
||||
Value: Params.TimeTickChannelName,
|
||||
Value: Params.DataCoordCfg.TimeTickChannelName,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -272,7 +272,7 @@ func (s *Server) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringRes
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
},
|
||||
Value: Params.SegmentInfoChannelName,
|
||||
Value: Params.DataCoordCfg.SegmentInfoChannelName,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -366,7 +366,7 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
||||
s.segmentManager.DropSegment(ctx, req.SegmentID)
|
||||
s.flushCh <- req.SegmentID
|
||||
|
||||
if Params.EnableCompaction {
|
||||
if Params.DataCoordCfg.EnableCompaction {
|
||||
cctx, cancel := context.WithTimeout(s.ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
@ -573,7 +573,7 @@ func (s *Server) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInf
|
||||
dresp, err := s.rootCoordClient.DescribeCollection(s.ctx, &milvuspb.DescribeCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_DescribeCollection,
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataCoordCfg.NodeID,
|
||||
},
|
||||
CollectionID: collectionID,
|
||||
})
|
||||
@ -644,19 +644,19 @@ func (s *Server) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedS
|
||||
// it may include SystemMetrics, Topology metrics, etc.
|
||||
func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
||||
log.Debug("received get metrics request",
|
||||
zap.Int64("nodeID", Params.NodeID),
|
||||
zap.Int64("nodeID", Params.DataCoordCfg.NodeID),
|
||||
zap.String("request", req.Request))
|
||||
|
||||
if s.isClosed() {
|
||||
log.Warn("DataCoord.GetMetrics failed",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataCoordCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.NodeID)))
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)))
|
||||
|
||||
return &milvuspb.GetMetricsResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: msgDataCoordIsUnhealthy(Params.NodeID),
|
||||
Reason: msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID),
|
||||
},
|
||||
Response: "",
|
||||
}, nil
|
||||
@ -665,7 +665,7 @@ func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
|
||||
metricType, err := metricsinfo.ParseMetricType(req.Request)
|
||||
if err != nil {
|
||||
log.Warn("DataCoord.GetMetrics failed to parse metric type",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataCoordCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.Error(err))
|
||||
|
||||
@ -692,7 +692,7 @@ func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
|
||||
metrics, err := s.getSystemInfoMetrics(ctx, req)
|
||||
|
||||
log.Debug("DataCoord.GetMetrics",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataCoordCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType),
|
||||
zap.Any("metrics", metrics), // TODO(dragondriver): necessary? may be very large
|
||||
@ -704,7 +704,7 @@ func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
|
||||
}
|
||||
|
||||
log.Debug("DataCoord.GetMetrics failed, request metric type is not implemented yet",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataCoordCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType))
|
||||
|
||||
@ -727,13 +727,13 @@ func (s *Server) CompleteCompaction(ctx context.Context, req *datapb.CompactionR
|
||||
|
||||
if s.isClosed() {
|
||||
log.Warn("failed to complete compaction", zap.Int64("planID", req.PlanID),
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.NodeID)))
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)))
|
||||
|
||||
resp.Reason = msgDataCoordIsUnhealthy(Params.NodeID)
|
||||
resp.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
if !Params.EnableCompaction {
|
||||
if !Params.DataCoordCfg.EnableCompaction {
|
||||
resp.Reason = "compaction disabled"
|
||||
return resp, nil
|
||||
}
|
||||
@ -761,12 +761,12 @@ func (s *Server) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompa
|
||||
|
||||
if s.isClosed() {
|
||||
log.Warn("failed to execute manual compaction", zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.NodeID)))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.NodeID)
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
if !Params.EnableCompaction {
|
||||
if !Params.DataCoordCfg.EnableCompaction {
|
||||
resp.Status.Reason = "compaction disabled"
|
||||
return resp, nil
|
||||
}
|
||||
@ -802,12 +802,12 @@ func (s *Server) GetCompactionState(ctx context.Context, req *milvuspb.GetCompac
|
||||
|
||||
if s.isClosed() {
|
||||
log.Warn("failed to get compaction state", zap.Int64("compactionID", req.GetCompactionID()),
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.NodeID)))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.NodeID)
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
if !Params.EnableCompaction {
|
||||
if !Params.DataCoordCfg.EnableCompaction {
|
||||
resp.Status.Reason = "compaction disabled"
|
||||
return resp, nil
|
||||
}
|
||||
@ -834,12 +834,12 @@ func (s *Server) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.
|
||||
}
|
||||
|
||||
if s.isClosed() {
|
||||
log.Warn("failed to get compaction state with plans", zap.Int64("compactionID", req.GetCompactionID()), zap.Error(errDataCoordIsUnhealthy(Params.NodeID)))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.NodeID)
|
||||
log.Warn("failed to get compaction state with plans", zap.Int64("compactionID", req.GetCompactionID()), zap.Error(errDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
if !Params.EnableCompaction {
|
||||
if !Params.DataCoordCfg.EnableCompaction {
|
||||
resp.Status.Reason = "compaction disabled"
|
||||
return resp, nil
|
||||
}
|
||||
@ -905,8 +905,8 @@ func (s *Server) WatchChannels(ctx context.Context, req *datapb.WatchChannelsReq
|
||||
|
||||
if s.isClosed() {
|
||||
log.Warn("failed to watch channels request", zap.Any("channels", req.GetChannelNames()),
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.NodeID)))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.NodeID)
|
||||
zap.Error(errDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)
|
||||
return resp, nil
|
||||
}
|
||||
for _, channelName := range req.GetChannelNames() {
|
||||
@ -934,7 +934,7 @@ func (s *Server) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateR
|
||||
if s.isClosed() {
|
||||
log.Warn("failed to get flush state because of closed server",
|
||||
zap.Int64s("segmentIDs", req.GetSegmentIDs()), zap.Int("len", len(req.GetSegmentIDs())))
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.NodeID)
|
||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
|
||||
@ -123,7 +123,7 @@ func getTimetravelReverseTime(ctx context.Context, allocator allocator) (*timetr
|
||||
}
|
||||
|
||||
pts, _ := tsoutil.ParseTS(ts)
|
||||
ttpts := pts.Add(-time.Duration(Params.CompactionRetentionDuration) * time.Second)
|
||||
ttpts := pts.Add(-time.Duration(Params.DataCoordCfg.CompactionRetentionDuration) * time.Second)
|
||||
tt := tsoutil.ComposeTS(ttpts.UnixNano()/int64(time.Millisecond), 0)
|
||||
return &timetravel{tt}, nil
|
||||
}
|
||||
|
||||
@ -113,10 +113,10 @@ func TestVerifyResponse(t *testing.T) {
|
||||
|
||||
func Test_getTimetravelReverseTime(t *testing.T) {
|
||||
Params.Init()
|
||||
Params.CompactionRetentionDuration = 43200 // 5 days
|
||||
Params.DataCoordCfg.CompactionRetentionDuration = 43200 // 5 days
|
||||
|
||||
tFixed := time.Date(2021, 11, 15, 0, 0, 0, 0, time.Local)
|
||||
tBefore := tFixed.Add(-time.Duration(Params.CompactionRetentionDuration) * time.Second)
|
||||
tBefore := tFixed.Add(-time.Duration(Params.DataCoordCfg.CompactionRetentionDuration) * time.Second)
|
||||
|
||||
type args struct {
|
||||
allocator allocator
|
||||
|
||||
@ -54,7 +54,7 @@ func (alloc *allocator) allocID() (UniqueID, error) {
|
||||
MsgType: commonpb.MsgType_RequestID,
|
||||
MsgID: 1, // GOOSE TODO
|
||||
Timestamp: 0, // GOOSE TODO
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataNodeCfg.NodeID,
|
||||
},
|
||||
Count: 1,
|
||||
})
|
||||
@ -76,7 +76,7 @@ func (alloc *allocator) allocIDBatch(count uint32) (UniqueID, uint32, error) {
|
||||
resp, err := alloc.rootCoord.AllocID(ctx, &rootcoordpb.AllocIDRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_RequestID,
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataNodeCfg.NodeID,
|
||||
},
|
||||
Count: count,
|
||||
})
|
||||
|
||||
@ -202,7 +202,7 @@ func (b *binlogIO) genDeltaBlobs(data *DeleteData, collID, partID, segID UniqueI
|
||||
return "", nil, err
|
||||
}
|
||||
|
||||
key := path.Join(Params.DeleteBinlogRootPath, k)
|
||||
key := path.Join(Params.DataNodeCfg.DeleteBinlogRootPath, k)
|
||||
|
||||
return key, blob.GetValue(), nil
|
||||
}
|
||||
@ -231,7 +231,7 @@ func (b *binlogIO) genInsertBlobs(data *InsertData, partID, segID UniqueID, meta
|
||||
// Blob Key is generated by Serialize from int64 fieldID in collection schema, which won't raise error in ParseInt
|
||||
fID, _ := strconv.ParseInt(blob.GetKey(), 10, 64)
|
||||
k := JoinIDPath(meta.GetID(), partID, segID, fID, <-generator)
|
||||
key := path.Join(Params.InsertBinlogRootPath, k)
|
||||
key := path.Join(Params.DataNodeCfg.InsertBinlogRootPath, k)
|
||||
|
||||
value := bytes.NewBuffer(blob.GetValue()).String()
|
||||
fileLen := len(value)
|
||||
@ -253,7 +253,7 @@ func (b *binlogIO) genInsertBlobs(data *InsertData, partID, segID UniqueID, meta
|
||||
fID, _ := strconv.ParseInt(blob.GetKey(), 10, 64)
|
||||
|
||||
k := JoinIDPath(meta.GetID(), partID, segID, fID, <-generator)
|
||||
key := path.Join(Params.StatsBinlogRootPath, k)
|
||||
key := path.Join(Params.DataNodeCfg.StatsBinlogRootPath, k)
|
||||
|
||||
value := bytes.NewBuffer(blob.GetValue()).String()
|
||||
fileLen := len(value)
|
||||
|
||||
@ -227,7 +227,7 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[UniqueID]Timestamp,
|
||||
|
||||
// calculate numRows from rowID field, fieldID 0
|
||||
numRows := int64(len(fID2Content[0]))
|
||||
num = int(Params.FlushInsertBufferSize / (int64(dim) * 4))
|
||||
num = int(Params.DataNodeCfg.FlushInsertBufferSize / (int64(dim) * 4))
|
||||
n = int(numRows)/num + 1
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
|
||||
@ -33,14 +33,8 @@ import (
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
|
||||
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
miniokv "github.com/milvus-io/milvus/internal/kv/minio"
|
||||
@ -48,17 +42,20 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/logutil"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -80,6 +77,8 @@ const illegalRequestErrStr = "Illegal request"
|
||||
// makes sure DataNode implements types.DataNode
|
||||
var _ types.DataNode = (*DataNode)(nil)
|
||||
|
||||
var Params paramtable.GlobalParamTable
|
||||
|
||||
// DataNode communicates with outside services and unioun all
|
||||
// services in datanode package.
|
||||
//
|
||||
@ -190,32 +189,33 @@ func (node *DataNode) Register() error {
|
||||
}
|
||||
|
||||
func (node *DataNode) initSession() error {
|
||||
node.session = sessionutil.NewSession(node.ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
node.session = sessionutil.NewSession(node.ctx, Params.DataNodeCfg.MetaRootPath, Params.DataNodeCfg.EtcdEndpoints)
|
||||
if node.session == nil {
|
||||
return errors.New("failed to initialize session")
|
||||
}
|
||||
node.session.Init(typeutil.DataNodeRole, Params.IP+":"+strconv.Itoa(Params.Port), false)
|
||||
Params.NodeID = node.session.ServerID
|
||||
node.session.Init(typeutil.DataNodeRole, Params.DataNodeCfg.IP+":"+strconv.Itoa(Params.DataNodeCfg.Port), false)
|
||||
Params.DataNodeCfg.NodeID = node.session.ServerID
|
||||
node.NodeID = node.session.ServerID
|
||||
Params.SetLogger(Params.NodeID)
|
||||
Params.BaseParams.SetLogger(Params.DataNodeCfg.NodeID)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Init function does nothing now.
|
||||
func (node *DataNode) Init() error {
|
||||
log.Debug("DataNode Init",
|
||||
zap.String("TimeTickChannelName", Params.TimeTickChannelName),
|
||||
zap.String("TimeTickChannelName", Params.DataNodeCfg.TimeTickChannelName),
|
||||
)
|
||||
if err := node.initSession(); err != nil {
|
||||
log.Error("DataNode init session failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
Params.initMsgChannelSubName()
|
||||
// TODO: caiyd need check
|
||||
//Params.initMsgChannelSubName()
|
||||
//TODO reset
|
||||
//Params.initLogCfg()
|
||||
log.Debug("DataNode Init",
|
||||
zap.String("MsgChannelSubName", Params.MsgChannelSubName),
|
||||
zap.String("MsgChannelSubName", Params.DataNodeCfg.MsgChannelSubName),
|
||||
)
|
||||
|
||||
return nil
|
||||
@ -225,7 +225,7 @@ func (node *DataNode) Init() error {
|
||||
func (node *DataNode) StartWatchChannels(ctx context.Context) {
|
||||
defer logutil.LogPanic()
|
||||
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
|
||||
watchPrefix := path.Join(Params.ChannelWatchSubPath, fmt.Sprintf("%d", node.NodeID))
|
||||
watchPrefix := path.Join(Params.DataNodeCfg.ChannelWatchSubPath, fmt.Sprintf("%d", node.NodeID))
|
||||
evtChan := node.watchKv.WatchWithPrefix(watchPrefix)
|
||||
// after watch, first check all exists nodes first
|
||||
err := node.checkWatchedList()
|
||||
@ -265,7 +265,7 @@ func (node *DataNode) StartWatchChannels(ctx context.Context) {
|
||||
// serves the corner case for etcd connection lost and missing some events
|
||||
func (node *DataNode) checkWatchedList() error {
|
||||
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
|
||||
prefix := path.Join(Params.ChannelWatchSubPath, fmt.Sprintf("%d", node.NodeID))
|
||||
prefix := path.Join(Params.DataNodeCfg.ChannelWatchSubPath, fmt.Sprintf("%d", node.NodeID))
|
||||
keys, values, err := node.watchKv.LoadWithPrefix(prefix)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -285,7 +285,7 @@ func (node *DataNode) handleChannelEvt(evt *clientv3.Event) {
|
||||
// guaranteed there is no "/" in channel name
|
||||
parts := strings.Split(string(evt.Kv.Key), "/")
|
||||
vchanName := parts[len(parts)-1]
|
||||
log.Warn("handle channel delete event", zap.Int64("node id", Params.NodeID), zap.String("vchannel", vchanName))
|
||||
log.Warn("handle channel delete event", zap.Int64("node id", Params.DataNodeCfg.NodeID), zap.String("vchannel", vchanName))
|
||||
node.ReleaseDataSyncService(vchanName)
|
||||
}
|
||||
}
|
||||
@ -315,7 +315,7 @@ func (node *DataNode) handleWatchInfo(key string, data []byte) {
|
||||
log.Warn("fail to Marshal watchInfo", zap.String("key", key), zap.Error(err))
|
||||
return
|
||||
}
|
||||
k := path.Join(Params.ChannelWatchSubPath, fmt.Sprintf("%d", node.NodeID), watchInfo.GetVchan().GetChannelName())
|
||||
k := path.Join(Params.DataNodeCfg.ChannelWatchSubPath, fmt.Sprintf("%d", node.NodeID), watchInfo.GetVchan().GetChannelName())
|
||||
err = node.watchKv.Save(k, string(v))
|
||||
if err != nil {
|
||||
log.Warn("fail to change WatchState to complete", zap.String("key", key), zap.Error(err))
|
||||
@ -414,7 +414,7 @@ func (node *DataNode) Start() error {
|
||||
}
|
||||
|
||||
connectEtcdFn := func() error {
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, Params.DataNodeCfg.MetaRootPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -427,12 +427,12 @@ func (node *DataNode) Start() error {
|
||||
}
|
||||
|
||||
option := &miniokv.Option{
|
||||
Address: Params.MinioAddress,
|
||||
AccessKeyID: Params.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinioSecretAccessKey,
|
||||
UseSSL: Params.MinioUseSSL,
|
||||
Address: Params.DataNodeCfg.MinioAddress,
|
||||
AccessKeyID: Params.DataNodeCfg.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.DataNodeCfg.MinioSecretAccessKey,
|
||||
UseSSL: Params.DataNodeCfg.MinioUseSSL,
|
||||
CreateBucket: true,
|
||||
BucketName: Params.MinioBucketName,
|
||||
BucketName: Params.DataNodeCfg.MinioBucketName,
|
||||
}
|
||||
|
||||
kv, err := miniokv.NewMinIOKV(node.ctx, option)
|
||||
@ -455,8 +455,8 @@ func (node *DataNode) Start() error {
|
||||
// Start node watch node
|
||||
go node.StartWatchChannels(node.ctx)
|
||||
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
Params.DataNodeCfg.CreatedTime = time.Now()
|
||||
Params.DataNodeCfg.UpdatedTime = time.Now()
|
||||
|
||||
node.UpdateStateCode(internalpb.StateCode_Healthy)
|
||||
return nil
|
||||
@ -691,19 +691,19 @@ func (node *DataNode) GetStatisticsChannel(ctx context.Context) (*milvuspb.Strin
|
||||
// TODO(dragondriver): cache the Metrics and set a retention to the cache
|
||||
func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
||||
log.Debug("DataNode.GetMetrics",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataNodeCfg.NodeID),
|
||||
zap.String("req", req.Request))
|
||||
|
||||
if !node.isHealthy() {
|
||||
log.Warn("DataNode.GetMetrics failed",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.Error(errDataNodeIsUnhealthy(Params.NodeID)))
|
||||
zap.Error(errDataNodeIsUnhealthy(Params.DataNodeCfg.NodeID)))
|
||||
|
||||
return &milvuspb.GetMetricsResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: msgDataNodeIsUnhealthy(Params.NodeID),
|
||||
Reason: msgDataNodeIsUnhealthy(Params.DataNodeCfg.NodeID),
|
||||
},
|
||||
Response: "",
|
||||
}, nil
|
||||
@ -712,7 +712,7 @@ func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRe
|
||||
metricType, err := metricsinfo.ParseMetricType(req.Request)
|
||||
if err != nil {
|
||||
log.Warn("DataNode.GetMetrics failed to parse metric type",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.Error(err))
|
||||
|
||||
@ -732,7 +732,7 @@ func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRe
|
||||
systemInfoMetrics, err := node.getSystemInfoMetrics(ctx, req)
|
||||
|
||||
log.Debug("DataNode.GetMetrics",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType),
|
||||
zap.Any("systemInfoMetrics", systemInfoMetrics), // TODO(dragondriver): necessary? may be very large
|
||||
@ -742,7 +742,7 @@ func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRe
|
||||
}
|
||||
|
||||
log.Debug("DataNode.GetMetrics failed, request metric type is not implemented yet",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.DataNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType))
|
||||
|
||||
|
||||
@ -51,10 +51,10 @@ import (
|
||||
|
||||
func TestMain(t *testing.M) {
|
||||
rand.Seed(time.Now().Unix())
|
||||
Params.InitAlias("datanode-alias-1")
|
||||
Params.DataNodeCfg.InitAlias("datanode-alias-1")
|
||||
Params.Init()
|
||||
// change to specific channel for test
|
||||
Params.TimeTickChannelName = Params.TimeTickChannelName + strconv.Itoa(rand.Int())
|
||||
Params.DataNodeCfg.TimeTickChannelName = Params.DataNodeCfg.TimeTickChannelName + strconv.Itoa(rand.Int())
|
||||
code := t.Run()
|
||||
os.Exit(code)
|
||||
}
|
||||
@ -229,7 +229,7 @@ func TestDataNode(t *testing.T) {
|
||||
// pulsar produce
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
"pulsarAddress": Params.DataNodeCfg.PulsarAddress,
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarBufSize": 1024}
|
||||
err = msFactory.SetParams(m)
|
||||
@ -485,18 +485,18 @@ func TestWatchChannel(t *testing.T) {
|
||||
defer cancel()
|
||||
|
||||
t.Run("test watch channel", func(t *testing.T) {
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, Params.DataNodeCfg.MetaRootPath)
|
||||
require.NoError(t, err)
|
||||
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
|
||||
path := fmt.Sprintf("%s/%d/%s", Params.ChannelWatchSubPath, node.NodeID, oldInvalidCh)
|
||||
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID, oldInvalidCh)
|
||||
err = kv.Save(path, string([]byte{23}))
|
||||
assert.NoError(t, err)
|
||||
|
||||
ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31())
|
||||
path = fmt.Sprintf("%s/%d/%s", Params.ChannelWatchSubPath, node.NodeID, ch)
|
||||
path = fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID, ch)
|
||||
c := make(chan struct{})
|
||||
go func() {
|
||||
ec := kv.WatchWithPrefix(fmt.Sprintf("%s/%d", Params.ChannelWatchSubPath, node.NodeID))
|
||||
ec := kv.WatchWithPrefix(fmt.Sprintf("%s/%d", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID))
|
||||
c <- struct{}{}
|
||||
cnt := 0
|
||||
for {
|
||||
@ -536,7 +536,7 @@ func TestWatchChannel(t *testing.T) {
|
||||
node.chanMut.RUnlock()
|
||||
assert.True(t, has)
|
||||
|
||||
err = kv.RemoveWithPrefix(fmt.Sprintf("%s/%d", Params.ChannelWatchSubPath, node.NodeID))
|
||||
err = kv.RemoveWithPrefix(fmt.Sprintf("%s/%d", Params.DataNodeCfg.ChannelWatchSubPath, node.NodeID))
|
||||
assert.Nil(t, err)
|
||||
//TODO there is not way to sync Release done, use sleep for now
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
@ -109,7 +109,7 @@ type nodeConfig struct {
|
||||
}
|
||||
|
||||
func newParallelConfig() parallelConfig {
|
||||
return parallelConfig{Params.FlowGraphMaxQueueLength, Params.FlowGraphMaxParallelism}
|
||||
return parallelConfig{Params.DataNodeCfg.FlowGraphMaxQueueLength, Params.DataNodeCfg.FlowGraphMaxParallelism}
|
||||
}
|
||||
|
||||
// start starts the flowgraph in datasyncservice
|
||||
@ -137,7 +137,7 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
|
||||
dsService.fg = flowgraph.NewTimeTickedFlowGraph(dsService.ctx)
|
||||
|
||||
m := map[string]interface{}{
|
||||
"PulsarAddress": Params.PulsarAddress,
|
||||
"PulsarAddress": Params.DataNodeCfg.PulsarAddress,
|
||||
"ReceiveBufSize": 1024,
|
||||
"PulsarBufSize": 1024,
|
||||
}
|
||||
|
||||
@ -177,7 +177,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
||||
defer cancel()
|
||||
|
||||
// init data node
|
||||
pulsarURL := Params.PulsarAddress
|
||||
pulsarURL := Params.DataNodeCfg.PulsarAddress
|
||||
|
||||
Factory := &MetaFactory{}
|
||||
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1")
|
||||
@ -199,7 +199,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
||||
|
||||
insertChannelName := "data_sync_service_test_dml"
|
||||
ddlChannelName := "data_sync_service_test_ddl"
|
||||
Params.FlushInsertBufferSize = 1
|
||||
Params.DataNodeCfg.FlushInsertBufferSize = 1
|
||||
|
||||
ufs := []*datapb.SegmentInfo{{
|
||||
CollectionID: collMeta.ID,
|
||||
|
||||
@ -240,7 +240,7 @@ func (ddn *ddNode) sendDeltaTimeTick(ts Timestamp) error {
|
||||
MsgType: commonpb.MsgType_TimeTick,
|
||||
MsgID: 0,
|
||||
Timestamp: ts,
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataNodeCfg.NodeID,
|
||||
},
|
||||
}
|
||||
timeTickMsg := &msgstream.TimeTickMsg{
|
||||
@ -264,8 +264,8 @@ func (ddn *ddNode) Close() {
|
||||
func newDDNode(ctx context.Context, collID UniqueID, vchanInfo *datapb.VchannelInfo,
|
||||
msFactory msgstream.Factory, compactor *compactionExecutor) *ddNode {
|
||||
baseNode := BaseNode{}
|
||||
baseNode.SetMaxQueueLength(Params.FlowGraphMaxQueueLength)
|
||||
baseNode.SetMaxParallelism(Params.FlowGraphMaxParallelism)
|
||||
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength)
|
||||
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism)
|
||||
|
||||
fs := make([]*datapb.SegmentInfo, 0, len(vchanInfo.GetFlushedSegments()))
|
||||
fs = append(fs, vchanInfo.GetFlushedSegments()...)
|
||||
@ -280,7 +280,7 @@ func newDDNode(ctx context.Context, collID UniqueID, vchanInfo *datapb.VchannelI
|
||||
return nil
|
||||
}
|
||||
pChannelName := rootcoord.ToPhysicalChannel(vchanInfo.ChannelName)
|
||||
deltaChannelName, err := rootcoord.ConvertChannelName(pChannelName, Params.DmlChannelName, Params.DeltaChannelName)
|
||||
deltaChannelName, err := rootcoord.ConvertChannelName(pChannelName, Params.DataNodeCfg.DmlChannelName, Params.DataNodeCfg.DeltaChannelName)
|
||||
if err != nil {
|
||||
log.Error(err.Error())
|
||||
return nil
|
||||
|
||||
@ -236,8 +236,8 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
|
||||
chanName := "datanode-test-FlowGraphDeletenode-operate"
|
||||
testPath := "/test/datanode/root/meta"
|
||||
assert.NoError(t, clearEtcd(testPath))
|
||||
Params.MetaRootPath = testPath
|
||||
Params.DeleteBinlogRootPath = testPath
|
||||
Params.DataNodeCfg.MetaRootPath = testPath
|
||||
Params.DataNodeCfg.DeleteBinlogRootPath = testPath
|
||||
|
||||
c := &nodeConfig{
|
||||
replica: replica,
|
||||
@ -260,8 +260,8 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
|
||||
chanName := "datanode-test-FlowGraphDeletenode-operate"
|
||||
testPath := "/test/datanode/root/meta"
|
||||
assert.NoError(t, clearEtcd(testPath))
|
||||
Params.MetaRootPath = testPath
|
||||
Params.DeleteBinlogRootPath = testPath
|
||||
Params.DataNodeCfg.MetaRootPath = testPath
|
||||
Params.DataNodeCfg.DeleteBinlogRootPath = testPath
|
||||
|
||||
c := &nodeConfig{
|
||||
replica: replica,
|
||||
@ -289,8 +289,8 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
|
||||
chanName := "datanode-test-FlowGraphDeletenode-operate"
|
||||
testPath := "/test/datanode/root/meta"
|
||||
assert.NoError(t, clearEtcd(testPath))
|
||||
Params.MetaRootPath = testPath
|
||||
Params.DeleteBinlogRootPath = testPath
|
||||
Params.DataNodeCfg.MetaRootPath = testPath
|
||||
Params.DataNodeCfg.DeleteBinlogRootPath = testPath
|
||||
|
||||
c := &nodeConfig{
|
||||
replica: replica,
|
||||
|
||||
@ -34,7 +34,7 @@ import (
|
||||
func newDmInputNode(ctx context.Context, seekPos *internalpb.MsgPosition, dmNodeConfig *nodeConfig) (*flowgraph.InputNode, error) {
|
||||
// subName should be unique, since pchannelName is shared among several collections
|
||||
// consumeSubName := Params.MsgChannelSubName + "-" + strconv.FormatInt(collID, 10)
|
||||
consumeSubName := fmt.Sprintf("%s-%d", Params.MsgChannelSubName, dmNodeConfig.collectionID)
|
||||
consumeSubName := fmt.Sprintf("%s-%d", Params.DataNodeCfg.MsgChannelSubName, dmNodeConfig.collectionID)
|
||||
insertStream, err := dmNodeConfig.msFactory.NewTtMsgStream(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
||||
@ -127,7 +127,7 @@ func newBufferData(dimension int64) (*BufferData, error) {
|
||||
return nil, errors.New("Invalid dimension")
|
||||
}
|
||||
|
||||
limit := Params.FlushInsertBufferSize / (dimension * 4)
|
||||
limit := Params.DataNodeCfg.FlushInsertBufferSize / (dimension * 4)
|
||||
|
||||
return &BufferData{&InsertData{Data: make(map[UniqueID]storage.FieldData)}, 0, limit}, nil
|
||||
}
|
||||
@ -721,8 +721,8 @@ func newInsertBufferNode(ctx context.Context, flushCh <-chan flushMsg, fm flushM
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
wTt.AsProducer([]string{Params.TimeTickChannelName})
|
||||
log.Debug("datanode AsProducer", zap.String("TimeTickChannelName", Params.TimeTickChannelName))
|
||||
wTt.AsProducer([]string{Params.DataNodeCfg.TimeTickChannelName})
|
||||
log.Debug("datanode AsProducer", zap.String("TimeTickChannelName", Params.DataNodeCfg.TimeTickChannelName))
|
||||
var wTtMsgStream msgstream.MsgStream = wTt
|
||||
wTtMsgStream.Start()
|
||||
|
||||
|
||||
@ -61,7 +61,7 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
|
||||
testPath := "/test/datanode/root/meta"
|
||||
err := clearEtcd(testPath)
|
||||
require.NoError(t, err)
|
||||
Params.MetaRootPath = testPath
|
||||
Params.DataNodeCfg.MetaRootPath = testPath
|
||||
|
||||
Factory := &MetaFactory{}
|
||||
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1")
|
||||
@ -76,7 +76,7 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
"pulsarAddress": Params.DataNodeCfg.PulsarAddress,
|
||||
"pulsarBufSize": 1024}
|
||||
err = msFactory.SetParams(m)
|
||||
assert.Nil(t, err)
|
||||
@ -151,7 +151,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
||||
testPath := "/test/datanode/root/meta"
|
||||
err := clearEtcd(testPath)
|
||||
require.NoError(t, err)
|
||||
Params.MetaRootPath = testPath
|
||||
Params.DataNodeCfg.MetaRootPath = testPath
|
||||
|
||||
Factory := &MetaFactory{}
|
||||
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1")
|
||||
@ -166,7 +166,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
"pulsarAddress": Params.DataNodeCfg.PulsarAddress,
|
||||
"pulsarBufSize": 1024}
|
||||
err = msFactory.SetParams(m)
|
||||
assert.Nil(t, err)
|
||||
@ -352,7 +352,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
|
||||
testPath := "/test/datanode/root/meta"
|
||||
err := clearEtcd(testPath)
|
||||
require.NoError(t, err)
|
||||
Params.MetaRootPath = testPath
|
||||
Params.DataNodeCfg.MetaRootPath = testPath
|
||||
|
||||
Factory := &MetaFactory{}
|
||||
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1")
|
||||
@ -372,7 +372,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
"pulsarAddress": Params.DataNodeCfg.PulsarAddress,
|
||||
"pulsarBufSize": 1024}
|
||||
err = msFactory.SetParams(m)
|
||||
assert.Nil(t, err)
|
||||
@ -412,10 +412,10 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
|
||||
|
||||
t.Run("Pure auto flush", func(t *testing.T) {
|
||||
// iBNode.insertBuffer.maxSize = 2
|
||||
tmp := Params.FlushInsertBufferSize
|
||||
Params.FlushInsertBufferSize = 4 * 4
|
||||
tmp := Params.DataNodeCfg.FlushInsertBufferSize
|
||||
Params.DataNodeCfg.FlushInsertBufferSize = 4 * 4
|
||||
defer func() {
|
||||
Params.FlushInsertBufferSize = tmp
|
||||
Params.DataNodeCfg.FlushInsertBufferSize = tmp
|
||||
}()
|
||||
|
||||
for i := range inMsg.insertMessages {
|
||||
@ -513,10 +513,10 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
|
||||
})
|
||||
|
||||
t.Run("Auto with manual flush", func(t *testing.T) {
|
||||
tmp := Params.FlushInsertBufferSize
|
||||
Params.FlushInsertBufferSize = 4 * 4
|
||||
tmp := Params.DataNodeCfg.FlushInsertBufferSize
|
||||
Params.DataNodeCfg.FlushInsertBufferSize = 4 * 4
|
||||
defer func() {
|
||||
Params.FlushInsertBufferSize = tmp
|
||||
Params.DataNodeCfg.FlushInsertBufferSize = tmp
|
||||
}()
|
||||
|
||||
fpMut.Lock()
|
||||
@ -626,7 +626,7 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
|
||||
testPath := "/test/datanode/root/meta"
|
||||
err := clearEtcd(testPath)
|
||||
require.NoError(t, err)
|
||||
Params.MetaRootPath = testPath
|
||||
Params.DataNodeCfg.MetaRootPath = testPath
|
||||
|
||||
Factory := &MetaFactory{}
|
||||
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1")
|
||||
@ -646,7 +646,7 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
"pulsarAddress": Params.DataNodeCfg.PulsarAddress,
|
||||
"pulsarBufSize": 1024}
|
||||
err = msFactory.SetParams(m)
|
||||
assert.Nil(t, err)
|
||||
@ -718,7 +718,7 @@ func TestInsertBufferNode_updateSegStatesInReplica(te *testing.T) {
|
||||
}
|
||||
|
||||
func TestInsertBufferNode_BufferData(te *testing.T) {
|
||||
Params.FlushInsertBufferSize = 16 * (1 << 20) // 16 MB
|
||||
Params.DataNodeCfg.FlushInsertBufferSize = 16 * (1 << 20) // 16 MB
|
||||
|
||||
tests := []struct {
|
||||
isValid bool
|
||||
|
||||
@ -351,7 +351,7 @@ func (m *rendezvousFlushManager) flushBufferData(data *BufferData, segmentID Uni
|
||||
// no error raise if alloc=false
|
||||
k := JoinIDPath(collID, partID, segmentID, fieldID, logidx)
|
||||
|
||||
key := path.Join(Params.InsertBinlogRootPath, k)
|
||||
key := path.Join(Params.DataNodeCfg.InsertBinlogRootPath, k)
|
||||
kvs[key] = string(blob.Value[:])
|
||||
field2Insert[fieldID] = &datapb.Binlog{
|
||||
EntriesNum: data.size,
|
||||
@ -377,7 +377,7 @@ func (m *rendezvousFlushManager) flushBufferData(data *BufferData, segmentID Uni
|
||||
// no error raise if alloc=false
|
||||
k := JoinIDPath(collID, partID, segmentID, fieldID, logidx)
|
||||
|
||||
key := path.Join(Params.StatsBinlogRootPath, k)
|
||||
key := path.Join(Params.DataNodeCfg.StatsBinlogRootPath, k)
|
||||
kvs[key] = string(blob.Value)
|
||||
field2Stats[fieldID] = &datapb.Binlog{
|
||||
EntriesNum: 0,
|
||||
@ -425,7 +425,7 @@ func (m *rendezvousFlushManager) flushDelData(data *DelDataBuf, segmentID Unique
|
||||
}
|
||||
|
||||
blobKey := JoinIDPath(collID, partID, segmentID, logID)
|
||||
blobPath := path.Join(Params.DeleteBinlogRootPath, blobKey)
|
||||
blobPath := path.Join(Params.DataNodeCfg.DeleteBinlogRootPath, blobKey)
|
||||
kvs := map[string]string{blobPath: string(blob.Value[:])}
|
||||
data.LogSize = int64(len(blob.Value))
|
||||
data.LogPath = blobPath
|
||||
@ -583,7 +583,7 @@ func dropVirtualChannelFunc(dsService *dataSyncService, opts ...retry.Option) fl
|
||||
MsgType: 0, //TODO msg type
|
||||
MsgID: 0, //TODO msg id
|
||||
Timestamp: 0, //TODO time stamp
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataNodeCfg.NodeID,
|
||||
},
|
||||
ChannelName: dsService.vchannelName,
|
||||
}
|
||||
@ -715,7 +715,7 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet
|
||||
MsgType: 0, //TODO msg type
|
||||
MsgID: 0, //TODO msg id
|
||||
Timestamp: 0, //TODO time stamp
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataNodeCfg.NodeID,
|
||||
},
|
||||
SegmentID: pack.segmentID,
|
||||
CollectionID: dsService.collectionID,
|
||||
|
||||
@ -53,7 +53,7 @@ func (mService *metaService) getCollectionSchema(ctx context.Context, collID Uni
|
||||
MsgType: commonpb.MsgType_DescribeCollection,
|
||||
MsgID: 0, //GOOSE TODO
|
||||
Timestamp: 0, // GOOSE TODO
|
||||
SourceID: Params.NodeID,
|
||||
SourceID: Params.DataNodeCfg.NodeID,
|
||||
},
|
||||
DbName: "default", // GOOSE TODO
|
||||
CollectionID: collID,
|
||||
|
||||
@ -29,7 +29,7 @@ func (node *DataNode) getSystemInfoMetrics(ctx context.Context, req *milvuspb.Ge
|
||||
// TODO(dragondriver): add more metrics
|
||||
nodeInfos := metricsinfo.DataNodeInfos{
|
||||
BaseComponentInfos: metricsinfo.BaseComponentInfos{
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, Params.NodeID),
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, Params.DataNodeCfg.NodeID),
|
||||
HardwareInfos: metricsinfo.HardwareMetrics{
|
||||
IP: node.session.Address,
|
||||
CPUCoreCount: metricsinfo.GetCPUCoreCount(false),
|
||||
@ -40,13 +40,13 @@ func (node *DataNode) getSystemInfoMetrics(ctx context.Context, req *milvuspb.Ge
|
||||
DiskUsage: metricsinfo.GetDiskUsage(),
|
||||
},
|
||||
SystemInfo: metricsinfo.DeployMetrics{},
|
||||
CreatedTime: Params.CreatedTime.String(),
|
||||
UpdatedTime: Params.UpdatedTime.String(),
|
||||
CreatedTime: Params.DataNodeCfg.CreatedTime.String(),
|
||||
UpdatedTime: Params.DataNodeCfg.UpdatedTime.String(),
|
||||
Type: typeutil.DataNodeRole,
|
||||
ID: node.session.ServerID,
|
||||
},
|
||||
SystemConfigurations: metricsinfo.DataNodeConfiguration{
|
||||
FlushInsertBufferSize: Params.FlushInsertBufferSize,
|
||||
FlushInsertBufferSize: Params.DataNodeCfg.FlushInsertBufferSize,
|
||||
},
|
||||
}
|
||||
|
||||
@ -60,7 +60,7 @@ func (node *DataNode) getSystemInfoMetrics(ctx context.Context, req *milvuspb.Ge
|
||||
Reason: err.Error(),
|
||||
},
|
||||
Response: "",
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, Params.NodeID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, Params.DataNodeCfg.NodeID),
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -70,6 +70,6 @@ func (node *DataNode) getSystemInfoMetrics(ctx context.Context, req *milvuspb.Ge
|
||||
Reason: "",
|
||||
},
|
||||
Response: resp,
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, Params.NodeID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, Params.DataNodeCfg.NodeID),
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -106,7 +106,7 @@ func makeNewChannelNames(names []string, suffix string) []string {
|
||||
}
|
||||
|
||||
func clearEtcd(rootPath string) error {
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, rootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.DataNodeCfg.EtcdEndpoints, rootPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
@ -1,298 +0,0 @@
|
||||
// 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 (
|
||||
"path"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
)
|
||||
|
||||
// ParamTable in DataNode contains all configs for DataNode
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
|
||||
// ID of the current DataNode
|
||||
NodeID UniqueID
|
||||
|
||||
// IP of the current DataNode
|
||||
IP string
|
||||
|
||||
// Port of the current DataNode
|
||||
Port int
|
||||
FlowGraphMaxQueueLength int32
|
||||
FlowGraphMaxParallelism int32
|
||||
FlushInsertBufferSize int64
|
||||
InsertBinlogRootPath string
|
||||
StatsBinlogRootPath string
|
||||
DeleteBinlogRootPath string
|
||||
Alias string // Different datanode in one machine
|
||||
|
||||
// Channel Name
|
||||
DmlChannelName string
|
||||
DeltaChannelName string
|
||||
|
||||
// Pulsar address
|
||||
PulsarAddress string
|
||||
|
||||
// Rocksmq path
|
||||
RocksmqPath string
|
||||
|
||||
// Cluster channels
|
||||
ClusterChannelPrefix string
|
||||
|
||||
// Timetick channel
|
||||
TimeTickChannelName string
|
||||
|
||||
// Channel subscribition name -
|
||||
MsgChannelSubName string
|
||||
|
||||
// etcd
|
||||
EtcdEndpoints []string
|
||||
MetaRootPath string
|
||||
ChannelWatchSubPath string
|
||||
|
||||
// MinIO
|
||||
MinioAddress string
|
||||
MinioAccessKeyID string
|
||||
MinioSecretAccessKey string
|
||||
MinioUseSSL bool
|
||||
MinioBucketName string
|
||||
|
||||
CreatedTime time.Time
|
||||
UpdatedTime time.Time
|
||||
}
|
||||
|
||||
// Params is global var in DataNode
|
||||
var Params ParamTable
|
||||
var once sync.Once
|
||||
|
||||
// InitAlias init this DataNode alias
|
||||
func (p *ParamTable) InitAlias(alias string) {
|
||||
p.Alias = alias
|
||||
}
|
||||
|
||||
// InitOnce call params Init only once
|
||||
func (p *ParamTable) InitOnce() {
|
||||
once.Do(func() {
|
||||
p.Init()
|
||||
})
|
||||
}
|
||||
|
||||
// Init initializes DataNode configs
|
||||
func (p *ParamTable) Init() {
|
||||
p.BaseTable.Init()
|
||||
|
||||
p.initFlowGraphMaxQueueLength()
|
||||
p.initFlowGraphMaxParallelism()
|
||||
p.initFlushInsertBufferSize()
|
||||
p.initInsertBinlogRootPath()
|
||||
p.initStatsBinlogRootPath()
|
||||
p.initDeleteBinlogRootPath()
|
||||
|
||||
p.initPulsarAddress()
|
||||
p.initRocksmqPath()
|
||||
|
||||
// Must init global msgchannel prefix before other channel names
|
||||
p.initClusterMsgChannelPrefix()
|
||||
p.initTimeTickChannelName()
|
||||
|
||||
p.initEtcdEndpoints()
|
||||
p.initMetaRootPath()
|
||||
p.initChannelWatchPath()
|
||||
|
||||
p.initMinioAddress()
|
||||
p.initMinioAccessKeyID()
|
||||
p.initMinioSecretAccessKey()
|
||||
p.initMinioUseSSL()
|
||||
p.initMinioBucketName()
|
||||
|
||||
p.initDmlChannelName()
|
||||
p.initDeltaChannelName()
|
||||
|
||||
p.initRoleName()
|
||||
}
|
||||
|
||||
func (p *ParamTable) initFlowGraphMaxQueueLength() {
|
||||
p.FlowGraphMaxQueueLength = p.ParseInt32WithDefault("dataNode.dataSync.flowGraph.maxQueueLength", 1024)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initFlowGraphMaxParallelism() {
|
||||
p.FlowGraphMaxParallelism = p.ParseInt32WithDefault("dataNode.dataSync.flowGraph.maxParallelism", 1024)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initFlushInsertBufferSize() {
|
||||
p.FlushInsertBufferSize = p.ParseInt64("_DATANODE_INSERTBUFSIZE")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initInsertBinlogRootPath() {
|
||||
// GOOSE TODO: rootPath change to TenentID
|
||||
rootPath, err := p.Load("minio.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.InsertBinlogRootPath = path.Join(rootPath, "insert_log")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initStatsBinlogRootPath() {
|
||||
rootPath, err := p.Load("minio.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.StatsBinlogRootPath = path.Join(rootPath, "stats_log")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDeleteBinlogRootPath() {
|
||||
rootPath, err := p.Load("minio.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.DeleteBinlogRootPath = path.Join(rootPath, "delta_log")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initPulsarAddress() {
|
||||
url, err := p.Load("_PulsarAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.PulsarAddress = url
|
||||
}
|
||||
|
||||
func (p *ParamTable) initRocksmqPath() {
|
||||
path, err := p.Load("_RocksmqPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.RocksmqPath = path
|
||||
}
|
||||
|
||||
func (p *ParamTable) initClusterMsgChannelPrefix() {
|
||||
name, err := p.Load("msgChannel.chanNamePrefix.cluster")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.ClusterChannelPrefix = name
|
||||
}
|
||||
|
||||
func (p *ParamTable) initTimeTickChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.dataCoordTimeTick")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.TimeTickChannelName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMsgChannelSubName() {
|
||||
config, err := p.Load("msgChannel.subNamePrefix.dataNodeSubNamePrefix")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config, strconv.FormatInt(p.NodeID, 10)}
|
||||
p.MsgChannelSubName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initEtcdEndpoints() {
|
||||
endpoints, err := p.Load("_EtcdEndpoints")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMetaRootPath() {
|
||||
rootPath, err := p.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := p.Load("etcd.metaSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MetaRootPath = path.Join(rootPath, subPath)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initChannelWatchPath() {
|
||||
p.ChannelWatchSubPath = "channelwatch"
|
||||
}
|
||||
|
||||
// --- MinIO ---
|
||||
func (p *ParamTable) initMinioAddress() {
|
||||
endpoint, err := p.Load("_MinioAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioAddress = endpoint
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioAccessKeyID() {
|
||||
keyID, err := p.Load("_MinioAccessKeyID")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioAccessKeyID = keyID
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioSecretAccessKey() {
|
||||
key, err := p.Load("_MinioSecretAccessKey")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioSecretAccessKey = key
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioUseSSL() {
|
||||
usessl, err := p.Load("_MinioUseSSL")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioUseSSL, _ = strconv.ParseBool(usessl)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioBucketName() {
|
||||
bucketName, err := p.Load("_MinioBucketName")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioBucketName = bucketName
|
||||
}
|
||||
|
||||
func (p *ParamTable) initRoleName() {
|
||||
p.RoleName = "datanode"
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDmlChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.rootCoordDml")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.DmlChannelName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDeltaChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.rootCoordDelta")
|
||||
if err != nil {
|
||||
config = "rootcoord-delta"
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.DeltaChannelName = strings.Join(s, "-")
|
||||
}
|
||||
@ -1,136 +0,0 @@
|
||||
// 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 (
|
||||
"log"
|
||||
"path"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestParamTable(t *testing.T) {
|
||||
Params.Init()
|
||||
Params.NodeID = 2
|
||||
Params.initMsgChannelSubName()
|
||||
t.Run("Test NodeID", func(t *testing.T) {
|
||||
id := Params.NodeID
|
||||
log.Println("NodeID:", id)
|
||||
})
|
||||
|
||||
t.Run("Test Alias", func(t *testing.T) {
|
||||
alias := Params.Alias
|
||||
log.Println("Alias:", alias)
|
||||
|
||||
})
|
||||
|
||||
t.Run("Test flowGraphMaxQueueLength", func(t *testing.T) {
|
||||
length := Params.FlowGraphMaxQueueLength
|
||||
log.Println("flowGraphMaxQueueLength:", length)
|
||||
})
|
||||
|
||||
t.Run("Test flowGraphMaxParallelism", func(t *testing.T) {
|
||||
maxParallelism := Params.FlowGraphMaxParallelism
|
||||
log.Println("flowGraphMaxParallelism:", maxParallelism)
|
||||
})
|
||||
|
||||
t.Run("Test FlushInsertBufSize", func(t *testing.T) {
|
||||
size := Params.FlushInsertBufferSize
|
||||
log.Println("FlushInsertBufferSize:", size)
|
||||
})
|
||||
|
||||
t.Run("Test InsertBinlogRootPath", func(t *testing.T) {
|
||||
path := Params.InsertBinlogRootPath
|
||||
log.Println("InsertBinlogRootPath:", path)
|
||||
})
|
||||
|
||||
t.Run("Test PulsarAddress", func(t *testing.T) {
|
||||
address := Params.PulsarAddress
|
||||
log.Println("PulsarAddress:", address)
|
||||
})
|
||||
|
||||
t.Run("Test ClusterChannelPrefix", func(t *testing.T) {
|
||||
path := Params.ClusterChannelPrefix
|
||||
assert.Equal(t, path, "by-dev")
|
||||
log.Println("ClusterChannelPrefix:", Params.ClusterChannelPrefix)
|
||||
})
|
||||
|
||||
t.Run("Test TimeTickChannelName", func(t *testing.T) {
|
||||
name := Params.TimeTickChannelName
|
||||
assert.Equal(t, name, "by-dev-datacoord-timetick-channel")
|
||||
log.Println("TimeTickChannelName:", name)
|
||||
})
|
||||
|
||||
t.Run("Test msgChannelSubName", func(t *testing.T) {
|
||||
name := Params.MsgChannelSubName
|
||||
assert.Equal(t, name, "by-dev-dataNode-2")
|
||||
log.Println("MsgChannelSubName:", name)
|
||||
})
|
||||
|
||||
t.Run("Test EtcdEndpoints", func(t *testing.T) {
|
||||
endpoints := Params.EtcdEndpoints
|
||||
log.Println("EtcdEndpoints:", endpoints)
|
||||
})
|
||||
|
||||
t.Run("Test MetaRootPath", func(t *testing.T) {
|
||||
path := Params.MetaRootPath
|
||||
log.Println("MetaRootPath:", path)
|
||||
})
|
||||
|
||||
t.Run("Test minioAccessKeyID", func(t *testing.T) {
|
||||
id := Params.MinioAccessKeyID
|
||||
log.Println("MinioAccessKeyID:", id)
|
||||
})
|
||||
|
||||
t.Run("Test minioSecretAccessKey", func(t *testing.T) {
|
||||
key := Params.MinioSecretAccessKey
|
||||
log.Println("MinioSecretAccessKey:", key)
|
||||
})
|
||||
|
||||
t.Run("Test MinioUseSSL", func(t *testing.T) {
|
||||
useSSL := Params.MinioUseSSL
|
||||
log.Println("MinioUseSSL:", useSSL)
|
||||
})
|
||||
|
||||
t.Run("Test MinioBucketName", func(t *testing.T) {
|
||||
name := Params.MinioBucketName
|
||||
log.Println("MinioBucketName:", name)
|
||||
})
|
||||
|
||||
t.Run("Test CreatedTime", func(t *testing.T) {
|
||||
Params.CreatedTime = time.Now()
|
||||
log.Println("CreatedTime: ", Params.CreatedTime)
|
||||
})
|
||||
|
||||
t.Run("Test UpdatedTime", func(t *testing.T) {
|
||||
Params.UpdatedTime = time.Now()
|
||||
log.Println("UpdatedTime: ", Params.UpdatedTime)
|
||||
})
|
||||
|
||||
t.Run("Test InsertBinlogRootPath", func(t *testing.T) {
|
||||
Params.Init()
|
||||
assert.Equal(t, path.Join("files", "insert_log"), Params.InsertBinlogRootPath)
|
||||
})
|
||||
|
||||
t.Run("Test StatsBinlogRootPath", func(t *testing.T) {
|
||||
p := new(ParamTable)
|
||||
p.Init()
|
||||
assert.Equal(t, path.Join("files", "stats_log"), Params.StatsBinlogRootPath)
|
||||
})
|
||||
}
|
||||
@ -131,12 +131,12 @@ var _ Replica = &SegmentReplica{}
|
||||
func newReplica(ctx context.Context, rc types.RootCoord, collID UniqueID) (*SegmentReplica, error) {
|
||||
// MinIO
|
||||
option := &miniokv.Option{
|
||||
Address: Params.MinioAddress,
|
||||
AccessKeyID: Params.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinioSecretAccessKey,
|
||||
UseSSL: Params.MinioUseSSL,
|
||||
Address: Params.DataNodeCfg.MinioAddress,
|
||||
AccessKeyID: Params.DataNodeCfg.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.DataNodeCfg.MinioSecretAccessKey,
|
||||
UseSSL: Params.DataNodeCfg.MinioUseSSL,
|
||||
CreateBucket: true,
|
||||
BucketName: Params.MinioBucketName,
|
||||
BucketName: Params.DataNodeCfg.MinioBucketName,
|
||||
}
|
||||
|
||||
minIOKV, err := miniokv.NewMinIOKV(ctx, option)
|
||||
|
||||
@ -32,7 +32,7 @@ func Test_NewClient(t *testing.T) {
|
||||
proxy.Params.InitOnce()
|
||||
|
||||
ctx := context.Background()
|
||||
client, err := NewClient(ctx, proxy.Params.MetaRootPath, proxy.Params.EtcdEndpoints)
|
||||
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, client)
|
||||
|
||||
|
||||
@ -84,9 +84,9 @@ func (s *Server) init() error {
|
||||
s.closer = closer
|
||||
|
||||
datacoord.Params.InitOnce()
|
||||
datacoord.Params.IP = Params.IP
|
||||
datacoord.Params.Port = Params.Port
|
||||
datacoord.Params.Address = Params.GetAddress()
|
||||
datacoord.Params.DataCoordCfg.IP = Params.IP
|
||||
datacoord.Params.DataCoordCfg.Port = Params.Port
|
||||
datacoord.Params.DataCoordCfg.Address = Params.GetAddress()
|
||||
|
||||
err := s.startGrpc()
|
||||
if err != nil {
|
||||
|
||||
@ -194,8 +194,8 @@ func (s *Server) init() error {
|
||||
Params.InitOnce(typeutil.DataNodeRole)
|
||||
|
||||
dn.Params.InitOnce()
|
||||
dn.Params.Port = Params.Port
|
||||
dn.Params.IP = Params.IP
|
||||
dn.Params.DataNodeCfg.Port = Params.Port
|
||||
dn.Params.DataNodeCfg.IP = Params.IP
|
||||
|
||||
closer := trace.InitTracing(fmt.Sprintf("data_node ip: %s, port: %d", Params.IP, Params.Port))
|
||||
s.closer = closer
|
||||
@ -210,7 +210,7 @@ func (s *Server) init() error {
|
||||
// --- RootCoord Client ---
|
||||
if s.newRootCoordClient != nil {
|
||||
log.Debug("Init root coord client ...")
|
||||
rootCoordClient, err := s.newRootCoordClient(dn.Params.MetaRootPath, dn.Params.EtcdEndpoints)
|
||||
rootCoordClient, err := s.newRootCoordClient(dn.Params.DataNodeCfg.MetaRootPath, dn.Params.DataNodeCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("DataNode newRootCoordClient failed", zap.Error(err))
|
||||
panic(err)
|
||||
@ -237,7 +237,7 @@ func (s *Server) init() error {
|
||||
// --- Data Server Client ---
|
||||
if s.newDataCoordClient != nil {
|
||||
log.Debug("DataNode Init data service client ...")
|
||||
dataCoordClient, err := s.newDataCoordClient(dn.Params.MetaRootPath, dn.Params.EtcdEndpoints)
|
||||
dataCoordClient, err := s.newDataCoordClient(dn.Params.DataNodeCfg.MetaRootPath, dn.Params.DataNodeCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("DataNode newDataCoordClient failed", zap.Error(err))
|
||||
panic(err)
|
||||
@ -261,7 +261,7 @@ func (s *Server) init() error {
|
||||
}
|
||||
}
|
||||
|
||||
s.datanode.SetNodeID(dn.Params.NodeID)
|
||||
s.datanode.SetNodeID(dn.Params.DataNodeCfg.NodeID)
|
||||
s.datanode.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||
|
||||
if err := s.datanode.Init(); err != nil {
|
||||
|
||||
@ -42,7 +42,7 @@ func TestIndexCoordClient(t *testing.T) {
|
||||
err = server.Run()
|
||||
assert.Nil(t, err)
|
||||
|
||||
icc, err := NewClient(ctx, indexcoord.Params.MetaRootPath, indexcoord.Params.EtcdEndpoints)
|
||||
icc, err := NewClient(ctx, indexcoord.Params.IndexCoordCfg.MetaRootPath, indexcoord.Params.IndexCoordCfg.EtcdEndpoints)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, icc)
|
||||
|
||||
|
||||
@ -79,14 +79,14 @@ func (s *Server) init() error {
|
||||
Params.InitOnce(typeutil.IndexCoordRole)
|
||||
|
||||
indexcoord.Params.InitOnce()
|
||||
indexcoord.Params.Address = Params.GetAddress()
|
||||
indexcoord.Params.Port = Params.Port
|
||||
indexcoord.Params.IndexCoordCfg.Address = Params.GetAddress()
|
||||
indexcoord.Params.IndexCoordCfg.Port = Params.Port
|
||||
|
||||
closer := trace.InitTracing("IndexCoord")
|
||||
s.closer = closer
|
||||
|
||||
s.loopWg.Add(1)
|
||||
go s.startGrpcLoop(indexcoord.Params.Port)
|
||||
go s.startGrpcLoop(indexcoord.Params.IndexCoordCfg.Port)
|
||||
// wait for grpc IndexCoord loop start
|
||||
if err := <-s.grpcErrChan; err != nil {
|
||||
log.Error("IndexCoord", zap.Any("init error", err))
|
||||
|
||||
@ -117,11 +117,11 @@ func (s *Server) init() error {
|
||||
Params.InitOnce(typeutil.IndexNodeRole)
|
||||
|
||||
indexnode.Params.InitOnce()
|
||||
indexnode.Params.Port = Params.Port
|
||||
indexnode.Params.IP = Params.IP
|
||||
indexnode.Params.Address = Params.GetAddress()
|
||||
indexnode.Params.IndexNodeCfg.Port = Params.Port
|
||||
indexnode.Params.IndexNodeCfg.IP = Params.IP
|
||||
indexnode.Params.IndexNodeCfg.Address = Params.GetAddress()
|
||||
|
||||
closer := trace.InitTracing(fmt.Sprintf("IndexNode-%d", indexnode.Params.NodeID))
|
||||
closer := trace.InitTracing(fmt.Sprintf("IndexNode-%d", indexnode.Params.IndexNodeCfg.NodeID))
|
||||
s.closer = closer
|
||||
|
||||
defer func() {
|
||||
|
||||
@ -156,10 +156,10 @@ func (s *Server) init() error {
|
||||
log.Debug("init params done ...")
|
||||
|
||||
// NetworkPort & IP don't matter here, NetworkAddress matters
|
||||
proxy.Params.NetworkPort = Params.Port
|
||||
proxy.Params.IP = Params.IP
|
||||
proxy.Params.ProxyCfg.NetworkPort = Params.Port
|
||||
proxy.Params.ProxyCfg.IP = Params.IP
|
||||
|
||||
proxy.Params.NetworkAddress = Params.GetAddress()
|
||||
proxy.Params.ProxyCfg.NetworkAddress = Params.GetAddress()
|
||||
|
||||
closer := trace.InitTracing(fmt.Sprintf("proxy ip: %s, port: %d", Params.IP, Params.Port))
|
||||
s.closer = closer
|
||||
@ -178,7 +178,7 @@ func (s *Server) init() error {
|
||||
}
|
||||
|
||||
if s.rootCoordClient == nil {
|
||||
s.rootCoordClient, err = rcc.NewClient(s.ctx, proxy.Params.MetaRootPath, proxy.Params.EtcdEndpoints)
|
||||
s.rootCoordClient, err = rcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("Proxy new rootCoordClient failed ", zap.Error(err))
|
||||
return err
|
||||
@ -198,7 +198,7 @@ func (s *Server) init() error {
|
||||
log.Debug("set rootcoord client ...")
|
||||
|
||||
if s.dataCoordClient == nil {
|
||||
s.dataCoordClient, err = dcc.NewClient(s.ctx, proxy.Params.MetaRootPath, proxy.Params.EtcdEndpoints)
|
||||
s.dataCoordClient, err = dcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("Proxy new dataCoordClient failed ", zap.Error(err))
|
||||
return err
|
||||
@ -214,7 +214,7 @@ func (s *Server) init() error {
|
||||
log.Debug("set data coordinator address ...")
|
||||
|
||||
if s.indexCoordClient == nil {
|
||||
s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.MetaRootPath, proxy.Params.EtcdEndpoints)
|
||||
s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("Proxy new indexCoordClient failed ", zap.Error(err))
|
||||
return err
|
||||
@ -230,7 +230,7 @@ func (s *Server) init() error {
|
||||
log.Debug("set index coordinator client ...")
|
||||
|
||||
if s.queryCooedClient == nil {
|
||||
s.queryCooedClient, err = qcc.NewClient(s.ctx, proxy.Params.MetaRootPath, proxy.Params.EtcdEndpoints)
|
||||
s.queryCooedClient, err = qcc.NewClient(s.ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
@ -32,7 +32,7 @@ func Test_NewClient(t *testing.T) {
|
||||
proxy.Params.InitOnce()
|
||||
|
||||
ctx := context.Background()
|
||||
client, err := NewClient(ctx, proxy.Params.MetaRootPath, proxy.Params.EtcdEndpoints)
|
||||
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, client)
|
||||
|
||||
|
||||
@ -105,8 +105,8 @@ func (s *Server) init() error {
|
||||
Params.InitOnce(typeutil.QueryCoordRole)
|
||||
|
||||
qc.Params.InitOnce()
|
||||
qc.Params.Address = Params.GetAddress()
|
||||
qc.Params.Port = Params.Port
|
||||
qc.Params.QueryCoordCfg.Address = Params.GetAddress()
|
||||
qc.Params.QueryCoordCfg.Port = Params.Port
|
||||
|
||||
closer := trace.InitTracing("querycoord")
|
||||
s.closer = closer
|
||||
@ -121,7 +121,7 @@ func (s *Server) init() error {
|
||||
|
||||
// --- Master Server Client ---
|
||||
if s.rootCoord == nil {
|
||||
s.rootCoord, err = rcc.NewClient(s.loopCtx, qc.Params.MetaRootPath, qc.Params.EtcdEndpoints)
|
||||
s.rootCoord, err = rcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err))
|
||||
panic(err)
|
||||
@ -152,7 +152,7 @@ func (s *Server) init() error {
|
||||
|
||||
// --- Data service client ---
|
||||
if s.dataCoord == nil {
|
||||
s.dataCoord, err = dcc.NewClient(s.loopCtx, qc.Params.MetaRootPath, qc.Params.EtcdEndpoints)
|
||||
s.dataCoord, err = dcc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err))
|
||||
panic(err)
|
||||
@ -180,7 +180,7 @@ func (s *Server) init() error {
|
||||
|
||||
// --- IndexCoord ---
|
||||
if s.indexCoord == nil {
|
||||
s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.MetaRootPath, qc.Params.EtcdEndpoints)
|
||||
s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.QueryCoordCfg.MetaRootPath, qc.Params.QueryCoordCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err))
|
||||
panic(err)
|
||||
|
||||
@ -85,8 +85,8 @@ func (s *Server) init() error {
|
||||
Params.InitOnce(typeutil.QueryNodeRole)
|
||||
|
||||
qn.Params.InitOnce()
|
||||
qn.Params.QueryNodeIP = Params.IP
|
||||
qn.Params.QueryNodePort = int64(Params.Port)
|
||||
qn.Params.QueryNodeCfg.QueryNodeIP = Params.IP
|
||||
qn.Params.QueryNodeCfg.QueryNodePort = int64(Params.Port)
|
||||
//qn.Params.QueryNodeID = Params.QueryNodeID
|
||||
|
||||
closer := trace.InitTracing(fmt.Sprintf("query_node ip: %s, port: %d", Params.IP, Params.Port))
|
||||
@ -103,7 +103,7 @@ func (s *Server) init() error {
|
||||
|
||||
// --- RootCoord Client ---
|
||||
if s.rootCoord == nil {
|
||||
s.rootCoord, err = rcc.NewClient(s.ctx, qn.Params.MetaRootPath, qn.Params.EtcdEndpoints)
|
||||
s.rootCoord, err = rcc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, qn.Params.QueryNodeCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("QueryNode new RootCoordClient failed", zap.Error(err))
|
||||
panic(err)
|
||||
@ -133,7 +133,7 @@ func (s *Server) init() error {
|
||||
|
||||
// --- IndexCoord ---
|
||||
if s.indexCoord == nil {
|
||||
s.indexCoord, err = icc.NewClient(s.ctx, qn.Params.MetaRootPath, qn.Params.EtcdEndpoints)
|
||||
s.indexCoord, err = icc.NewClient(s.ctx, qn.Params.QueryNodeCfg.MetaRootPath, qn.Params.QueryNodeCfg.EtcdEndpoints)
|
||||
if err != nil {
|
||||
log.Debug("QueryNode new IndexCoordClient failed", zap.Error(err))
|
||||
panic(err)
|
||||
@ -203,7 +203,7 @@ func (s *Server) startGrpcLoop(grpcPort int) {
|
||||
addr := ":" + strconv.Itoa(grpcPort)
|
||||
lis, err = net.Listen("tcp", addr)
|
||||
if err == nil {
|
||||
qn.Params.QueryNodePort = int64(lis.Addr().(*net.TCPAddr).Port)
|
||||
qn.Params.QueryNodeCfg.QueryNodePort = int64(lis.Addr().(*net.TCPAddr).Port)
|
||||
} else {
|
||||
// set port=0 to get next available port
|
||||
grpcPort = 0
|
||||
|
||||
@ -32,7 +32,7 @@ func Test_NewClient(t *testing.T) {
|
||||
proxy.Params.InitOnce()
|
||||
|
||||
ctx := context.Background()
|
||||
client, err := NewClient(ctx, proxy.Params.MetaRootPath, proxy.Params.EtcdEndpoints)
|
||||
client, err := NewClient(ctx, proxy.Params.ProxyCfg.MetaRootPath, proxy.Params.ProxyCfg.EtcdEndpoints)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, client)
|
||||
|
||||
|
||||
@ -147,8 +147,8 @@ func (s *Server) init() error {
|
||||
Params.InitOnce(typeutil.RootCoordRole)
|
||||
|
||||
rootcoord.Params.InitOnce()
|
||||
rootcoord.Params.Address = Params.GetAddress()
|
||||
rootcoord.Params.Port = Params.Port
|
||||
rootcoord.Params.RootCoordCfg.Address = Params.GetAddress()
|
||||
rootcoord.Params.RootCoordCfg.Port = Params.Port
|
||||
log.Debug("grpc init done ...")
|
||||
|
||||
closer := trace.InitTracing("root_coord")
|
||||
@ -181,7 +181,7 @@ func (s *Server) init() error {
|
||||
|
||||
if s.newDataCoordClient != nil {
|
||||
log.Debug("RootCoord start to create DataCoord client")
|
||||
dataCoord := s.newDataCoordClient(rootcoord.Params.MetaRootPath, rootcoord.Params.EtcdEndpoints)
|
||||
dataCoord := s.newDataCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
||||
if err := s.rootCoord.SetDataCoord(s.ctx, dataCoord); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
@ -189,7 +189,7 @@ func (s *Server) init() error {
|
||||
}
|
||||
if s.newIndexCoordClient != nil {
|
||||
log.Debug("RootCoord start to create IndexCoord client")
|
||||
indexCoord := s.newIndexCoordClient(rootcoord.Params.MetaRootPath, rootcoord.Params.EtcdEndpoints)
|
||||
indexCoord := s.newIndexCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
||||
if err := s.rootCoord.SetIndexCoord(indexCoord); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
@ -197,7 +197,7 @@ func (s *Server) init() error {
|
||||
}
|
||||
if s.newQueryCoordClient != nil {
|
||||
log.Debug("RootCoord start to create QueryCoord client")
|
||||
queryCoord := s.newQueryCoordClient(rootcoord.Params.MetaRootPath, rootcoord.Params.EtcdEndpoints)
|
||||
queryCoord := s.newQueryCoordClient(rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
||||
if err := s.rootCoord.SetQueryCoord(queryCoord); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
@ -79,15 +79,15 @@ func TestGrpcService(t *testing.T) {
|
||||
assert.Nil(t, err)
|
||||
|
||||
rootcoord.Params.Init()
|
||||
rootcoord.Params.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
|
||||
rootcoord.Params.KvRootPath = fmt.Sprintf("/%d/test/kv", randVal)
|
||||
rootcoord.Params.MsgChannelSubName = fmt.Sprintf("msgChannel%d", randVal)
|
||||
rootcoord.Params.TimeTickChannel = fmt.Sprintf("timeTick%d", randVal)
|
||||
rootcoord.Params.StatisticsChannel = fmt.Sprintf("stateChannel%d", randVal)
|
||||
rootcoord.Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
|
||||
rootcoord.Params.RootCoordCfg.KvRootPath = fmt.Sprintf("/%d/test/kv", randVal)
|
||||
rootcoord.Params.RootCoordCfg.MsgChannelSubName = fmt.Sprintf("msgChannel%d", randVal)
|
||||
rootcoord.Params.RootCoordCfg.TimeTickChannel = fmt.Sprintf("timeTick%d", randVal)
|
||||
rootcoord.Params.RootCoordCfg.StatisticsChannel = fmt.Sprintf("stateChannel%d", randVal)
|
||||
|
||||
rootcoord.Params.MaxPartitionNum = 64
|
||||
rootcoord.Params.DefaultPartitionName = "_default"
|
||||
rootcoord.Params.DefaultIndexName = "_default"
|
||||
rootcoord.Params.RootCoordCfg.MaxPartitionNum = 64
|
||||
rootcoord.Params.RootCoordCfg.DefaultPartitionName = "_default"
|
||||
rootcoord.Params.RootCoordCfg.DefaultIndexName = "_default"
|
||||
|
||||
t.Logf("service port = %d", Params.Port)
|
||||
|
||||
@ -98,9 +98,9 @@ func TestGrpcService(t *testing.T) {
|
||||
assert.Nil(t, err)
|
||||
svr.rootCoord.UpdateStateCode(internalpb.StateCode_Initializing)
|
||||
|
||||
etcdCli, err := initEtcd(rootcoord.Params.EtcdEndpoints)
|
||||
etcdCli, err := initEtcd(rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
||||
assert.Nil(t, err)
|
||||
sessKey := path.Join(rootcoord.Params.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||
sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
||||
assert.Nil(t, err)
|
||||
|
||||
@ -113,7 +113,7 @@ func TestGrpcService(t *testing.T) {
|
||||
_, err = etcdCli.Put(ctx, path.Join(sessKey, typeutil.ProxyRole+"-100"), string(pnb))
|
||||
assert.Nil(t, err)
|
||||
|
||||
rootcoord.Params.Address = Params.GetAddress()
|
||||
rootcoord.Params.RootCoordCfg.Address = Params.GetAddress()
|
||||
|
||||
err = core.Init()
|
||||
assert.Nil(t, err)
|
||||
@ -156,7 +156,7 @@ func TestGrpcService(t *testing.T) {
|
||||
return []string{"file1", "file2", "file3"}, nil
|
||||
}
|
||||
core.CallGetNumRowsService = func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (int64, error) {
|
||||
return rootcoord.Params.MinSegmentSizeToEnableIndex, nil
|
||||
return rootcoord.Params.RootCoordCfg.MinSegmentSizeToEnableIndex, nil
|
||||
}
|
||||
core.CallWatchChannels = func(ctx context.Context, collectionID int64, channelNames []string) error {
|
||||
return nil
|
||||
@ -214,7 +214,7 @@ func TestGrpcService(t *testing.T) {
|
||||
|
||||
svr.rootCoord.UpdateStateCode(internalpb.StateCode_Healthy)
|
||||
|
||||
cli, err := rcc.NewClient(context.Background(), rootcoord.Params.MetaRootPath, rootcoord.Params.EtcdEndpoints)
|
||||
cli, err := rcc.NewClient(context.Background(), rootcoord.Params.RootCoordCfg.MetaRootPath, rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
||||
assert.Nil(t, err)
|
||||
|
||||
err = cli.Init()
|
||||
@ -637,7 +637,7 @@ func TestGrpcService(t *testing.T) {
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
|
||||
assert.Equal(t, 1, len(rsp.IndexDescriptions))
|
||||
assert.Equal(t, rootcoord.Params.DefaultIndexName, rsp.IndexDescriptions[0].IndexName)
|
||||
assert.Equal(t, rootcoord.Params.RootCoordCfg.DefaultIndexName, rsp.IndexDescriptions[0].IndexName)
|
||||
})
|
||||
|
||||
t.Run("flush segment", func(t *testing.T) {
|
||||
@ -685,7 +685,7 @@ func TestGrpcService(t *testing.T) {
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, rsp.Status.ErrorCode)
|
||||
assert.Equal(t, 1, len(rsp.IndexDescriptions))
|
||||
assert.Equal(t, rootcoord.Params.DefaultIndexName, rsp.IndexDescriptions[0].IndexName)
|
||||
assert.Equal(t, rootcoord.Params.RootCoordCfg.DefaultIndexName, rsp.IndexDescriptions[0].IndexName)
|
||||
|
||||
})
|
||||
|
||||
@ -700,9 +700,9 @@ func TestGrpcService(t *testing.T) {
|
||||
DbName: dbName,
|
||||
CollectionName: collName,
|
||||
FieldName: fieldName,
|
||||
IndexName: rootcoord.Params.DefaultIndexName,
|
||||
IndexName: rootcoord.Params.RootCoordCfg.DefaultIndexName,
|
||||
}
|
||||
_, idx, err := core.MetaTable.GetIndexByName(collName, rootcoord.Params.DefaultIndexName)
|
||||
_, idx, err := core.MetaTable.GetIndexByName(collName, rootcoord.Params.RootCoordCfg.DefaultIndexName)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(idx), 1)
|
||||
rsp, err := cli.DropIndex(ctx, req)
|
||||
@ -735,7 +735,7 @@ func TestGrpcService(t *testing.T) {
|
||||
assert.Equal(t, 1, len(collMeta.PartitionIDs))
|
||||
partName, err := core.MetaTable.GetPartitionNameByID(collMeta.ID, collMeta.PartitionIDs[0], 0)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, rootcoord.Params.DefaultPartitionName, partName)
|
||||
assert.Equal(t, rootcoord.Params.RootCoordCfg.DefaultPartitionName, partName)
|
||||
assert.Equal(t, 2, len(collectionMetaCache))
|
||||
})
|
||||
|
||||
@ -910,11 +910,11 @@ func TestRun(t *testing.T) {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
randVal := rand.Int()
|
||||
rootcoord.Params.Init()
|
||||
rootcoord.Params.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
|
||||
rootcoord.Params.RootCoordCfg.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
|
||||
|
||||
etcdCli, err := initEtcd(rootcoord.Params.EtcdEndpoints)
|
||||
etcdCli, err := initEtcd(rootcoord.Params.RootCoordCfg.EtcdEndpoints)
|
||||
assert.Nil(t, err)
|
||||
sessKey := path.Join(rootcoord.Params.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||
sessKey := path.Join(rootcoord.Params.RootCoordCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
|
||||
assert.Nil(t, err)
|
||||
err = svr.Run()
|
||||
|
||||
@ -27,9 +27,6 @@ import (
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
@ -44,11 +41,14 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/tso"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// make sure IndexCoord implements types.IndexCoord
|
||||
@ -58,6 +58,8 @@ const (
|
||||
indexSizeFactor = 6
|
||||
)
|
||||
|
||||
var Params paramtable.GlobalParamTable
|
||||
|
||||
// IndexCoord is a component responsible for scheduling index construction tasks and maintaining index status.
|
||||
// IndexCoord accepts requests from rootcoord to build indexes, delete indexes, and query index information.
|
||||
// IndexCoord is responsible for assigning IndexBuildID to the request to build the index, and forwarding the
|
||||
@ -132,12 +134,12 @@ func (i *IndexCoord) Register() error {
|
||||
}
|
||||
|
||||
func (i *IndexCoord) initSession() error {
|
||||
i.session = sessionutil.NewSession(i.loopCtx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
i.session = sessionutil.NewSession(i.loopCtx, Params.IndexCoordCfg.MetaRootPath, Params.IndexCoordCfg.EtcdEndpoints)
|
||||
if i.session == nil {
|
||||
return errors.New("failed to initialize session")
|
||||
}
|
||||
i.session.Init(typeutil.IndexCoordRole, Params.Address, true)
|
||||
Params.SetLogger(i.session.ServerID)
|
||||
i.session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true)
|
||||
Params.BaseParams.SetLogger(i.session.ServerID)
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -157,7 +159,7 @@ func (i *IndexCoord) Init() error {
|
||||
}
|
||||
|
||||
connectEtcdFn := func() error {
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -203,8 +205,8 @@ func (i *IndexCoord) Init() error {
|
||||
}
|
||||
|
||||
//init idAllocator
|
||||
kvRootPath := Params.KvRootPath
|
||||
etcdKV, err := tsoutil.NewTSOKVBase(Params.EtcdEndpoints, kvRootPath, "index_gid")
|
||||
kvRootPath := Params.IndexCoordCfg.KvRootPath
|
||||
etcdKV, err := tsoutil.NewTSOKVBase(Params.IndexCoordCfg.EtcdEndpoints, kvRootPath, "index_gid")
|
||||
if err != nil {
|
||||
log.Error("IndexCoord TSOKVBase initialize failed", zap.Error(err))
|
||||
initErr = err
|
||||
@ -218,11 +220,11 @@ func (i *IndexCoord) Init() error {
|
||||
}
|
||||
|
||||
option := &miniokv.Option{
|
||||
Address: Params.MinIOAddress,
|
||||
AccessKeyID: Params.MinIOAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinIOSecretAccessKey,
|
||||
UseSSL: Params.MinIOUseSSL,
|
||||
BucketName: Params.MinioBucketName,
|
||||
Address: Params.IndexCoordCfg.MinIOAddress,
|
||||
AccessKeyID: Params.IndexCoordCfg.MinIOAccessKeyID,
|
||||
SecretAccessKeyID: Params.IndexCoordCfg.MinIOSecretAccessKey,
|
||||
UseSSL: Params.IndexCoordCfg.MinIOUseSSL,
|
||||
BucketName: Params.IndexCoordCfg.MinioBucketName,
|
||||
CreateBucket: true,
|
||||
}
|
||||
|
||||
@ -278,8 +280,8 @@ func (i *IndexCoord) Start() error {
|
||||
cb()
|
||||
}
|
||||
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
Params.IndexCoordCfg.CreatedTime = time.Now()
|
||||
Params.IndexCoordCfg.UpdatedTime = time.Now()
|
||||
|
||||
i.UpdateStateCode(internalpb.StateCode_Healthy)
|
||||
log.Debug("IndexCoord start successfully", zap.Any("State", i.stateCode.Load()))
|
||||
@ -698,7 +700,7 @@ func (i *IndexCoord) recycleUnusedIndexFiles() {
|
||||
log.Debug("IndexCoord recycleUnusedIndexFiles", zap.Int("Need recycle tasks num", len(metas)))
|
||||
for _, meta := range metas {
|
||||
if meta.indexMeta.MarkDeleted {
|
||||
unusedIndexFilePathPrefix := Params.IndexStorageRootPath + "/" + strconv.Itoa(int(meta.indexMeta.IndexBuildID))
|
||||
unusedIndexFilePathPrefix := Params.IndexCoordCfg.IndexStorageRootPath + "/" + strconv.Itoa(int(meta.indexMeta.IndexBuildID))
|
||||
log.Debug("IndexCoord recycleUnusedIndexFiles",
|
||||
zap.Int64("Recycle the index files for deleted index with indexBuildID", meta.indexMeta.IndexBuildID))
|
||||
if err := i.kv.RemoveWithPrefix(unusedIndexFilePathPrefix); err != nil {
|
||||
@ -712,7 +714,7 @@ func (i *IndexCoord) recycleUnusedIndexFiles() {
|
||||
log.Debug("IndexCoord recycleUnusedIndexFiles",
|
||||
zap.Int64("Recycle the low version index files of the index with indexBuildID", meta.indexMeta.IndexBuildID))
|
||||
for j := 1; j < int(meta.indexMeta.Version); j++ {
|
||||
unusedIndexFilePathPrefix := Params.IndexStorageRootPath + "/" + strconv.Itoa(int(meta.indexMeta.IndexBuildID)) + "/" + strconv.Itoa(j)
|
||||
unusedIndexFilePathPrefix := Params.IndexCoordCfg.IndexStorageRootPath + "/" + strconv.Itoa(int(meta.indexMeta.IndexBuildID)) + "/" + strconv.Itoa(j)
|
||||
if err := i.kv.RemoveWithPrefix(unusedIndexFilePathPrefix); err != nil {
|
||||
log.Error("IndexCoord recycleUnusedIndexFiles Remove index files failed",
|
||||
zap.Bool("MarkDeleted", false), zap.Error(err))
|
||||
|
||||
@ -67,10 +67,10 @@ func (icm *Mock) Register() error {
|
||||
if icm.Failure {
|
||||
return errors.New("IndexCoordinate register failed")
|
||||
}
|
||||
icm.etcdKV, _ = etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
icm.etcdKV, _ = etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath)
|
||||
err := icm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexCoordRole)
|
||||
session := sessionutil.NewSession(context.Background(), Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
session.Init(typeutil.IndexCoordRole, Params.Address, true)
|
||||
session := sessionutil.NewSession(context.Background(), Params.IndexCoordCfg.MetaRootPath, Params.IndexCoordCfg.EtcdEndpoints)
|
||||
session.Init(typeutil.IndexCoordRole, Params.IndexCoordCfg.Address, true)
|
||||
session.Register()
|
||||
return err
|
||||
}
|
||||
|
||||
@ -31,7 +31,7 @@ import (
|
||||
|
||||
func TestMetaTable(t *testing.T) {
|
||||
Params.Init()
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
|
||||
req := &indexpb.BuildIndexRequest{
|
||||
@ -311,7 +311,7 @@ func TestMetaTable(t *testing.T) {
|
||||
|
||||
func TestMetaTable_Error(t *testing.T) {
|
||||
Params.Init()
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexCoordCfg.EtcdEndpoints, Params.IndexCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
|
||||
t.Run("reloadFromKV error", func(t *testing.T) {
|
||||
|
||||
@ -50,13 +50,13 @@ func getSystemInfoMetrics(
|
||||
DiskUsage: metricsinfo.GetDiskUsage(),
|
||||
},
|
||||
SystemInfo: metricsinfo.DeployMetrics{},
|
||||
CreatedTime: Params.CreatedTime.String(),
|
||||
UpdatedTime: Params.UpdatedTime.String(),
|
||||
CreatedTime: Params.IndexCoordCfg.CreatedTime.String(),
|
||||
UpdatedTime: Params.IndexCoordCfg.UpdatedTime.String(),
|
||||
Type: typeutil.IndexCoordRole,
|
||||
ID: coord.session.ServerID,
|
||||
},
|
||||
SystemConfigurations: metricsinfo.IndexCoordConfiguration{
|
||||
MinioBucketName: Params.MinioBucketName,
|
||||
MinioBucketName: Params.IndexCoordCfg.MinioBucketName,
|
||||
},
|
||||
},
|
||||
ConnectedNodes: make([]metricsinfo.IndexNodeInfos, 0),
|
||||
|
||||
@ -1,171 +0,0 @@
|
||||
// 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 indexcoord
|
||||
|
||||
import (
|
||||
"path"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
)
|
||||
|
||||
// ParamTable is used to record configuration items.
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
|
||||
Address string
|
||||
Port int
|
||||
|
||||
EtcdEndpoints []string
|
||||
KvRootPath string
|
||||
MetaRootPath string
|
||||
IndexStorageRootPath string
|
||||
|
||||
MinIOAddress string
|
||||
MinIOAccessKeyID string
|
||||
MinIOSecretAccessKey string
|
||||
MinIOUseSSL bool
|
||||
MinioBucketName string
|
||||
|
||||
CreatedTime time.Time
|
||||
UpdatedTime time.Time
|
||||
}
|
||||
|
||||
// Params is an alias for ParamTable.
|
||||
var Params ParamTable
|
||||
var once sync.Once
|
||||
|
||||
// Init is used to initialize configuration items.
|
||||
func (pt *ParamTable) Init() {
|
||||
pt.BaseTable.Init()
|
||||
|
||||
pt.initEtcdEndpoints()
|
||||
pt.initMetaRootPath()
|
||||
pt.initKvRootPath()
|
||||
pt.initMinIOAddress()
|
||||
pt.initMinIOAccessKeyID()
|
||||
pt.initMinIOSecretAccessKey()
|
||||
pt.initMinIOUseSSL()
|
||||
pt.initMinioBucketName()
|
||||
pt.initIndexStorageRootPath()
|
||||
pt.initRoleName()
|
||||
}
|
||||
|
||||
// InitOnce is used to initialize configuration items, and it will only be called once.
|
||||
func (pt *ParamTable) InitOnce() {
|
||||
once.Do(func() {
|
||||
pt.Init()
|
||||
})
|
||||
}
|
||||
|
||||
// initEtcdEndpoints initializes the etcd address of configuration items.
|
||||
func (pt *ParamTable) initEtcdEndpoints() {
|
||||
endpoints, err := pt.Load("_EtcdEndpoints")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.EtcdEndpoints = strings.Split(endpoints, ",")
|
||||
}
|
||||
|
||||
// initMetaRootPath initializes the root path of etcd.
|
||||
func (pt *ParamTable) initMetaRootPath() {
|
||||
rootPath, err := pt.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := pt.Load("etcd.metaSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MetaRootPath = rootPath + "/" + subPath
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initKvRootPath() {
|
||||
rootPath, err := pt.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := pt.Load("etcd.kvSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.KvRootPath = rootPath + "/" + subPath
|
||||
}
|
||||
|
||||
// initMinIOAddress initializes init the minio address of configuration items.
|
||||
func (pt *ParamTable) initMinIOAddress() {
|
||||
ret, err := pt.Load("_MinioAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinIOAddress = ret
|
||||
}
|
||||
|
||||
// initMinIOAccessKeyID initializes the minio access key of configuration items.
|
||||
func (pt *ParamTable) initMinIOAccessKeyID() {
|
||||
ret, err := pt.Load("minio.accessKeyID")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinIOAccessKeyID = ret
|
||||
}
|
||||
|
||||
// initMinIOSecretAccessKey initializes the minio secret access key.
|
||||
func (pt *ParamTable) initMinIOSecretAccessKey() {
|
||||
ret, err := pt.Load("minio.secretAccessKey")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinIOSecretAccessKey = ret
|
||||
}
|
||||
|
||||
// initMinIOUseSSL initializes the minio use SSL of configuration items.
|
||||
func (pt *ParamTable) initMinIOUseSSL() {
|
||||
ret, err := pt.Load("minio.useSSL")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinIOUseSSL, err = strconv.ParseBool(ret)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
// initMinioBucketName initializes the minio bucket name of configuration items.
|
||||
func (pt *ParamTable) initMinioBucketName() {
|
||||
bucketName, err := pt.Load("minio.bucketName")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinioBucketName = bucketName
|
||||
}
|
||||
|
||||
// initIndexStorageRootPath initializes the root path of index files.
|
||||
func (pt *ParamTable) initIndexStorageRootPath() {
|
||||
rootPath, err := pt.Load("minio.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.IndexStorageRootPath = path.Join(rootPath, "index_files")
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initRoleName() {
|
||||
pt.RoleName = "indexcoord"
|
||||
}
|
||||
@ -1,177 +0,0 @@
|
||||
// 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 indexcoord
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestParamTable(t *testing.T) {
|
||||
Params.Init()
|
||||
|
||||
t.Run("Address", func(t *testing.T) {
|
||||
t.Logf("Address: %v", Params.Address)
|
||||
})
|
||||
|
||||
t.Run("Port", func(t *testing.T) {
|
||||
t.Logf("Port: %v", Params.Port)
|
||||
})
|
||||
|
||||
t.Run("EtcdEndpoints", func(t *testing.T) {
|
||||
t.Logf("EtcdEndpoints: %v", Params.EtcdEndpoints)
|
||||
})
|
||||
|
||||
t.Run("KvRootPath", func(t *testing.T) {
|
||||
t.Logf("KvRootPath: %v", Params.KvRootPath)
|
||||
})
|
||||
|
||||
t.Run("MetaRootPath", func(t *testing.T) {
|
||||
t.Logf("MetaRootPath: %v", Params.MetaRootPath)
|
||||
})
|
||||
|
||||
t.Run("MinIOAddress", func(t *testing.T) {
|
||||
t.Logf("MinIOAddress: %v", Params.MinIOAddress)
|
||||
})
|
||||
|
||||
t.Run("MinIOAccessKeyID", func(t *testing.T) {
|
||||
t.Logf("MinIOAccessKeyID: %v", Params.MinIOAccessKeyID)
|
||||
})
|
||||
|
||||
t.Run("MinIOSecretAccessKey", func(t *testing.T) {
|
||||
t.Logf("MinIOSecretAccessKey: %v", Params.MinIOSecretAccessKey)
|
||||
})
|
||||
|
||||
t.Run("MinIOUseSSL", func(t *testing.T) {
|
||||
t.Logf("MinIOUseSSL: %v", Params.MinIOUseSSL)
|
||||
})
|
||||
|
||||
t.Run("MinioBucketName", func(t *testing.T) {
|
||||
t.Logf("MinioBucketName: %v", Params.MinioBucketName)
|
||||
})
|
||||
|
||||
t.Run("CreatedTime", func(t *testing.T) {
|
||||
Params.CreatedTime = time.Now()
|
||||
t.Logf("CreatedTime: %v", Params.CreatedTime)
|
||||
})
|
||||
|
||||
t.Run("UpdatedTime", func(t *testing.T) {
|
||||
Params.UpdatedTime = time.Now()
|
||||
t.Logf("UpdatedTime: %v", Params.UpdatedTime)
|
||||
})
|
||||
|
||||
t.Run("initIndexStorageRootPath", func(t *testing.T) {
|
||||
t.Logf("IndexStorageRootPath: %v", Params.IndexStorageRootPath)
|
||||
})
|
||||
}
|
||||
|
||||
//TODO: Params Load should be return error when key does not exist.
|
||||
//func shouldPanic(t *testing.T, name string, f func()) {
|
||||
// defer func() {
|
||||
// if r := recover(); r != nil {
|
||||
// t.Logf("%v recover: %v", name, r)
|
||||
// }
|
||||
// }()
|
||||
// f()
|
||||
//}
|
||||
//
|
||||
//func TestParamTable_Panic(t *testing.T) {
|
||||
// Params.Init()
|
||||
//
|
||||
//
|
||||
// t.Run("initEtcdEndpoints", func(t *testing.T) {
|
||||
// err := Params.Remove("_EtcdEndpoints")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initEtcdEndpoints", func() {
|
||||
// Params.initEtcdEndpoints()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMetaRootPath", func(t *testing.T) {
|
||||
// err := Params.Remove("etcd.rootPath")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMetaRootPath", func() {
|
||||
// Params.initMetaRootPath()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initKvRootPath", func(t *testing.T) {
|
||||
// err := Params.Remove("etcd.rootPath")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initKvRootPath", func() {
|
||||
// Params.initKvRootPath()
|
||||
// })
|
||||
//
|
||||
// err = Params.Save("etcd.rootPath", "test")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// err = Params.Remove("etcd.kvSubPath")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initKvRootPath", func() {
|
||||
// Params.initKvRootPath()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinIOAddress", func(t *testing.T) {
|
||||
// err := Params.Remove("_MinioAddress")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOAddress", func() {
|
||||
// Params.initMinIOAddress()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinIOAccessKeyID", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.accessKeyID")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOAccessKeyID", func() {
|
||||
// Params.initMinIOAccessKeyID()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinIOSecretAccessKey", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.secretAccessKey")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOSecretAccessKey", func() {
|
||||
// Params.initMinIOSecretAccessKey()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinIOUseSSL", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.useSSL")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOUseSSL", func() {
|
||||
// Params.initMinIOUseSSL()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinioBucketName", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.bucketName")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinioBucketName", func() {
|
||||
// Params.initMinioBucketName()
|
||||
// })
|
||||
// })
|
||||
//}
|
||||
@ -40,12 +40,6 @@ import (
|
||||
"unsafe"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
miniokv "github.com/milvus-io/milvus/internal/kv/minio"
|
||||
@ -54,10 +48,14 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// UniqueID is an alias of int64, is used as a unique identifier for the request.
|
||||
@ -66,6 +64,8 @@ type UniqueID = typeutil.UniqueID
|
||||
// make sure IndexNode implements types.IndexNode
|
||||
var _ types.IndexNode = (*IndexNode)(nil)
|
||||
|
||||
var Params paramtable.GlobalParamTable
|
||||
|
||||
// IndexNode is a component that executes the task of building indexes.
|
||||
type IndexNode struct {
|
||||
stateCode atomic.Value
|
||||
@ -131,21 +131,21 @@ func (i *IndexNode) initKnowhere() {
|
||||
C.IndexBuilderInit()
|
||||
|
||||
// override index builder SIMD type
|
||||
cSimdType := C.CString(Params.SimdType)
|
||||
cSimdType := C.CString(Params.IndexNodeCfg.SimdType)
|
||||
cRealSimdType := C.IndexBuilderSetSimdType(cSimdType)
|
||||
Params.SimdType = C.GoString(cRealSimdType)
|
||||
Params.IndexNodeCfg.SimdType = C.GoString(cRealSimdType)
|
||||
C.free(unsafe.Pointer(cRealSimdType))
|
||||
C.free(unsafe.Pointer(cSimdType))
|
||||
}
|
||||
|
||||
func (i *IndexNode) initSession() error {
|
||||
i.session = sessionutil.NewSession(i.loopCtx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
i.session = sessionutil.NewSession(i.loopCtx, Params.IndexNodeCfg.MetaRootPath, Params.IndexNodeCfg.EtcdEndpoints)
|
||||
if i.session == nil {
|
||||
return errors.New("failed to initialize session")
|
||||
}
|
||||
i.session.Init(typeutil.IndexNodeRole, Params.IP+":"+strconv.Itoa(Params.Port), false)
|
||||
Params.NodeID = i.session.ServerID
|
||||
Params.SetLogger(Params.NodeID)
|
||||
i.session.Init(typeutil.IndexNodeRole, Params.IndexNodeCfg.IP+":"+strconv.Itoa(Params.IndexNodeCfg.Port), false)
|
||||
Params.IndexNodeCfg.NodeID = i.session.ServerID
|
||||
Params.BaseParams.SetLogger(Params.IndexNodeCfg.NodeID)
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -166,7 +166,7 @@ func (i *IndexNode) Init() error {
|
||||
log.Debug("IndexNode init session successful", zap.Int64("serverID", i.session.ServerID))
|
||||
|
||||
connectEtcdFn := func() error {
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.IndexNodeCfg.EtcdEndpoints, Params.IndexNodeCfg.MetaRootPath)
|
||||
i.etcdKV = etcdKV
|
||||
return err
|
||||
}
|
||||
@ -179,11 +179,11 @@ func (i *IndexNode) Init() error {
|
||||
log.Debug("IndexNode connected to etcd successfully")
|
||||
|
||||
option := &miniokv.Option{
|
||||
Address: Params.MinIOAddress,
|
||||
AccessKeyID: Params.MinIOAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinIOSecretAccessKey,
|
||||
UseSSL: Params.MinIOUseSSL,
|
||||
BucketName: Params.MinioBucketName,
|
||||
Address: Params.IndexNodeCfg.MinIOAddress,
|
||||
AccessKeyID: Params.IndexNodeCfg.MinIOAccessKeyID,
|
||||
SecretAccessKeyID: Params.IndexNodeCfg.MinIOSecretAccessKey,
|
||||
UseSSL: Params.IndexNodeCfg.MinIOUseSSL,
|
||||
BucketName: Params.IndexNodeCfg.MinioBucketName,
|
||||
CreateBucket: true,
|
||||
}
|
||||
kv, err := miniokv.NewMinIOKV(i.loopCtx, option)
|
||||
@ -212,8 +212,8 @@ func (i *IndexNode) Start() error {
|
||||
i.once.Do(func() {
|
||||
startErr = i.sched.Start()
|
||||
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
Params.IndexNodeCfg.CreatedTime = time.Now()
|
||||
Params.IndexNodeCfg.UpdatedTime = time.Now()
|
||||
|
||||
i.UpdateStateCode(internalpb.StateCode_Healthy)
|
||||
log.Debug("IndexNode", zap.Any("State", i.stateCode.Load()))
|
||||
@ -286,7 +286,7 @@ func (i *IndexNode) CreateIndex(ctx context.Context, request *indexpb.CreateInde
|
||||
req: request,
|
||||
kv: i.kv,
|
||||
etcdKV: i.etcdKV,
|
||||
nodeID: Params.NodeID,
|
||||
nodeID: Params.IndexNodeCfg.NodeID,
|
||||
serializedSize: 0,
|
||||
}
|
||||
|
||||
@ -360,19 +360,19 @@ func (i *IndexNode) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringR
|
||||
// TODO(dragondriver): cache the Metrics and set a retention to the cache
|
||||
func (i *IndexNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
||||
log.Debug("IndexNode.GetMetrics",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.IndexNodeCfg.NodeID),
|
||||
zap.String("req", req.Request))
|
||||
|
||||
if !i.isHealthy() {
|
||||
log.Warn("IndexNode.GetMetrics failed",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.IndexNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.Error(errIndexNodeIsUnhealthy(Params.NodeID)))
|
||||
zap.Error(errIndexNodeIsUnhealthy(Params.IndexNodeCfg.NodeID)))
|
||||
|
||||
return &milvuspb.GetMetricsResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: msgIndexNodeIsUnhealthy(Params.NodeID),
|
||||
Reason: msgIndexNodeIsUnhealthy(Params.IndexNodeCfg.NodeID),
|
||||
},
|
||||
Response: "",
|
||||
}, nil
|
||||
@ -381,7 +381,7 @@ func (i *IndexNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequ
|
||||
metricType, err := metricsinfo.ParseMetricType(req.Request)
|
||||
if err != nil {
|
||||
log.Warn("IndexNode.GetMetrics failed to parse metric type",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.IndexNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.Error(err))
|
||||
|
||||
@ -398,7 +398,7 @@ func (i *IndexNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequ
|
||||
metrics, err := getSystemInfoMetrics(ctx, req, i)
|
||||
|
||||
log.Debug("IndexNode.GetMetrics",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.IndexNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType),
|
||||
zap.Any("metrics", metrics), // TODO(dragondriver): necessary? may be very large
|
||||
@ -408,7 +408,7 @@ func (i *IndexNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequ
|
||||
}
|
||||
|
||||
log.Warn("IndexNode.GetMetrics failed, request metric type is not implemented yet",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.IndexNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType))
|
||||
|
||||
|
||||
@ -182,11 +182,11 @@ func (inm *Mock) Register() error {
|
||||
return errors.New("IndexNode register failed")
|
||||
}
|
||||
Params.Init()
|
||||
inm.etcdKV, _ = etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
inm.etcdKV, _ = etcdkv.NewEtcdKV(Params.IndexNodeCfg.EtcdEndpoints, Params.IndexNodeCfg.MetaRootPath)
|
||||
if err := inm.etcdKV.RemoveWithPrefix("session/" + typeutil.IndexNodeRole); err != nil {
|
||||
return err
|
||||
}
|
||||
session := sessionutil.NewSession(context.Background(), Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
session := sessionutil.NewSession(context.Background(), Params.IndexNodeCfg.MetaRootPath, Params.IndexNodeCfg.EtcdEndpoints)
|
||||
session.Init(typeutil.IndexNodeRole, "localhost:21121", false)
|
||||
session.Register()
|
||||
return nil
|
||||
@ -296,7 +296,7 @@ func (inm *Mock) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
|
||||
metrics, err := getMockSystemInfoMetrics(ctx, req, inm)
|
||||
|
||||
log.Debug("IndexNode.GetMetrics",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.IndexNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType),
|
||||
zap.Any("metrics", metrics), // TODO(dragondriver): necessary? may be very large
|
||||
@ -306,7 +306,7 @@ func (inm *Mock) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
|
||||
}
|
||||
|
||||
log.Warn("IndexNode.GetMetrics failed, request metric type is not implemented yet",
|
||||
zap.Int64("node_id", Params.NodeID),
|
||||
zap.Int64("node_id", Params.IndexNodeCfg.NodeID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType))
|
||||
|
||||
@ -327,7 +327,7 @@ func getMockSystemInfoMetrics(
|
||||
// TODO(dragondriver): add more metrics
|
||||
nodeInfos := metricsinfo.IndexNodeInfos{
|
||||
BaseComponentInfos: metricsinfo.BaseComponentInfos{
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.NodeID),
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.IndexNodeCfg.NodeID),
|
||||
HardwareInfos: metricsinfo.HardwareMetrics{
|
||||
CPUCoreCount: metricsinfo.GetCPUCoreCount(false),
|
||||
CPUCoreUsage: metricsinfo.GetCPUUsage(),
|
||||
@ -337,14 +337,14 @@ func getMockSystemInfoMetrics(
|
||||
DiskUsage: metricsinfo.GetDiskUsage(),
|
||||
},
|
||||
SystemInfo: metricsinfo.DeployMetrics{},
|
||||
CreatedTime: Params.CreatedTime.String(),
|
||||
UpdatedTime: Params.UpdatedTime.String(),
|
||||
CreatedTime: Params.IndexNodeCfg.CreatedTime.String(),
|
||||
UpdatedTime: Params.IndexNodeCfg.UpdatedTime.String(),
|
||||
Type: typeutil.IndexNodeRole,
|
||||
},
|
||||
SystemConfigurations: metricsinfo.IndexNodeConfiguration{
|
||||
MinioBucketName: Params.MinioBucketName,
|
||||
MinioBucketName: Params.IndexNodeCfg.MinioBucketName,
|
||||
|
||||
SimdType: Params.SimdType,
|
||||
SimdType: Params.IndexNodeCfg.SimdType,
|
||||
},
|
||||
}
|
||||
|
||||
@ -358,6 +358,6 @@ func getMockSystemInfoMetrics(
|
||||
Reason: "",
|
||||
},
|
||||
Response: resp,
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.NodeID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.IndexNodeCfg.NodeID),
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -35,7 +35,7 @@ func getSystemInfoMetrics(
|
||||
// TODO(dragondriver): add more metrics
|
||||
nodeInfos := metricsinfo.IndexNodeInfos{
|
||||
BaseComponentInfos: metricsinfo.BaseComponentInfos{
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.NodeID),
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.IndexNodeCfg.NodeID),
|
||||
HardwareInfos: metricsinfo.HardwareMetrics{
|
||||
IP: node.session.Address,
|
||||
CPUCoreCount: metricsinfo.GetCPUCoreCount(false),
|
||||
@ -46,15 +46,15 @@ func getSystemInfoMetrics(
|
||||
DiskUsage: metricsinfo.GetDiskUsage(),
|
||||
},
|
||||
SystemInfo: metricsinfo.DeployMetrics{},
|
||||
CreatedTime: Params.CreatedTime.String(),
|
||||
UpdatedTime: Params.UpdatedTime.String(),
|
||||
CreatedTime: Params.IndexNodeCfg.CreatedTime.String(),
|
||||
UpdatedTime: Params.IndexNodeCfg.UpdatedTime.String(),
|
||||
Type: typeutil.IndexNodeRole,
|
||||
ID: node.session.ServerID,
|
||||
},
|
||||
SystemConfigurations: metricsinfo.IndexNodeConfiguration{
|
||||
MinioBucketName: Params.MinioBucketName,
|
||||
MinioBucketName: Params.IndexNodeCfg.MinioBucketName,
|
||||
|
||||
SimdType: Params.SimdType,
|
||||
SimdType: Params.IndexNodeCfg.SimdType,
|
||||
},
|
||||
}
|
||||
|
||||
@ -68,7 +68,7 @@ func getSystemInfoMetrics(
|
||||
Reason: err.Error(),
|
||||
},
|
||||
Response: "",
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.NodeID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.IndexNodeCfg.NodeID),
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -78,6 +78,6 @@ func getSystemInfoMetrics(
|
||||
Reason: "",
|
||||
},
|
||||
Response: resp,
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.NodeID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, Params.IndexNodeCfg.NodeID),
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -1,175 +0,0 @@
|
||||
// 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 indexnode
|
||||
|
||||
import (
|
||||
"path"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
)
|
||||
|
||||
// ParamTable is used to record configuration items.
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
|
||||
IP string
|
||||
Address string
|
||||
Port int
|
||||
|
||||
NodeID int64
|
||||
Alias string
|
||||
|
||||
EtcdEndpoints []string
|
||||
MetaRootPath string
|
||||
IndexStorageRootPath string
|
||||
|
||||
MinIOAddress string
|
||||
MinIOAccessKeyID string
|
||||
MinIOSecretAccessKey string
|
||||
MinIOUseSSL bool
|
||||
MinioBucketName string
|
||||
|
||||
SimdType string
|
||||
|
||||
CreatedTime time.Time
|
||||
UpdatedTime time.Time
|
||||
}
|
||||
|
||||
// Params is a package scoped variable of type ParamTable.
|
||||
var Params ParamTable
|
||||
var once sync.Once
|
||||
|
||||
// InitAlias initializes an alias for the IndexNode role.
|
||||
func (pt *ParamTable) InitAlias(alias string) {
|
||||
pt.Alias = alias
|
||||
}
|
||||
|
||||
// Init is used to initialize configuration items.
|
||||
func (pt *ParamTable) Init() {
|
||||
pt.BaseTable.Init()
|
||||
pt.initParams()
|
||||
}
|
||||
|
||||
// InitOnce is used to initialize configuration items, and it will only be called once.
|
||||
func (pt *ParamTable) InitOnce() {
|
||||
once.Do(func() {
|
||||
pt.Init()
|
||||
})
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initParams() {
|
||||
pt.initMinIOAddress()
|
||||
pt.initMinIOAccessKeyID()
|
||||
pt.initMinIOSecretAccessKey()
|
||||
pt.initMinIOUseSSL()
|
||||
pt.initMinioBucketName()
|
||||
pt.initEtcdEndpoints()
|
||||
pt.initMetaRootPath()
|
||||
pt.initIndexStorageRootPath()
|
||||
pt.initRoleName()
|
||||
pt.initKnowhereSimdType()
|
||||
}
|
||||
|
||||
// initMinIOAddress load minio address from BaseTable.
|
||||
func (pt *ParamTable) initMinIOAddress() {
|
||||
ret, err := pt.Load("_MinioAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinIOAddress = ret
|
||||
}
|
||||
|
||||
// initMinIOAccessKeyID load access key for minio from BaseTable.
|
||||
func (pt *ParamTable) initMinIOAccessKeyID() {
|
||||
ret, err := pt.Load("_MinioAccessKeyID")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinIOAccessKeyID = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMinIOSecretAccessKey() {
|
||||
ret, err := pt.Load("_MinioSecretAccessKey")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinIOSecretAccessKey = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMinIOUseSSL() {
|
||||
ret, err := pt.Load("_MinioUseSSL")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinIOUseSSL, err = strconv.ParseBool(ret)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initEtcdEndpoints() {
|
||||
endpoints, err := pt.Load("_EtcdEndpoints")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.EtcdEndpoints = strings.Split(endpoints, ",")
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMetaRootPath() {
|
||||
rootPath, err := pt.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := pt.Load("etcd.metaSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MetaRootPath = path.Join(rootPath, subPath)
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initIndexStorageRootPath() {
|
||||
rootPath, err := pt.Load("minio.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.IndexStorageRootPath = path.Join(rootPath, "index_files")
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMinioBucketName() {
|
||||
bucketName, err := pt.Load("_MinioBucketName")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MinioBucketName = bucketName
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initRoleName() {
|
||||
pt.RoleName = "indexnode"
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initKnowhereSimdType() {
|
||||
simdType := pt.LoadWithDefault("knowhere.simdType", "auto")
|
||||
pt.SimdType = simdType
|
||||
log.Debug("initialize the knowhere simd type", zap.String("simd_type", pt.SimdType))
|
||||
}
|
||||
@ -1,188 +0,0 @@
|
||||
// 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 indexnode
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestParamTable(t *testing.T) {
|
||||
Params.Init()
|
||||
|
||||
t.Run("IP", func(t *testing.T) {
|
||||
t.Logf("IP: %v", Params.IP)
|
||||
})
|
||||
|
||||
t.Run("Address", func(t *testing.T) {
|
||||
t.Logf("Address: %v", Params.Address)
|
||||
})
|
||||
|
||||
t.Run("Port", func(t *testing.T) {
|
||||
t.Logf("Port: %v", Params.Port)
|
||||
})
|
||||
|
||||
t.Run("NodeID", func(t *testing.T) {
|
||||
t.Logf("NodeID: %v", Params.NodeID)
|
||||
})
|
||||
|
||||
t.Run("Alias", func(t *testing.T) {
|
||||
t.Logf("Alias: %v", Params.Alias)
|
||||
})
|
||||
|
||||
t.Run("EtcdEndpoints", func(t *testing.T) {
|
||||
t.Logf("EtcdEndpoints: %v", Params.EtcdEndpoints)
|
||||
})
|
||||
|
||||
t.Run("MetaRootPath", func(t *testing.T) {
|
||||
t.Logf("MetaRootPath: %v", Params.MetaRootPath)
|
||||
})
|
||||
|
||||
t.Run("MinIOAddress", func(t *testing.T) {
|
||||
t.Logf("MinIOAddress: %v", Params.MinIOAddress)
|
||||
})
|
||||
|
||||
t.Run("MinIOAccessKeyID", func(t *testing.T) {
|
||||
t.Logf("MinIOAccessKeyID: %v", Params.MinIOAccessKeyID)
|
||||
})
|
||||
|
||||
t.Run("MinIOSecretAccessKey", func(t *testing.T) {
|
||||
t.Logf("MinIOSecretAccessKey: %v", Params.MinIOSecretAccessKey)
|
||||
})
|
||||
|
||||
t.Run("MinIOUseSSL", func(t *testing.T) {
|
||||
t.Logf("MinIOUseSSL: %v", Params.MinIOUseSSL)
|
||||
})
|
||||
|
||||
t.Run("MinioBucketName", func(t *testing.T) {
|
||||
t.Logf("MinioBucketName: %v", Params.MinioBucketName)
|
||||
})
|
||||
|
||||
t.Run("SimdType", func(t *testing.T) {
|
||||
t.Logf("SimdType: %v", Params.SimdType)
|
||||
})
|
||||
// FIXME(dragondriver): how to cover panic case? we use `LoadWithDefault` to initialize `SimdType`
|
||||
|
||||
t.Run("CreatedTime", func(t *testing.T) {
|
||||
Params.CreatedTime = time.Now()
|
||||
t.Logf("CreatedTime: %v", Params.CreatedTime)
|
||||
})
|
||||
|
||||
t.Run("UpdatedTime", func(t *testing.T) {
|
||||
Params.UpdatedTime = time.Now()
|
||||
t.Logf("UpdatedTime: %v", Params.UpdatedTime)
|
||||
})
|
||||
|
||||
t.Run("IndexStorageRootPath", func(t *testing.T) {
|
||||
t.Logf("IndexStorageRootPath: %v", Params.IndexStorageRootPath)
|
||||
})
|
||||
}
|
||||
|
||||
//TODO: Params Load should be return error when key does not exist.
|
||||
//func shouldPanic(t *testing.T, name string, f func()) {
|
||||
// defer func() {
|
||||
// if r := recover(); r != nil {
|
||||
// t.Logf("%v recover: %v", name, r)
|
||||
// }
|
||||
// }()
|
||||
// f()
|
||||
//}
|
||||
//
|
||||
//func TestParamTable_Panic(t *testing.T) {
|
||||
// Params.Init()
|
||||
//
|
||||
// t.Run("initMinIOAddress", func(t *testing.T) {
|
||||
// err := Params.Remove("_MinioAddress")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOAddress", func() {
|
||||
// Params.initMinIOAddress()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinIOAccessKeyID", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.accessKeyID")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOAccessKeyID", func() {
|
||||
// Params.initMinIOAccessKeyID()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinIOAccessKeyID", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.accessKeyID")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOAccessKeyID", func() {
|
||||
// Params.initMinIOAccessKeyID()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinIOSecretAccessKey", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.secretAccessKey")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOSecretAccessKey", func() {
|
||||
// Params.initMinIOSecretAccessKey()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinIOUseSSL", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.useSSL")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinIOUseSSL", func() {
|
||||
// Params.initMinIOUseSSL()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initEtcdEndpoints", func(t *testing.T) {
|
||||
// err := Params.Remove("_EtcdEndpoints")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initEtcdEndpoints", func() {
|
||||
// Params.initEtcdEndpoints()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMetaRootPath", func(t *testing.T) {
|
||||
// err := Params.Remove("etcd.rootPath")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMetaRootPath", func() {
|
||||
// Params.initMetaRootPath()
|
||||
// })
|
||||
//
|
||||
// err = Params.Save("etcd.rootPath", "test")
|
||||
// assert.Nil(t, err)
|
||||
// err = Params.Remove("etcd.metaSubPath")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMetaRootPath", func() {
|
||||
// Params.initMetaRootPath()
|
||||
// })
|
||||
// })
|
||||
//
|
||||
// t.Run("initMinioBucketName", func(t *testing.T) {
|
||||
// err := Params.Remove("minio.bucketName")
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// shouldPanic(t, "initMinioBucketName", func() {
|
||||
// Params.initMinioBucketName()
|
||||
// })
|
||||
// })
|
||||
//}
|
||||
@ -417,7 +417,7 @@ func (it *IndexBuildTask) Execute(ctx context.Context) error {
|
||||
|
||||
getSavePathByKey := func(key string) string {
|
||||
|
||||
return path.Join(Params.IndexStorageRootPath, strconv.Itoa(int(it.req.IndexBuildID)), strconv.Itoa(int(it.req.Version)),
|
||||
return path.Join(Params.IndexNodeCfg.IndexStorageRootPath, strconv.Itoa(int(it.req.IndexBuildID)), strconv.Itoa(int(it.req.Version)),
|
||||
strconv.Itoa(int(partitionID)), strconv.Itoa(int(segmentID)), key)
|
||||
}
|
||||
saveBlob := func(path string, value []byte) error {
|
||||
|
||||
@ -1962,7 +1962,7 @@ func (node *Proxy) Insert(ctx context.Context, request *milvuspb.InsertRequest)
|
||||
}
|
||||
|
||||
if len(it.PartitionName) <= 0 {
|
||||
it.PartitionName = Params.DefaultPartitionName
|
||||
it.PartitionName = Params.ProxyCfg.DefaultPartitionName
|
||||
}
|
||||
|
||||
constructFailedResponse := func(err error) *milvuspb.MutationResult {
|
||||
@ -2132,9 +2132,9 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest)
|
||||
SearchRequest: &internalpb.SearchRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
},
|
||||
resultBuf: make(chan []*internalpb.SearchResults),
|
||||
query: request,
|
||||
@ -2355,9 +2355,9 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (*
|
||||
RetrieveRequest: &internalpb.RetrieveRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Retrieve,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
},
|
||||
resultBuf: make(chan []*internalpb.RetrieveResults),
|
||||
query: queryRequest,
|
||||
@ -2737,9 +2737,9 @@ func (node *Proxy) CalcDistance(ctx context.Context, request *milvuspb.CalcDista
|
||||
RetrieveRequest: &internalpb.RetrieveRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Retrieve,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
},
|
||||
resultBuf: make(chan []*internalpb.RetrieveResults),
|
||||
query: queryRequest,
|
||||
@ -3168,7 +3168,7 @@ func (node *Proxy) GetPersistentSegmentInfo(ctx context.Context, req *milvuspb.G
|
||||
MsgType: commonpb.MsgType_SegmentInfo,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
SegmentIDs: segments,
|
||||
})
|
||||
@ -3228,7 +3228,7 @@ func (node *Proxy) GetQuerySegmentInfo(ctx context.Context, req *milvuspb.GetQue
|
||||
MsgType: commonpb.MsgType_SegmentInfo,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
CollectionID: collID,
|
||||
SegmentIDs: segments,
|
||||
@ -3270,7 +3270,7 @@ func (node *Proxy) getSegmentsOfCollection(ctx context.Context, dbName string, c
|
||||
MsgType: commonpb.MsgType_DescribeCollection,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
@ -3287,7 +3287,7 @@ func (node *Proxy) getSegmentsOfCollection(ctx context.Context, dbName string, c
|
||||
MsgType: commonpb.MsgType_ShowPartitions,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
@ -3307,7 +3307,7 @@ func (node *Proxy) getSegmentsOfCollection(ctx context.Context, dbName string, c
|
||||
MsgType: commonpb.MsgType_ShowSegments,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
CollectionID: collectionID,
|
||||
PartitionID: partitionID,
|
||||
@ -3393,19 +3393,19 @@ func (node *Proxy) RegisterLink(ctx context.Context, req *milvuspb.RegisterLinkR
|
||||
// TODO(dragondriver): cache the Metrics and set a retention to the cache
|
||||
func (node *Proxy) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
||||
log.Debug("Proxy.GetMetrics",
|
||||
zap.Int64("node_id", Params.ProxyID),
|
||||
zap.Int64("node_id", Params.ProxyCfg.ProxyID),
|
||||
zap.String("req", req.Request))
|
||||
|
||||
if !node.checkHealthy() {
|
||||
log.Warn("Proxy.GetMetrics failed",
|
||||
zap.Int64("node_id", Params.ProxyID),
|
||||
zap.Int64("node_id", Params.ProxyCfg.ProxyID),
|
||||
zap.String("req", req.Request),
|
||||
zap.Error(errProxyIsUnhealthy(Params.ProxyID)))
|
||||
zap.Error(errProxyIsUnhealthy(Params.ProxyCfg.ProxyID)))
|
||||
|
||||
return &milvuspb.GetMetricsResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: msgProxyIsUnhealthy(Params.ProxyID),
|
||||
Reason: msgProxyIsUnhealthy(Params.ProxyCfg.ProxyID),
|
||||
},
|
||||
Response: "",
|
||||
}, nil
|
||||
@ -3414,7 +3414,7 @@ func (node *Proxy) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsReque
|
||||
metricType, err := metricsinfo.ParseMetricType(req.Request)
|
||||
if err != nil {
|
||||
log.Warn("Proxy.GetMetrics failed to parse metric type",
|
||||
zap.Int64("node_id", Params.ProxyID),
|
||||
zap.Int64("node_id", Params.ProxyCfg.ProxyID),
|
||||
zap.String("req", req.Request),
|
||||
zap.Error(err))
|
||||
|
||||
@ -3440,7 +3440,7 @@ func (node *Proxy) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsReque
|
||||
MsgType: commonpb.MsgType_SystemInfo,
|
||||
MsgID: msgID,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
}
|
||||
|
||||
if metricType == metricsinfo.SystemInfoMetrics {
|
||||
@ -3454,7 +3454,7 @@ func (node *Proxy) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsReque
|
||||
metrics, err := getSystemInfoMetrics(ctx, req, node)
|
||||
|
||||
log.Debug("Proxy.GetMetrics",
|
||||
zap.Int64("node_id", Params.ProxyID),
|
||||
zap.Int64("node_id", Params.ProxyCfg.ProxyID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType),
|
||||
zap.Any("metrics", metrics), // TODO(dragondriver): necessary? may be very large
|
||||
@ -3466,7 +3466,7 @@ func (node *Proxy) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsReque
|
||||
}
|
||||
|
||||
log.Debug("Proxy.GetMetrics failed, request metric type is not implemented yet",
|
||||
zap.Int64("node_id", Params.ProxyID),
|
||||
zap.Int64("node_id", Params.ProxyCfg.ProxyID),
|
||||
zap.String("req", req.Request),
|
||||
zap.String("metric_type", metricType))
|
||||
|
||||
@ -3482,7 +3482,7 @@ func (node *Proxy) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsReque
|
||||
// LoadBalance would do a load balancing operation between query nodes
|
||||
func (node *Proxy) LoadBalance(ctx context.Context, req *milvuspb.LoadBalanceRequest) (*commonpb.Status, error) {
|
||||
log.Debug("Proxy.LoadBalance",
|
||||
zap.Int64("proxy_id", Params.ProxyID),
|
||||
zap.Int64("proxy_id", Params.ProxyCfg.ProxyID),
|
||||
zap.Any("req", req))
|
||||
|
||||
if !node.checkHealthy() {
|
||||
@ -3497,7 +3497,7 @@ func (node *Proxy) LoadBalance(ctx context.Context, req *milvuspb.LoadBalanceReq
|
||||
MsgType: commonpb.MsgType_LoadBalanceSegments,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
SourceNodeIDs: []int64{req.SrcNodeID},
|
||||
DstNodeIDs: req.DstNodeIDs,
|
||||
|
||||
@ -46,7 +46,7 @@ func getSystemInfoMetrics(
|
||||
|
||||
identifierMap := make(map[string]int)
|
||||
|
||||
proxyRoleName := metricsinfo.ConstructComponentName(typeutil.ProxyRole, Params.ProxyID)
|
||||
proxyRoleName := metricsinfo.ConstructComponentName(typeutil.ProxyRole, Params.ProxyCfg.ProxyID)
|
||||
identifierMap[proxyRoleName] = int(node.session.ServerID)
|
||||
|
||||
proxyTopologyNode := metricsinfo.SystemTopologyNode{
|
||||
@ -67,14 +67,14 @@ func getSystemInfoMetrics(
|
||||
DiskUsage: metricsinfo.GetDiskUsage(),
|
||||
},
|
||||
SystemInfo: metricsinfo.DeployMetrics{},
|
||||
CreatedTime: Params.CreatedTime.String(),
|
||||
UpdatedTime: Params.UpdatedTime.String(),
|
||||
CreatedTime: Params.ProxyCfg.CreatedTime.String(),
|
||||
UpdatedTime: Params.ProxyCfg.UpdatedTime.String(),
|
||||
Type: typeutil.ProxyRole,
|
||||
ID: node.session.ServerID,
|
||||
},
|
||||
SystemConfigurations: metricsinfo.ProxyConfiguration{
|
||||
DefaultPartitionName: Params.DefaultPartitionName,
|
||||
DefaultIndexName: Params.DefaultIndexName,
|
||||
DefaultPartitionName: Params.ProxyCfg.DefaultPartitionName,
|
||||
DefaultIndexName: Params.ProxyCfg.DefaultIndexName,
|
||||
},
|
||||
},
|
||||
}
|
||||
@ -424,7 +424,7 @@ func getSystemInfoMetrics(
|
||||
Reason: err.Error(),
|
||||
},
|
||||
Response: "",
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.ProxyRole, Params.ProxyID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.ProxyRole, Params.ProxyCfg.ProxyID),
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -434,6 +434,6 @@ func getSystemInfoMetrics(
|
||||
Reason: "",
|
||||
},
|
||||
Response: resp,
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.ProxyRole, Params.ProxyID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.ProxyRole, Params.ProxyCfg.ProxyID),
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -1,298 +0,0 @@
|
||||
// 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 proxy
|
||||
|
||||
import (
|
||||
"path"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
)
|
||||
|
||||
const (
|
||||
// SuggestPulsarMaxMessageSize defines the maximum size of Pulsar message.
|
||||
SuggestPulsarMaxMessageSize = 5 * 1024 * 1024
|
||||
)
|
||||
|
||||
// ParamTable is a derived struct of paramtable.BaseTable. It achieves Composition by
|
||||
// embedding paramtable.BaseTable. It is used to quickly and easily access the system configuration.
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
|
||||
// NetworkPort & IP are not used
|
||||
NetworkPort int
|
||||
IP string
|
||||
|
||||
NetworkAddress string
|
||||
|
||||
Alias string
|
||||
|
||||
EtcdEndpoints []string
|
||||
MetaRootPath string
|
||||
PulsarAddress string
|
||||
|
||||
RocksmqPath string // not used in Proxy
|
||||
|
||||
ProxyID UniqueID
|
||||
TimeTickInterval time.Duration
|
||||
MsgStreamTimeTickBufSize int64
|
||||
MaxNameLength int64
|
||||
MaxFieldNum int64
|
||||
MaxShardNum int32
|
||||
MaxDimension int64
|
||||
DefaultPartitionName string
|
||||
DefaultIndexName string
|
||||
BufFlagExpireTime time.Duration
|
||||
BufFlagCleanupInterval time.Duration
|
||||
|
||||
// --- Channels ---
|
||||
ClusterChannelPrefix string
|
||||
ProxyTimeTickChannelNames []string
|
||||
ProxySubName string
|
||||
|
||||
// required from query coord
|
||||
SearchResultChannelNames []string
|
||||
RetrieveResultChannelNames []string
|
||||
|
||||
MaxTaskNum int64
|
||||
|
||||
PulsarMaxMessageSize int
|
||||
|
||||
CreatedTime time.Time
|
||||
UpdatedTime time.Time
|
||||
}
|
||||
|
||||
// Params is a package scoped variable of type ParamTable.
|
||||
var Params ParamTable
|
||||
var once sync.Once
|
||||
|
||||
// InitOnce ensures that Init is only called once.
|
||||
func (pt *ParamTable) InitOnce() {
|
||||
once.Do(func() {
|
||||
pt.Init()
|
||||
})
|
||||
}
|
||||
|
||||
// Init is an override method of BaseTable's Init. It mainly calls the
|
||||
// Init of BaseTable and do some other initialization.
|
||||
func (pt *ParamTable) Init() {
|
||||
pt.BaseTable.Init()
|
||||
pt.initEtcdEndpoints()
|
||||
pt.initMetaRootPath()
|
||||
pt.initPulsarAddress()
|
||||
pt.initRocksmqPath()
|
||||
pt.initTimeTickInterval()
|
||||
// Has to init global msgchannel prefix before other channel names
|
||||
pt.initClusterMsgChannelPrefix()
|
||||
pt.initProxySubName()
|
||||
pt.initProxyTimeTickChannelNames()
|
||||
pt.initMsgStreamTimeTickBufSize()
|
||||
pt.initMaxNameLength()
|
||||
pt.initMaxFieldNum()
|
||||
pt.initMaxShardNum()
|
||||
pt.initMaxDimension()
|
||||
pt.initDefaultPartitionName()
|
||||
pt.initDefaultIndexName()
|
||||
|
||||
pt.initPulsarMaxMessageSize()
|
||||
|
||||
pt.initMaxTaskNum()
|
||||
pt.initBufFlagExpireTime()
|
||||
pt.initBufFlagCleanupInterval()
|
||||
|
||||
pt.initRoleName()
|
||||
}
|
||||
|
||||
// InitAlias initialize Alias member.
|
||||
func (pt *ParamTable) InitAlias(alias string) {
|
||||
pt.Alias = alias
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initPulsarAddress() {
|
||||
ret, err := pt.Load("_PulsarAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.PulsarAddress = ret
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initRocksmqPath() {
|
||||
path, err := pt.Load("_RocksmqPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.RocksmqPath = path
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initTimeTickInterval() {
|
||||
interval := pt.ParseIntWithDefault("proxy.timeTickInterval", 200)
|
||||
pt.TimeTickInterval = time.Duration(interval) * time.Millisecond
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initClusterMsgChannelPrefix() {
|
||||
config, err := pt.Load("msgChannel.chanNamePrefix.cluster")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.ClusterChannelPrefix = config
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initProxySubName() {
|
||||
config, err := pt.Load("msgChannel.subNamePrefix.proxySubNamePrefix")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{pt.ClusterChannelPrefix, config, strconv.FormatInt(pt.ProxyID, 10)}
|
||||
pt.ProxySubName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initProxyTimeTickChannelNames() {
|
||||
config, err := pt.Load("msgChannel.chanNamePrefix.proxyTimeTick")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{pt.ClusterChannelPrefix, config, "0"}
|
||||
prefix := strings.Join(s, "-")
|
||||
pt.ProxyTimeTickChannelNames = []string{prefix}
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMsgStreamTimeTickBufSize() {
|
||||
pt.MsgStreamTimeTickBufSize = pt.ParseInt64WithDefault("proxy.msgStream.timeTick.bufSize", 512)
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMaxNameLength() {
|
||||
str := pt.LoadWithDefault("proxy.maxNameLength", "255")
|
||||
maxNameLength, err := strconv.ParseInt(str, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MaxNameLength = maxNameLength
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMaxShardNum() {
|
||||
str := pt.LoadWithDefault("proxy.maxShardNum", "256")
|
||||
maxShardNum, err := strconv.ParseInt(str, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MaxShardNum = int32(maxShardNum)
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMaxFieldNum() {
|
||||
str := pt.LoadWithDefault("proxy.maxFieldNum", "64")
|
||||
maxFieldNum, err := strconv.ParseInt(str, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MaxFieldNum = maxFieldNum
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMaxDimension() {
|
||||
str := pt.LoadWithDefault("proxy.maxDimension", "32768")
|
||||
maxDimension, err := strconv.ParseInt(str, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MaxDimension = maxDimension
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initDefaultPartitionName() {
|
||||
name := pt.LoadWithDefault("common.defaultPartitionName", "_default")
|
||||
pt.DefaultPartitionName = name
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initDefaultIndexName() {
|
||||
name := pt.LoadWithDefault("common.defaultIndexName", "_default_idx")
|
||||
pt.DefaultIndexName = name
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initPulsarMaxMessageSize() {
|
||||
// pulsarHost, err := pt.Load("pulsar.address")
|
||||
// if err != nil {
|
||||
// panic(err)
|
||||
// }
|
||||
|
||||
// pulsarRestPort, err := pt.Load("pulsar.rest-port")
|
||||
// if err != nil {
|
||||
// panic(err)
|
||||
// }
|
||||
|
||||
// protocol := "http"
|
||||
// url := "/admin/v2/brokers/configuration/runtime"
|
||||
// runtimeConfig, err := GetPulsarConfig(protocol, pulsarHost, pulsarRestPort, url)
|
||||
// if err != nil {
|
||||
// panic(err)
|
||||
// }
|
||||
// maxMessageSizeStr := fmt.Sprintf("%v", runtimeConfig[PulsarMaxMessageSizeKey])
|
||||
// pt.PulsarMaxMessageSize, err = strconv.Atoi(maxMessageSizeStr)
|
||||
// if err != nil {
|
||||
// panic(err)
|
||||
// }
|
||||
|
||||
maxMessageSizeStr, err := pt.Load("pulsar.maxMessageSize")
|
||||
if err != nil {
|
||||
pt.PulsarMaxMessageSize = SuggestPulsarMaxMessageSize
|
||||
} else {
|
||||
maxMessageSize, err := strconv.Atoi(maxMessageSizeStr)
|
||||
if err != nil {
|
||||
pt.PulsarMaxMessageSize = SuggestPulsarMaxMessageSize
|
||||
} else {
|
||||
pt.PulsarMaxMessageSize = maxMessageSize
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initRoleName() {
|
||||
pt.RoleName = "proxy"
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initEtcdEndpoints() {
|
||||
endpoints, err := pt.Load("_EtcdEndpoints")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.EtcdEndpoints = strings.Split(endpoints, ",")
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMetaRootPath() {
|
||||
rootPath, err := pt.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := pt.Load("etcd.metaSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pt.MetaRootPath = path.Join(rootPath, subPath)
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initMaxTaskNum() {
|
||||
pt.MaxTaskNum = pt.ParseInt64WithDefault("proxy.maxTaskNum", 1024)
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initBufFlagExpireTime() {
|
||||
expireTime := pt.ParseInt64WithDefault("proxy.bufFlagExpireTime", 3600)
|
||||
pt.BufFlagExpireTime = time.Duration(expireTime) * time.Second
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initBufFlagCleanupInterval() {
|
||||
interval := pt.ParseInt64WithDefault("proxy.bufFlagCleanupInterval", 600)
|
||||
pt.BufFlagCleanupInterval = time.Duration(interval) * time.Second
|
||||
}
|
||||
@ -1,141 +0,0 @@
|
||||
// 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 proxy
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestParamTable_Normal(t *testing.T) {
|
||||
Params.Init()
|
||||
|
||||
t.Run("EtcdEndPoints", func(t *testing.T) {
|
||||
t.Logf("EtcdEndPoints: %v", Params.EtcdEndpoints)
|
||||
})
|
||||
|
||||
t.Run("MetaRootPath", func(t *testing.T) {
|
||||
t.Logf("MetaRootPath: %s", Params.MetaRootPath)
|
||||
})
|
||||
|
||||
t.Run("PulsarAddress", func(t *testing.T) {
|
||||
t.Logf("PulsarAddress: %s", Params.PulsarAddress)
|
||||
})
|
||||
|
||||
t.Run("RocksmqPath", func(t *testing.T) {
|
||||
t.Logf("RocksmqPath: %s", Params.RocksmqPath)
|
||||
})
|
||||
|
||||
t.Run("TimeTickInterval", func(t *testing.T) {
|
||||
t.Logf("TimeTickInterval: %v", Params.TimeTickInterval)
|
||||
})
|
||||
|
||||
t.Run("ProxySubName", func(t *testing.T) {
|
||||
assert.Equal(t, Params.ProxySubName, "by-dev-proxy-0")
|
||||
t.Logf("ProxySubName: %s", Params.ProxySubName)
|
||||
})
|
||||
|
||||
t.Run("ProxyTimeTickChannelNames", func(t *testing.T) {
|
||||
assert.Equal(t, Params.ProxyTimeTickChannelNames, []string{"by-dev-proxyTimeTick-0"})
|
||||
t.Logf("ProxyTimeTickChannelNames: %v", Params.ProxyTimeTickChannelNames)
|
||||
})
|
||||
|
||||
t.Run("MsgStreamTimeTickBufSize", func(t *testing.T) {
|
||||
t.Logf("MsgStreamTimeTickBufSize: %d", Params.MsgStreamTimeTickBufSize)
|
||||
})
|
||||
|
||||
t.Run("MaxNameLength", func(t *testing.T) {
|
||||
t.Logf("MaxNameLength: %d", Params.MaxNameLength)
|
||||
})
|
||||
|
||||
t.Run("MaxFieldNum", func(t *testing.T) {
|
||||
t.Logf("MaxFieldNum: %d", Params.MaxFieldNum)
|
||||
})
|
||||
|
||||
t.Run("MaxShardNum", func(t *testing.T) {
|
||||
t.Logf("MaxShardNum: %d", Params.MaxShardNum)
|
||||
})
|
||||
|
||||
t.Run("MaxDimension", func(t *testing.T) {
|
||||
t.Logf("MaxDimension: %d", Params.MaxDimension)
|
||||
})
|
||||
|
||||
t.Run("DefaultPartitionName", func(t *testing.T) {
|
||||
t.Logf("DefaultPartitionName: %s", Params.DefaultPartitionName)
|
||||
})
|
||||
|
||||
t.Run("DefaultIndexName", func(t *testing.T) {
|
||||
t.Logf("DefaultIndexName: %s", Params.DefaultIndexName)
|
||||
})
|
||||
|
||||
t.Run("PulsarMaxMessageSize", func(t *testing.T) {
|
||||
t.Logf("PulsarMaxMessageSize: %d", Params.PulsarMaxMessageSize)
|
||||
})
|
||||
|
||||
t.Run("RoleName", func(t *testing.T) {
|
||||
t.Logf("RoleName: %s", typeutil.ProxyRole)
|
||||
})
|
||||
|
||||
t.Run("MaxTaskNum", func(t *testing.T) {
|
||||
t.Logf("MaxTaskNum: %d", Params.MaxTaskNum)
|
||||
})
|
||||
}
|
||||
|
||||
func shouldPanic(t *testing.T, name string, f func()) {
|
||||
defer func() { recover() }()
|
||||
f()
|
||||
t.Errorf("%s should have panicked", name)
|
||||
}
|
||||
|
||||
func TestParamTable_Panics(t *testing.T) {
|
||||
shouldPanic(t, "proxy.timeTickInterval", func() {
|
||||
Params.Save("proxy.timeTickInterval", "")
|
||||
Params.initTimeTickInterval()
|
||||
})
|
||||
|
||||
shouldPanic(t, "proxy.msgStream.timeTick.bufSize", func() {
|
||||
Params.Save("proxy.msgStream.timeTick.bufSize", "abc")
|
||||
Params.initMsgStreamTimeTickBufSize()
|
||||
})
|
||||
|
||||
shouldPanic(t, "proxy.maxNameLength", func() {
|
||||
Params.Save("proxy.maxNameLength", "abc")
|
||||
Params.initMaxNameLength()
|
||||
})
|
||||
|
||||
shouldPanic(t, "proxy.maxFieldNum", func() {
|
||||
Params.Save("proxy.maxFieldNum", "abc")
|
||||
Params.initMaxFieldNum()
|
||||
})
|
||||
|
||||
shouldPanic(t, "proxy.maxShardNum", func() {
|
||||
Params.Save("proxy.maxShardNum", "abc")
|
||||
Params.initMaxShardNum()
|
||||
})
|
||||
|
||||
shouldPanic(t, "proxy.maxDimension", func() {
|
||||
Params.Save("proxy.maxDimension", "-asdf")
|
||||
Params.initMaxDimension()
|
||||
})
|
||||
|
||||
shouldPanic(t, "proxy.maxTaskNum", func() {
|
||||
Params.Save("proxy.maxTaskNum", "-asdf")
|
||||
Params.initMaxTaskNum()
|
||||
})
|
||||
}
|
||||
@ -25,23 +25,21 @@ import (
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/logutil"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// UniqueID is alias of typeutil.UniqueID
|
||||
@ -56,6 +54,8 @@ const channelMgrTickerInterval = 100 * time.Millisecond
|
||||
// make sure Proxy implements types.Proxy
|
||||
var _ types.Proxy = (*Proxy)(nil)
|
||||
|
||||
var Params paramtable.GlobalParamTable
|
||||
|
||||
// Proxy of milvus
|
||||
type Proxy struct {
|
||||
ctx context.Context
|
||||
@ -125,13 +125,13 @@ func (node *Proxy) Register() error {
|
||||
}
|
||||
|
||||
func (node *Proxy) initSession() error {
|
||||
node.session = sessionutil.NewSession(node.ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
node.session = sessionutil.NewSession(node.ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
||||
if node.session == nil {
|
||||
return errors.New("new session failed, maybe etcd cannot be connected")
|
||||
}
|
||||
node.session.Init(typeutil.ProxyRole, Params.NetworkAddress, false)
|
||||
Params.ProxyID = node.session.ServerID
|
||||
Params.SetLogger(Params.ProxyID)
|
||||
node.session.Init(typeutil.ProxyRole, Params.ProxyCfg.NetworkAddress, false)
|
||||
Params.ProxyCfg.ProxyID = node.session.ServerID
|
||||
Params.BaseParams.SetLogger(Params.ProxyCfg.ProxyID)
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -142,7 +142,8 @@ func (node *Proxy) Init() error {
|
||||
log.Error("Proxy init session failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
Params.initProxySubName()
|
||||
// TODO: caiyd need check
|
||||
//Params.initProxySubName()
|
||||
// wait for datacoord state changed to Healthy
|
||||
if node.dataCoord != nil {
|
||||
log.Debug("Proxy wait for DataCoord ready")
|
||||
@ -191,28 +192,28 @@ func (node *Proxy) Init() error {
|
||||
|
||||
// TODO SearchResultChannelNames and RetrieveResultChannelNames should not be part in the Param table
|
||||
// we should maintain a separate map for search result
|
||||
Params.SearchResultChannelNames = []string{resp.QueryResultChannel}
|
||||
Params.RetrieveResultChannelNames = []string{resp.QueryResultChannel}
|
||||
log.Debug("Proxy CreateQueryChannel success", zap.Any("SearchResultChannelNames", Params.SearchResultChannelNames))
|
||||
log.Debug("Proxy CreateQueryChannel success", zap.Any("RetrieveResultChannelNames", Params.RetrieveResultChannelNames))
|
||||
Params.ProxyCfg.SearchResultChannelNames = []string{resp.QueryResultChannel}
|
||||
Params.ProxyCfg.RetrieveResultChannelNames = []string{resp.QueryResultChannel}
|
||||
log.Debug("Proxy CreateQueryChannel success", zap.Any("SearchResultChannelNames", Params.ProxyCfg.SearchResultChannelNames))
|
||||
log.Debug("Proxy CreateQueryChannel success", zap.Any("RetrieveResultChannelNames", Params.ProxyCfg.RetrieveResultChannelNames))
|
||||
}
|
||||
|
||||
m := map[string]interface{}{
|
||||
"PulsarAddress": Params.PulsarAddress,
|
||||
"PulsarAddress": Params.ProxyCfg.PulsarAddress,
|
||||
"PulsarBufSize": 1024}
|
||||
err = node.msFactory.SetParams(m)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
idAllocator, err := allocator.NewIDAllocator(node.ctx, node.rootCoord, Params.ProxyID)
|
||||
idAllocator, err := allocator.NewIDAllocator(node.ctx, node.rootCoord, Params.ProxyCfg.ProxyID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
node.idAllocator = idAllocator
|
||||
|
||||
tsoAllocator, err := newTimestampAllocator(node.ctx, node.rootCoord, Params.ProxyID)
|
||||
tsoAllocator, err := newTimestampAllocator(node.ctx, node.rootCoord, Params.ProxyCfg.ProxyID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -223,7 +224,7 @@ func (node *Proxy) Init() error {
|
||||
panic(err)
|
||||
}
|
||||
node.segAssigner = segAssigner
|
||||
node.segAssigner.PeerID = Params.ProxyID
|
||||
node.segAssigner.PeerID = Params.ProxyCfg.ProxyID
|
||||
|
||||
dmlChannelsFunc := getDmlChannelsFunc(node.ctx, node.rootCoord)
|
||||
dqlChannelsFunc := getDqlChannelsFunc(node.ctx, node.session.ServerID, node.queryCoord)
|
||||
@ -349,8 +350,8 @@ func (node *Proxy) Start() error {
|
||||
cb()
|
||||
}
|
||||
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
Params.ProxyCfg.CreatedTime = time.Now()
|
||||
Params.ProxyCfg.UpdatedTime = time.Now()
|
||||
|
||||
node.UpdateStateCode(internalpb.StateCode_Healthy)
|
||||
log.Debug("Proxy", zap.Any("State", node.stateCode.Load()))
|
||||
|
||||
@ -112,7 +112,7 @@ func runRootCoord(ctx context.Context, localMsg bool) *grpcrootcoord.Server {
|
||||
go func() {
|
||||
rootcoord.Params.Init()
|
||||
if !localMsg {
|
||||
logutil.SetupLogger(&rootcoord.Params.Log)
|
||||
logutil.SetupLogger(&rootcoord.Params.BaseParams.Log)
|
||||
defer log.Sync()
|
||||
}
|
||||
|
||||
@ -143,7 +143,7 @@ func runQueryCoord(ctx context.Context, localMsg bool) *grpcquerycoord.Server {
|
||||
querycoord.Params.Init()
|
||||
|
||||
if !localMsg {
|
||||
logutil.SetupLogger(&querycoord.Params.Log)
|
||||
logutil.SetupLogger(&querycoord.Params.BaseParams.Log)
|
||||
defer log.Sync()
|
||||
}
|
||||
|
||||
@ -171,11 +171,11 @@ func runQueryNode(ctx context.Context, localMsg bool, alias string) *grpcqueryno
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
querynode.Params.InitAlias(alias)
|
||||
querynode.Params.QueryNodeCfg.InitAlias(alias)
|
||||
querynode.Params.Init()
|
||||
|
||||
if !localMsg {
|
||||
logutil.SetupLogger(&querynode.Params.Log)
|
||||
logutil.SetupLogger(&querynode.Params.BaseParams.Log)
|
||||
defer log.Sync()
|
||||
}
|
||||
|
||||
@ -206,7 +206,7 @@ func runDataCoord(ctx context.Context, localMsg bool) *grpcdatacoordclient.Serve
|
||||
datacoord.Params.Init()
|
||||
|
||||
if !localMsg {
|
||||
logutil.SetupLogger(&datacoord.Params.Log)
|
||||
logutil.SetupLogger(&datacoord.Params.BaseParams.Log)
|
||||
defer log.Sync()
|
||||
}
|
||||
|
||||
@ -234,11 +234,11 @@ func runDataNode(ctx context.Context, localMsg bool, alias string) *grpcdatanode
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
datanode.Params.InitAlias(alias)
|
||||
datanode.Params.DataNodeCfg.InitAlias(alias)
|
||||
datanode.Params.Init()
|
||||
|
||||
if !localMsg {
|
||||
logutil.SetupLogger(&datanode.Params.Log)
|
||||
logutil.SetupLogger(&datanode.Params.BaseParams.Log)
|
||||
defer log.Sync()
|
||||
}
|
||||
|
||||
@ -269,7 +269,7 @@ func runIndexCoord(ctx context.Context, localMsg bool) *grpcindexcoord.Server {
|
||||
indexcoord.Params.Init()
|
||||
|
||||
if !localMsg {
|
||||
logutil.SetupLogger(&indexcoord.Params.Log)
|
||||
logutil.SetupLogger(&indexcoord.Params.BaseParams.Log)
|
||||
defer log.Sync()
|
||||
}
|
||||
|
||||
@ -296,11 +296,11 @@ func runIndexNode(ctx context.Context, localMsg bool, alias string) *grpcindexno
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
indexnode.Params.InitAlias(alias)
|
||||
indexnode.Params.IndexNodeCfg.InitAlias(alias)
|
||||
indexnode.Params.Init()
|
||||
|
||||
if !localMsg {
|
||||
logutil.SetupLogger(&indexnode.Params.Log)
|
||||
logutil.SetupLogger(&indexnode.Params.BaseParams.Log)
|
||||
defer log.Sync()
|
||||
}
|
||||
|
||||
@ -419,7 +419,7 @@ func TestProxy(t *testing.T) {
|
||||
Params.Init()
|
||||
log.Info("Initialize parameter table of proxy")
|
||||
|
||||
rootCoordClient, err := rcc.NewClient(ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
rootCoordClient, err := rcc.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
||||
assert.NoError(t, err)
|
||||
err = rootCoordClient.Init()
|
||||
assert.NoError(t, err)
|
||||
@ -428,7 +428,7 @@ func TestProxy(t *testing.T) {
|
||||
proxy.SetRootCoordClient(rootCoordClient)
|
||||
log.Info("Proxy set root coordinator client")
|
||||
|
||||
dataCoordClient, err := grpcdatacoordclient2.NewClient(ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
dataCoordClient, err := grpcdatacoordclient2.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
||||
assert.NoError(t, err)
|
||||
err = dataCoordClient.Init()
|
||||
assert.NoError(t, err)
|
||||
@ -437,7 +437,7 @@ func TestProxy(t *testing.T) {
|
||||
proxy.SetDataCoordClient(dataCoordClient)
|
||||
log.Info("Proxy set data coordinator client")
|
||||
|
||||
queryCoordClient, err := grpcquerycoordclient.NewClient(ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
queryCoordClient, err := grpcquerycoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
||||
assert.NoError(t, err)
|
||||
err = queryCoordClient.Init()
|
||||
assert.NoError(t, err)
|
||||
@ -446,7 +446,7 @@ func TestProxy(t *testing.T) {
|
||||
proxy.SetQueryCoordClient(queryCoordClient)
|
||||
log.Info("Proxy set query coordinator client")
|
||||
|
||||
indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.ProxyCfg.MetaRootPath, Params.ProxyCfg.EtcdEndpoints)
|
||||
assert.NoError(t, err)
|
||||
err = indexCoordClient.Init()
|
||||
assert.NoError(t, err)
|
||||
@ -477,7 +477,7 @@ func TestProxy(t *testing.T) {
|
||||
states, err := proxy.GetComponentStates(ctx)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, states.Status.ErrorCode)
|
||||
assert.Equal(t, Params.ProxyID, states.State.NodeID)
|
||||
assert.Equal(t, Params.ProxyCfg.ProxyID, states.State.NodeID)
|
||||
assert.Equal(t, typeutil.ProxyRole, states.State.Role)
|
||||
assert.Equal(t, proxy.stateCode.Load().(internalpb.StateCode), states.State.StateCode)
|
||||
})
|
||||
|
||||
@ -718,7 +718,7 @@ func (it *insertTask) PreExecute(ctx context.Context) error {
|
||||
sp, ctx := trace.StartSpanFromContextWithOperationName(it.ctx, "Proxy-Insert-PreExecute")
|
||||
defer sp.Finish()
|
||||
it.Base.MsgType = commonpb.MsgType_Insert
|
||||
it.Base.SourceID = Params.ProxyID
|
||||
it.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
it.result = &milvuspb.MutationResult{
|
||||
Status: &commonpb.Status{
|
||||
@ -889,7 +889,7 @@ func (it *insertTask) _assignSegmentID(stream msgstream.MsgStream, pack *msgstre
|
||||
return 0
|
||||
}
|
||||
|
||||
threshold := Params.PulsarMaxMessageSize
|
||||
threshold := Params.ProxyCfg.PulsarMaxMessageSize
|
||||
log.Debug("Proxy", zap.Int("threshold of message size: ", threshold))
|
||||
// not accurate
|
||||
/* #nosec G103 */
|
||||
@ -1002,7 +1002,7 @@ func (it *insertTask) Execute(ctx context.Context) error {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
partitionID, err = globalMetaCache.GetPartitionID(ctx, collectionName, Params.DefaultPartitionName)
|
||||
partitionID, err = globalMetaCache.GetPartitionID(ctx, collectionName, Params.ProxyCfg.DefaultPartitionName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -1104,13 +1104,13 @@ func (cct *createCollectionTask) SetTs(ts Timestamp) {
|
||||
func (cct *createCollectionTask) OnEnqueue() error {
|
||||
cct.Base = &commonpb.MsgBase{}
|
||||
cct.Base.MsgType = commonpb.MsgType_CreateCollection
|
||||
cct.Base.SourceID = Params.ProxyID
|
||||
cct.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
return nil
|
||||
}
|
||||
|
||||
func (cct *createCollectionTask) PreExecute(ctx context.Context) error {
|
||||
cct.Base.MsgType = commonpb.MsgType_CreateCollection
|
||||
cct.Base.SourceID = Params.ProxyID
|
||||
cct.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
cct.schema = &schemapb.CollectionSchema{}
|
||||
err := proto.Unmarshal(cct.Schema, cct.schema)
|
||||
@ -1123,12 +1123,12 @@ func (cct *createCollectionTask) PreExecute(ctx context.Context) error {
|
||||
return err
|
||||
}
|
||||
|
||||
if cct.ShardsNum > Params.MaxShardNum {
|
||||
return fmt.Errorf("maximum shards's number should be limited to %d", Params.MaxShardNum)
|
||||
if cct.ShardsNum > Params.ProxyCfg.MaxShardNum {
|
||||
return fmt.Errorf("maximum shards's number should be limited to %d", Params.ProxyCfg.MaxShardNum)
|
||||
}
|
||||
|
||||
if int64(len(cct.schema.Fields)) > Params.MaxFieldNum {
|
||||
return fmt.Errorf("maximum field's number should be limited to %d", Params.MaxFieldNum)
|
||||
if int64(len(cct.schema.Fields)) > Params.ProxyCfg.MaxFieldNum {
|
||||
return fmt.Errorf("maximum field's number should be limited to %d", Params.ProxyCfg.MaxFieldNum)
|
||||
}
|
||||
|
||||
// validate collection name
|
||||
@ -1248,7 +1248,7 @@ func (dct *dropCollectionTask) OnEnqueue() error {
|
||||
|
||||
func (dct *dropCollectionTask) PreExecute(ctx context.Context) error {
|
||||
dct.Base.MsgType = commonpb.MsgType_DropCollection
|
||||
dct.Base.SourceID = Params.ProxyID
|
||||
dct.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
if err := validateCollectionName(dct.CollectionName); err != nil {
|
||||
return err
|
||||
@ -1376,7 +1376,7 @@ func (st *searchTask) SetTs(ts Timestamp) {
|
||||
func (st *searchTask) OnEnqueue() error {
|
||||
st.Base = &commonpb.MsgBase{}
|
||||
st.Base.MsgType = commonpb.MsgType_Search
|
||||
st.Base.SourceID = Params.ProxyID
|
||||
st.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -1422,7 +1422,7 @@ func (st *searchTask) PreExecute(ctx context.Context) error {
|
||||
sp, ctx := trace.StartSpanFromContextWithOperationName(st.TraceCtx(), "Proxy-Search-PreExecute")
|
||||
defer sp.Finish()
|
||||
st.Base.MsgType = commonpb.MsgType_Search
|
||||
st.Base.SourceID = Params.ProxyID
|
||||
st.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collectionName := st.query.CollectionName
|
||||
collID, err := globalMetaCache.GetCollectionID(ctx, collectionName)
|
||||
@ -1446,7 +1446,7 @@ func (st *searchTask) PreExecute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_ShowCollections,
|
||||
MsgID: st.Base.MsgID,
|
||||
Timestamp: st.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbID: 0, // TODO(dragondriver)
|
||||
})
|
||||
@ -1589,7 +1589,7 @@ func (st *searchTask) PreExecute(ctx context.Context) error {
|
||||
st.SearchRequest.TimeoutTimestamp = tsoutil.ComposeTSByTime(deadline, 0)
|
||||
}
|
||||
|
||||
st.SearchRequest.ResultChannelID = Params.SearchResultChannelNames[0]
|
||||
st.SearchRequest.ResultChannelID = Params.ProxyCfg.SearchResultChannelNames[0]
|
||||
st.SearchRequest.DbID = 0 // todo
|
||||
st.SearchRequest.CollectionID = collID
|
||||
st.SearchRequest.PartitionIDs = make([]UniqueID, 0)
|
||||
@ -1639,7 +1639,7 @@ func (st *searchTask) Execute(ctx context.Context) error {
|
||||
SearchRequest: *st.SearchRequest,
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
Ctx: ctx,
|
||||
HashValues: []uint32{uint32(Params.ProxyID)},
|
||||
HashValues: []uint32{uint32(Params.ProxyCfg.ProxyID)},
|
||||
BeginTimestamp: st.Base.Timestamp,
|
||||
EndTimestamp: st.Base.Timestamp,
|
||||
},
|
||||
@ -2059,7 +2059,7 @@ func IDs2Expr(fieldName string, ids []int64) string {
|
||||
|
||||
func (qt *queryTask) PreExecute(ctx context.Context) error {
|
||||
qt.Base.MsgType = commonpb.MsgType_Retrieve
|
||||
qt.Base.SourceID = Params.ProxyID
|
||||
qt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collectionName := qt.query.CollectionName
|
||||
|
||||
@ -2096,7 +2096,7 @@ func (qt *queryTask) PreExecute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_ShowCollections,
|
||||
MsgID: qt.Base.MsgID,
|
||||
Timestamp: qt.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbID: 0, // TODO(dragondriver)
|
||||
})
|
||||
@ -2201,7 +2201,7 @@ func (qt *queryTask) PreExecute(ctx context.Context) error {
|
||||
qt.RetrieveRequest.TimeoutTimestamp = tsoutil.ComposeTSByTime(deadline, 0)
|
||||
}
|
||||
|
||||
qt.ResultChannelID = Params.RetrieveResultChannelNames[0]
|
||||
qt.ResultChannelID = Params.ProxyCfg.RetrieveResultChannelNames[0]
|
||||
qt.DbID = 0 // todo(yukun)
|
||||
|
||||
qt.CollectionID = collectionID
|
||||
@ -2255,7 +2255,7 @@ func (qt *queryTask) Execute(ctx context.Context) error {
|
||||
RetrieveRequest: *qt.RetrieveRequest,
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
Ctx: ctx,
|
||||
HashValues: []uint32{uint32(Params.ProxyID)},
|
||||
HashValues: []uint32{uint32(Params.ProxyCfg.ProxyID)},
|
||||
BeginTimestamp: qt.Base.Timestamp,
|
||||
EndTimestamp: qt.Base.Timestamp,
|
||||
},
|
||||
@ -2461,7 +2461,7 @@ func (hct *hasCollectionTask) OnEnqueue() error {
|
||||
|
||||
func (hct *hasCollectionTask) PreExecute(ctx context.Context) error {
|
||||
hct.Base.MsgType = commonpb.MsgType_HasCollection
|
||||
hct.Base.SourceID = Params.ProxyID
|
||||
hct.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
if err := validateCollectionName(hct.CollectionName); err != nil {
|
||||
return err
|
||||
@ -2532,7 +2532,7 @@ func (dct *describeCollectionTask) OnEnqueue() error {
|
||||
|
||||
func (dct *describeCollectionTask) PreExecute(ctx context.Context) error {
|
||||
dct.Base.MsgType = commonpb.MsgType_DescribeCollection
|
||||
dct.Base.SourceID = Params.ProxyID
|
||||
dct.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
if dct.CollectionID != 0 && len(dct.CollectionName) == 0 {
|
||||
return nil
|
||||
@ -2646,7 +2646,7 @@ func (g *getCollectionStatisticsTask) OnEnqueue() error {
|
||||
|
||||
func (g *getCollectionStatisticsTask) PreExecute(ctx context.Context) error {
|
||||
g.Base.MsgType = commonpb.MsgType_GetCollectionStatistics
|
||||
g.Base.SourceID = Params.ProxyID
|
||||
g.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -2733,7 +2733,7 @@ func (g *getPartitionStatisticsTask) OnEnqueue() error {
|
||||
|
||||
func (g *getPartitionStatisticsTask) PreExecute(ctx context.Context) error {
|
||||
g.Base.MsgType = commonpb.MsgType_GetPartitionStatistics
|
||||
g.Base.SourceID = Params.ProxyID
|
||||
g.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -2826,7 +2826,7 @@ func (sct *showCollectionsTask) OnEnqueue() error {
|
||||
|
||||
func (sct *showCollectionsTask) PreExecute(ctx context.Context) error {
|
||||
sct.Base.MsgType = commonpb.MsgType_ShowCollections
|
||||
sct.Base.SourceID = Params.ProxyID
|
||||
sct.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
if sct.GetType() == milvuspb.ShowType_InMemory {
|
||||
for _, collectionName := range sct.CollectionNames {
|
||||
if err := validateCollectionName(collectionName); err != nil {
|
||||
@ -2980,7 +2980,7 @@ func (cpt *createPartitionTask) OnEnqueue() error {
|
||||
|
||||
func (cpt *createPartitionTask) PreExecute(ctx context.Context) error {
|
||||
cpt.Base.MsgType = commonpb.MsgType_CreatePartition
|
||||
cpt.Base.SourceID = Params.ProxyID
|
||||
cpt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName, partitionTag := cpt.CollectionName, cpt.PartitionName
|
||||
|
||||
@ -3057,7 +3057,7 @@ func (dpt *dropPartitionTask) OnEnqueue() error {
|
||||
|
||||
func (dpt *dropPartitionTask) PreExecute(ctx context.Context) error {
|
||||
dpt.Base.MsgType = commonpb.MsgType_DropPartition
|
||||
dpt.Base.SourceID = Params.ProxyID
|
||||
dpt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName, partitionTag := dpt.CollectionName, dpt.PartitionName
|
||||
|
||||
@ -3134,7 +3134,7 @@ func (hpt *hasPartitionTask) OnEnqueue() error {
|
||||
|
||||
func (hpt *hasPartitionTask) PreExecute(ctx context.Context) error {
|
||||
hpt.Base.MsgType = commonpb.MsgType_HasPartition
|
||||
hpt.Base.SourceID = Params.ProxyID
|
||||
hpt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName, partitionTag := hpt.CollectionName, hpt.PartitionName
|
||||
|
||||
@ -3211,7 +3211,7 @@ func (spt *showPartitionsTask) OnEnqueue() error {
|
||||
|
||||
func (spt *showPartitionsTask) PreExecute(ctx context.Context) error {
|
||||
spt.Base.MsgType = commonpb.MsgType_ShowPartitions
|
||||
spt.Base.SourceID = Params.ProxyID
|
||||
spt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
if err := validateCollectionName(spt.CollectionName); err != nil {
|
||||
return err
|
||||
@ -3375,7 +3375,7 @@ func (cit *createIndexTask) OnEnqueue() error {
|
||||
|
||||
func (cit *createIndexTask) PreExecute(ctx context.Context) error {
|
||||
cit.Base.MsgType = commonpb.MsgType_CreateIndex
|
||||
cit.Base.SourceID = Params.ProxyID
|
||||
cit.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName, fieldName := cit.CollectionName, cit.FieldName
|
||||
|
||||
@ -3489,7 +3489,7 @@ func (dit *describeIndexTask) OnEnqueue() error {
|
||||
|
||||
func (dit *describeIndexTask) PreExecute(ctx context.Context) error {
|
||||
dit.Base.MsgType = commonpb.MsgType_DescribeIndex
|
||||
dit.Base.SourceID = Params.ProxyID
|
||||
dit.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
if err := validateCollectionName(dit.CollectionName); err != nil {
|
||||
return err
|
||||
@ -3497,7 +3497,7 @@ func (dit *describeIndexTask) PreExecute(ctx context.Context) error {
|
||||
|
||||
// only support default index name for now. @2021.02.18
|
||||
if dit.IndexName == "" {
|
||||
dit.IndexName = Params.DefaultIndexName
|
||||
dit.IndexName = Params.ProxyCfg.DefaultIndexName
|
||||
}
|
||||
|
||||
return nil
|
||||
@ -3566,7 +3566,7 @@ func (dit *dropIndexTask) OnEnqueue() error {
|
||||
|
||||
func (dit *dropIndexTask) PreExecute(ctx context.Context) error {
|
||||
dit.Base.MsgType = commonpb.MsgType_DropIndex
|
||||
dit.Base.SourceID = Params.ProxyID
|
||||
dit.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName, fieldName := dit.CollectionName, dit.FieldName
|
||||
|
||||
@ -3579,7 +3579,7 @@ func (dit *dropIndexTask) PreExecute(ctx context.Context) error {
|
||||
}
|
||||
|
||||
if dit.IndexName == "" {
|
||||
dit.IndexName = Params.DefaultIndexName
|
||||
dit.IndexName = Params.ProxyCfg.DefaultIndexName
|
||||
}
|
||||
|
||||
return nil
|
||||
@ -3650,7 +3650,7 @@ func (gibpt *getIndexBuildProgressTask) OnEnqueue() error {
|
||||
|
||||
func (gibpt *getIndexBuildProgressTask) PreExecute(ctx context.Context) error {
|
||||
gibpt.Base.MsgType = commonpb.MsgType_GetIndexBuildProgress
|
||||
gibpt.Base.SourceID = Params.ProxyID
|
||||
gibpt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
if err := validateCollectionName(gibpt.CollectionName); err != nil {
|
||||
return err
|
||||
@ -3671,7 +3671,7 @@ func (gibpt *getIndexBuildProgressTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_ShowPartitions,
|
||||
MsgID: gibpt.Base.MsgID,
|
||||
Timestamp: gibpt.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: gibpt.DbName,
|
||||
CollectionName: collectionName,
|
||||
@ -3683,7 +3683,7 @@ func (gibpt *getIndexBuildProgressTask) Execute(ctx context.Context) error {
|
||||
}
|
||||
|
||||
if gibpt.IndexName == "" {
|
||||
gibpt.IndexName = Params.DefaultIndexName
|
||||
gibpt.IndexName = Params.ProxyCfg.DefaultIndexName
|
||||
}
|
||||
|
||||
describeIndexReq := milvuspb.DescribeIndexRequest{
|
||||
@ -3691,7 +3691,7 @@ func (gibpt *getIndexBuildProgressTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_DescribeIndex,
|
||||
MsgID: gibpt.Base.MsgID,
|
||||
Timestamp: gibpt.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: gibpt.DbName,
|
||||
CollectionName: gibpt.CollectionName,
|
||||
@ -3723,7 +3723,7 @@ func (gibpt *getIndexBuildProgressTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_ShowSegments,
|
||||
MsgID: gibpt.Base.MsgID,
|
||||
Timestamp: gibpt.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
CollectionID: collectionID,
|
||||
PartitionID: partitionID,
|
||||
@ -3749,7 +3749,7 @@ func (gibpt *getIndexBuildProgressTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_DescribeSegment,
|
||||
MsgID: gibpt.Base.MsgID,
|
||||
Timestamp: gibpt.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
CollectionID: collectionID,
|
||||
SegmentID: segmentID,
|
||||
@ -3789,7 +3789,7 @@ func (gibpt *getIndexBuildProgressTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_SegmentInfo,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
SegmentIDs: allSegmentIDs,
|
||||
})
|
||||
@ -3871,7 +3871,7 @@ func (gist *getIndexStateTask) OnEnqueue() error {
|
||||
|
||||
func (gist *getIndexStateTask) PreExecute(ctx context.Context) error {
|
||||
gist.Base.MsgType = commonpb.MsgType_GetIndexState
|
||||
gist.Base.SourceID = Params.ProxyID
|
||||
gist.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
if err := validateCollectionName(gist.CollectionName); err != nil {
|
||||
return err
|
||||
@ -3892,7 +3892,7 @@ func (gist *getIndexStateTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_ShowPartitions,
|
||||
MsgID: gist.Base.MsgID,
|
||||
Timestamp: gist.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: gist.DbName,
|
||||
CollectionName: collectionName,
|
||||
@ -3904,7 +3904,7 @@ func (gist *getIndexStateTask) Execute(ctx context.Context) error {
|
||||
}
|
||||
|
||||
if gist.IndexName == "" {
|
||||
gist.IndexName = Params.DefaultIndexName
|
||||
gist.IndexName = Params.ProxyCfg.DefaultIndexName
|
||||
}
|
||||
|
||||
describeIndexReq := milvuspb.DescribeIndexRequest{
|
||||
@ -3912,7 +3912,7 @@ func (gist *getIndexStateTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_DescribeIndex,
|
||||
MsgID: gist.Base.MsgID,
|
||||
Timestamp: gist.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: gist.DbName,
|
||||
CollectionName: gist.CollectionName,
|
||||
@ -3944,7 +3944,7 @@ func (gist *getIndexStateTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_ShowSegments,
|
||||
MsgID: gist.Base.MsgID,
|
||||
Timestamp: gist.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
CollectionID: collectionID,
|
||||
PartitionID: partitionID,
|
||||
@ -3969,7 +3969,7 @@ func (gist *getIndexStateTask) Execute(ctx context.Context) error {
|
||||
MsgType: commonpb.MsgType_DescribeSegment,
|
||||
MsgID: gist.Base.MsgID,
|
||||
Timestamp: gist.Base.Timestamp,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
CollectionID: collectionID,
|
||||
SegmentID: segmentID,
|
||||
@ -4077,7 +4077,7 @@ func (ft *flushTask) OnEnqueue() error {
|
||||
|
||||
func (ft *flushTask) PreExecute(ctx context.Context) error {
|
||||
ft.Base.MsgType = commonpb.MsgType_Flush
|
||||
ft.Base.SourceID = Params.ProxyID
|
||||
ft.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -4170,7 +4170,7 @@ func (lct *loadCollectionTask) OnEnqueue() error {
|
||||
func (lct *loadCollectionTask) PreExecute(ctx context.Context) error {
|
||||
log.Debug("loadCollectionTask PreExecute", zap.String("role", typeutil.ProxyRole), zap.Int64("msgID", lct.Base.MsgID))
|
||||
lct.Base.MsgType = commonpb.MsgType_LoadCollection
|
||||
lct.Base.SourceID = Params.ProxyID
|
||||
lct.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName := lct.CollectionName
|
||||
|
||||
@ -4267,7 +4267,7 @@ func (rct *releaseCollectionTask) OnEnqueue() error {
|
||||
|
||||
func (rct *releaseCollectionTask) PreExecute(ctx context.Context) error {
|
||||
rct.Base.MsgType = commonpb.MsgType_ReleaseCollection
|
||||
rct.Base.SourceID = Params.ProxyID
|
||||
rct.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName := rct.CollectionName
|
||||
|
||||
@ -4352,7 +4352,7 @@ func (lpt *loadPartitionsTask) OnEnqueue() error {
|
||||
|
||||
func (lpt *loadPartitionsTask) PreExecute(ctx context.Context) error {
|
||||
lpt.Base.MsgType = commonpb.MsgType_LoadPartitions
|
||||
lpt.Base.SourceID = Params.ProxyID
|
||||
lpt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName := lpt.CollectionName
|
||||
|
||||
@ -4447,7 +4447,7 @@ func (rpt *releasePartitionsTask) OnEnqueue() error {
|
||||
|
||||
func (rpt *releasePartitionsTask) PreExecute(ctx context.Context) error {
|
||||
rpt.Base.MsgType = commonpb.MsgType_ReleasePartitions
|
||||
rpt.Base.SourceID = Params.ProxyID
|
||||
rpt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collName := rpt.CollectionName
|
||||
|
||||
@ -4604,7 +4604,7 @@ func getPrimaryKeysFromExpr(schema *schemapb.CollectionSchema, expr string) (res
|
||||
|
||||
func (dt *deleteTask) PreExecute(ctx context.Context) error {
|
||||
dt.Base.MsgType = commonpb.MsgType_Delete
|
||||
dt.Base.SourceID = Params.ProxyID
|
||||
dt.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
dt.result = &milvuspb.MutationResult{
|
||||
Status: &commonpb.Status{
|
||||
@ -4842,7 +4842,7 @@ func (c *CreateAliasTask) OnEnqueue() error {
|
||||
|
||||
func (c *CreateAliasTask) PreExecute(ctx context.Context) error {
|
||||
c.Base.MsgType = commonpb.MsgType_CreateAlias
|
||||
c.Base.SourceID = Params.ProxyID
|
||||
c.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collAlias := c.Alias
|
||||
// collection alias uses the same format as collection name
|
||||
@ -4915,7 +4915,7 @@ func (d *DropAliasTask) OnEnqueue() error {
|
||||
|
||||
func (d *DropAliasTask) PreExecute(ctx context.Context) error {
|
||||
d.Base.MsgType = commonpb.MsgType_DropAlias
|
||||
d.Base.SourceID = Params.ProxyID
|
||||
d.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
collAlias := d.Alias
|
||||
if err := ValidateCollectionAlias(collAlias); err != nil {
|
||||
return err
|
||||
@ -4980,7 +4980,7 @@ func (a *AlterAliasTask) OnEnqueue() error {
|
||||
|
||||
func (a *AlterAliasTask) PreExecute(ctx context.Context) error {
|
||||
a.Base.MsgType = commonpb.MsgType_AlterAlias
|
||||
a.Base.SourceID = Params.ProxyID
|
||||
a.Base.SourceID = Params.ProxyCfg.ProxyID
|
||||
|
||||
collAlias := a.Alias
|
||||
// collection alias uses the same format as collection name
|
||||
|
||||
@ -209,8 +209,8 @@ func newBaseTaskQueue(tsoAllocatorIns tsoAllocator, idAllocatorIns idAllocatorIn
|
||||
activeTasks: make(map[UniqueID]task),
|
||||
utLock: sync.RWMutex{},
|
||||
atLock: sync.RWMutex{},
|
||||
maxTaskNum: Params.MaxTaskNum,
|
||||
utBufChan: make(chan int, Params.MaxTaskNum),
|
||||
maxTaskNum: Params.ProxyCfg.MaxTaskNum,
|
||||
utBufChan: make(chan int, Params.ProxyCfg.MaxTaskNum),
|
||||
tsoAllocatorIns: tsoAllocatorIns,
|
||||
idAllocatorIns: idAllocatorIns,
|
||||
}
|
||||
@ -631,17 +631,17 @@ func (sched *taskScheduler) collectResultLoop() {
|
||||
|
||||
queryResultMsgStream, _ := sched.msFactory.NewQueryMsgStream(sched.ctx)
|
||||
// proxy didn't need to walk through all the search results in channel, because it no longer has client connections.
|
||||
queryResultMsgStream.AsConsumerWithPosition(Params.SearchResultChannelNames, Params.ProxySubName, mqclient.SubscriptionPositionLatest)
|
||||
log.Debug("Proxy", zap.Strings("SearchResultChannelNames", Params.SearchResultChannelNames),
|
||||
zap.Any("ProxySubName", Params.ProxySubName))
|
||||
queryResultMsgStream.AsConsumerWithPosition(Params.ProxyCfg.SearchResultChannelNames, Params.ProxyCfg.ProxySubName, mqclient.SubscriptionPositionLatest)
|
||||
log.Debug("Proxy", zap.Strings("SearchResultChannelNames", Params.ProxyCfg.SearchResultChannelNames),
|
||||
zap.Any("ProxySubName", Params.ProxyCfg.ProxySubName))
|
||||
|
||||
queryResultMsgStream.Start()
|
||||
defer queryResultMsgStream.Close()
|
||||
|
||||
searchResultBufs := make(map[UniqueID]*searchResultBuf)
|
||||
searchResultBufFlags := newIDCache(Params.BufFlagExpireTime, Params.BufFlagCleanupInterval) // if value is true, we can ignore searchResult
|
||||
searchResultBufFlags := newIDCache(Params.ProxyCfg.BufFlagExpireTime, Params.ProxyCfg.BufFlagCleanupInterval) // if value is true, we can ignore searchResult
|
||||
queryResultBufs := make(map[UniqueID]*queryResultBuf)
|
||||
queryResultBufFlags := newIDCache(Params.BufFlagExpireTime, Params.BufFlagCleanupInterval) // if value is true, we can ignore queryResult
|
||||
queryResultBufFlags := newIDCache(Params.ProxyCfg.BufFlagExpireTime, Params.ProxyCfg.BufFlagCleanupInterval) // if value is true, we can ignore queryResult
|
||||
|
||||
for {
|
||||
select {
|
||||
|
||||
@ -801,9 +801,9 @@ func TestSearchTask(t *testing.T) {
|
||||
SearchRequest: &internalpb.SearchRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
},
|
||||
resultBuf: make(chan []*internalpb.SearchResults),
|
||||
query: nil,
|
||||
@ -831,9 +831,9 @@ func TestSearchTask(t *testing.T) {
|
||||
SearchRequest: &internalpb.SearchRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
},
|
||||
resultBuf: make(chan []*internalpb.SearchResults),
|
||||
query: nil,
|
||||
@ -866,9 +866,9 @@ func TestSearchTask(t *testing.T) {
|
||||
SearchRequest: &internalpb.SearchRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
},
|
||||
resultBuf: make(chan []*internalpb.SearchResults),
|
||||
query: nil,
|
||||
@ -990,7 +990,7 @@ func TestCreateCollectionTask(t *testing.T) {
|
||||
assert.Error(t, err)
|
||||
task.Schema = marshaledSchema
|
||||
|
||||
task.ShardsNum = Params.MaxShardNum + 1
|
||||
task.ShardsNum = Params.ProxyCfg.MaxShardNum + 1
|
||||
err = task.PreExecute(ctx)
|
||||
assert.Error(t, err)
|
||||
task.ShardsNum = shardsNum
|
||||
@ -1002,7 +1002,7 @@ func TestCreateCollectionTask(t *testing.T) {
|
||||
Name: collectionName,
|
||||
Description: "",
|
||||
AutoID: false,
|
||||
Fields: make([]*schemapb.FieldSchema, Params.MaxFieldNum+1),
|
||||
Fields: make([]*schemapb.FieldSchema, Params.ProxyCfg.MaxFieldNum+1),
|
||||
}
|
||||
marshaledSchemaWithTooManyFields, err := proto.Marshal(schemaWithTooManyFields)
|
||||
assert.NoError(t, err)
|
||||
@ -1022,7 +1022,7 @@ func TestCreateCollectionTask(t *testing.T) {
|
||||
assert.Error(t, err)
|
||||
|
||||
schema.Name = prefix
|
||||
for i := 0; i < int(Params.MaxNameLength); i++ {
|
||||
for i := 0; i < int(Params.ProxyCfg.MaxNameLength); i++ {
|
||||
schema.Name += strconv.Itoa(i % 10)
|
||||
}
|
||||
tooLongNameSchema, err := proto.Marshal(schema)
|
||||
@ -1108,7 +1108,7 @@ func TestCreateCollectionTask(t *testing.T) {
|
||||
schema.Fields[idx].TypeParams = []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "dim",
|
||||
Value: strconv.Itoa(int(Params.MaxDimension) + 1),
|
||||
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension) + 1),
|
||||
},
|
||||
}
|
||||
}
|
||||
@ -1124,7 +1124,7 @@ func TestCreateCollectionTask(t *testing.T) {
|
||||
schema.Fields[1].TypeParams = []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "dim",
|
||||
Value: strconv.Itoa(int(Params.MaxDimension) + 1),
|
||||
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension) + 1),
|
||||
},
|
||||
}
|
||||
binaryTooLargeDimSchema, err := proto.Marshal(schema)
|
||||
@ -1223,7 +1223,7 @@ func TestDropCollectionTask(t *testing.T) {
|
||||
assert.Equal(t, UniqueID(100), task.ID())
|
||||
assert.Equal(t, Timestamp(100), task.BeginTs())
|
||||
assert.Equal(t, Timestamp(100), task.EndTs())
|
||||
assert.Equal(t, Params.ProxyID, task.GetBase().GetSourceID())
|
||||
assert.Equal(t, Params.ProxyCfg.ProxyID, task.GetBase().GetSourceID())
|
||||
// missing collectionID in globalMetaCache
|
||||
err = task.Execute(ctx)
|
||||
assert.NotNil(t, err)
|
||||
@ -1300,7 +1300,7 @@ func TestHasCollectionTask(t *testing.T) {
|
||||
assert.Equal(t, UniqueID(100), task.ID())
|
||||
assert.Equal(t, Timestamp(100), task.BeginTs())
|
||||
assert.Equal(t, Timestamp(100), task.EndTs())
|
||||
assert.Equal(t, Params.ProxyID, task.GetBase().GetSourceID())
|
||||
assert.Equal(t, Params.ProxyCfg.ProxyID, task.GetBase().GetSourceID())
|
||||
// missing collectionID in globalMetaCache
|
||||
err = task.Execute(ctx)
|
||||
assert.Nil(t, err)
|
||||
@ -1361,7 +1361,7 @@ func TestDescribeCollectionTask(t *testing.T) {
|
||||
assert.Equal(t, UniqueID(100), task.ID())
|
||||
assert.Equal(t, Timestamp(100), task.BeginTs())
|
||||
assert.Equal(t, Timestamp(100), task.EndTs())
|
||||
assert.Equal(t, Params.ProxyID, task.GetBase().GetSourceID())
|
||||
assert.Equal(t, Params.ProxyCfg.ProxyID, task.GetBase().GetSourceID())
|
||||
// missing collectionID in globalMetaCache
|
||||
err := task.Execute(ctx)
|
||||
assert.Nil(t, err)
|
||||
@ -1543,7 +1543,7 @@ func TestCreatePartitionTask(t *testing.T) {
|
||||
assert.Equal(t, UniqueID(100), task.ID())
|
||||
assert.Equal(t, Timestamp(100), task.BeginTs())
|
||||
assert.Equal(t, Timestamp(100), task.EndTs())
|
||||
assert.Equal(t, Params.ProxyID, task.GetBase().GetSourceID())
|
||||
assert.Equal(t, Params.ProxyCfg.ProxyID, task.GetBase().GetSourceID())
|
||||
err := task.Execute(ctx)
|
||||
assert.NotNil(t, err)
|
||||
|
||||
@ -1590,7 +1590,7 @@ func TestDropPartitionTask(t *testing.T) {
|
||||
assert.Equal(t, UniqueID(100), task.ID())
|
||||
assert.Equal(t, Timestamp(100), task.BeginTs())
|
||||
assert.Equal(t, Timestamp(100), task.EndTs())
|
||||
assert.Equal(t, Params.ProxyID, task.GetBase().GetSourceID())
|
||||
assert.Equal(t, Params.ProxyCfg.ProxyID, task.GetBase().GetSourceID())
|
||||
err := task.Execute(ctx)
|
||||
assert.NotNil(t, err)
|
||||
|
||||
@ -1637,7 +1637,7 @@ func TestHasPartitionTask(t *testing.T) {
|
||||
assert.Equal(t, UniqueID(100), task.ID())
|
||||
assert.Equal(t, Timestamp(100), task.BeginTs())
|
||||
assert.Equal(t, Timestamp(100), task.EndTs())
|
||||
assert.Equal(t, Params.ProxyID, task.GetBase().GetSourceID())
|
||||
assert.Equal(t, Params.ProxyCfg.ProxyID, task.GetBase().GetSourceID())
|
||||
err := task.Execute(ctx)
|
||||
assert.NotNil(t, err)
|
||||
|
||||
@ -1685,7 +1685,7 @@ func TestShowPartitionsTask(t *testing.T) {
|
||||
assert.Equal(t, UniqueID(100), task.ID())
|
||||
assert.Equal(t, Timestamp(100), task.BeginTs())
|
||||
assert.Equal(t, Timestamp(100), task.EndTs())
|
||||
assert.Equal(t, Params.ProxyID, task.GetBase().GetSourceID())
|
||||
assert.Equal(t, Params.ProxyCfg.ProxyID, task.GetBase().GetSourceID())
|
||||
err := task.Execute(ctx)
|
||||
assert.NotNil(t, err)
|
||||
|
||||
@ -1711,7 +1711,7 @@ func TestSearchTask_all(t *testing.T) {
|
||||
var err error
|
||||
|
||||
Params.Init()
|
||||
Params.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
Params.ProxyCfg.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
|
||||
rc := NewRootCoordMock()
|
||||
rc.Start()
|
||||
@ -1785,7 +1785,7 @@ func TestSearchTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_LoadCollection,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbID: 0,
|
||||
CollectionID: collectionID,
|
||||
@ -1806,9 +1806,9 @@ func TestSearchTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
DbID: 0,
|
||||
CollectionID: 0,
|
||||
PartitionIDs: nil,
|
||||
@ -2057,7 +2057,7 @@ func TestSearchTaskWithInvalidRoundDecimal(t *testing.T) {
|
||||
var err error
|
||||
|
||||
Params.Init()
|
||||
Params.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
Params.ProxyCfg.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
|
||||
rc := NewRootCoordMock()
|
||||
rc.Start()
|
||||
@ -2131,7 +2131,7 @@ func TestSearchTaskWithInvalidRoundDecimal(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_LoadCollection,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbID: 0,
|
||||
CollectionID: collectionID,
|
||||
@ -2152,9 +2152,9 @@ func TestSearchTaskWithInvalidRoundDecimal(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
DbID: 0,
|
||||
CollectionID: 0,
|
||||
PartitionIDs: nil,
|
||||
@ -2401,7 +2401,7 @@ func TestSearchTask_7803_reduce(t *testing.T) {
|
||||
var err error
|
||||
|
||||
Params.Init()
|
||||
Params.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
Params.ProxyCfg.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
|
||||
rc := NewRootCoordMock()
|
||||
rc.Start()
|
||||
@ -2471,7 +2471,7 @@ func TestSearchTask_7803_reduce(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_LoadCollection,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbID: 0,
|
||||
CollectionID: collectionID,
|
||||
@ -2492,9 +2492,9 @@ func TestSearchTask_7803_reduce(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyID, 10),
|
||||
ResultChannelID: strconv.FormatInt(Params.ProxyCfg.ProxyID, 10),
|
||||
DbID: 0,
|
||||
CollectionID: 0,
|
||||
PartitionIDs: nil,
|
||||
@ -2756,7 +2756,7 @@ func TestSearchTask_PreExecute(t *testing.T) {
|
||||
var err error
|
||||
|
||||
Params.Init()
|
||||
Params.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
Params.ProxyCfg.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
|
||||
rc := NewRootCoordMock()
|
||||
rc.Start()
|
||||
@ -3043,7 +3043,7 @@ func TestSearchTask_Execute(t *testing.T) {
|
||||
var err error
|
||||
|
||||
Params.Init()
|
||||
Params.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
Params.ProxyCfg.SearchResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
|
||||
rc := NewRootCoordMock()
|
||||
rc.Start()
|
||||
@ -3188,7 +3188,7 @@ func TestQueryTask_all(t *testing.T) {
|
||||
var err error
|
||||
|
||||
Params.Init()
|
||||
Params.RetrieveResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
Params.ProxyCfg.RetrieveResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
|
||||
rc := NewRootCoordMock()
|
||||
rc.Start()
|
||||
@ -3259,7 +3259,7 @@ func TestQueryTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_LoadCollection,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbID: 0,
|
||||
CollectionID: collectionID,
|
||||
@ -3275,9 +3275,9 @@ func TestQueryTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_Retrieve,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
ResultChannelID: strconv.Itoa(int(Params.ProxyID)),
|
||||
ResultChannelID: strconv.Itoa(int(Params.ProxyCfg.ProxyID)),
|
||||
DbID: 0,
|
||||
CollectionID: collectionID,
|
||||
PartitionIDs: nil,
|
||||
@ -3299,7 +3299,7 @@ func TestQueryTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_Retrieve,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
@ -3351,7 +3351,7 @@ func TestQueryTask_all(t *testing.T) {
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
},
|
||||
ResultChannelID: strconv.Itoa(int(Params.ProxyID)),
|
||||
ResultChannelID: strconv.Itoa(int(Params.ProxyCfg.ProxyID)),
|
||||
Ids: &schemapb.IDs{
|
||||
IdField: &schemapb.IDs_IntId{
|
||||
IntId: &schemapb.LongArray{
|
||||
@ -3501,7 +3501,7 @@ func TestTask_all(t *testing.T) {
|
||||
var err error
|
||||
|
||||
Params.Init()
|
||||
Params.RetrieveResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
Params.ProxyCfg.RetrieveResultChannelNames = []string{funcutil.GenRandomStr()}
|
||||
|
||||
rc := NewRootCoordMock()
|
||||
rc.Start()
|
||||
@ -3560,7 +3560,7 @@ func TestTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_CreatePartition,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
@ -3590,7 +3590,7 @@ func TestTask_all(t *testing.T) {
|
||||
_ = ticker.start()
|
||||
defer ticker.close()
|
||||
|
||||
idAllocator, err := allocator.NewIDAllocator(ctx, rc, Params.ProxyID)
|
||||
idAllocator, err := allocator.NewIDAllocator(ctx, rc, Params.ProxyCfg.ProxyID)
|
||||
assert.NoError(t, err)
|
||||
_ = idAllocator.Start()
|
||||
defer idAllocator.Close()
|
||||
@ -3622,7 +3622,7 @@ func TestTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
@ -3777,7 +3777,7 @@ func TestTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_Delete,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
CollectionName: collectionName,
|
||||
PartitionName: partitionName,
|
||||
@ -3788,7 +3788,7 @@ func TestTask_all(t *testing.T) {
|
||||
MsgType: commonpb.MsgType_Delete,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: Params.ProxyID,
|
||||
SourceID: Params.ProxyCfg.ProxyID,
|
||||
},
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
|
||||
@ -51,9 +51,9 @@ func validateCollectionNameOrAlias(entity, entityType string) error {
|
||||
}
|
||||
|
||||
invalidMsg := fmt.Sprintf("Invalid collection %s: %s. ", entityType, entity)
|
||||
if int64(len(entity)) > Params.MaxNameLength {
|
||||
if int64(len(entity)) > Params.ProxyCfg.MaxNameLength {
|
||||
msg := invalidMsg + fmt.Sprintf("The length of a collection %s must be less than ", entityType) +
|
||||
strconv.FormatInt(Params.MaxNameLength, 10) + " characters."
|
||||
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
@ -91,9 +91,9 @@ func validatePartitionTag(partitionTag string, strictCheck bool) error {
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
if int64(len(partitionTag)) > Params.MaxNameLength {
|
||||
if int64(len(partitionTag)) > Params.ProxyCfg.MaxNameLength {
|
||||
msg := invalidMsg + "The length of a partition tag must be less than " +
|
||||
strconv.FormatInt(Params.MaxNameLength, 10) + " characters."
|
||||
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
@ -125,9 +125,9 @@ func validateFieldName(fieldName string) error {
|
||||
}
|
||||
|
||||
invalidMsg := "Invalid field name: " + fieldName + ". "
|
||||
if int64(len(fieldName)) > Params.MaxNameLength {
|
||||
if int64(len(fieldName)) > Params.ProxyCfg.MaxNameLength {
|
||||
msg := invalidMsg + "The length of a field name must be less than " +
|
||||
strconv.FormatInt(Params.MaxNameLength, 10) + " characters."
|
||||
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
|
||||
return errors.New(msg)
|
||||
}
|
||||
|
||||
@ -149,8 +149,8 @@ func validateFieldName(fieldName string) error {
|
||||
}
|
||||
|
||||
func validateDimension(dim int64, isBinary bool) error {
|
||||
if dim <= 0 || dim > Params.MaxDimension {
|
||||
return fmt.Errorf("invalid dimension: %d. should be in range 1 ~ %d", dim, Params.MaxDimension)
|
||||
if dim <= 0 || dim > Params.ProxyCfg.MaxDimension {
|
||||
return fmt.Errorf("invalid dimension: %d. should be in range 1 ~ %d", dim, Params.ProxyCfg.MaxDimension)
|
||||
}
|
||||
if isBinary && dim%8 != 0 {
|
||||
return fmt.Errorf("invalid dimension: %d. should be multiple of 8. ", dim)
|
||||
|
||||
@ -102,13 +102,13 @@ func TestValidateFieldName(t *testing.T) {
|
||||
|
||||
func TestValidateDimension(t *testing.T) {
|
||||
assert.Nil(t, validateDimension(1, false))
|
||||
assert.Nil(t, validateDimension(Params.MaxDimension, false))
|
||||
assert.Nil(t, validateDimension(Params.ProxyCfg.MaxDimension, false))
|
||||
assert.Nil(t, validateDimension(8, true))
|
||||
assert.Nil(t, validateDimension(Params.MaxDimension, true))
|
||||
assert.Nil(t, validateDimension(Params.ProxyCfg.MaxDimension, true))
|
||||
|
||||
// invalid dim
|
||||
assert.NotNil(t, validateDimension(-1, false))
|
||||
assert.NotNil(t, validateDimension(Params.MaxDimension+1, false))
|
||||
assert.NotNil(t, validateDimension(Params.ProxyCfg.MaxDimension+1, false))
|
||||
assert.NotNil(t, validateDimension(9, true))
|
||||
}
|
||||
|
||||
|
||||
@ -32,10 +32,10 @@ import (
|
||||
func TestShuffleChannelsToQueryNode(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.Address, true)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
||||
clusterSession.Register()
|
||||
meta, err := newMeta(baseCtx, kv, nil, nil)
|
||||
assert.Nil(t, err)
|
||||
|
||||
@ -131,12 +131,12 @@ func newQueryNodeCluster(ctx context.Context, clusterMeta Meta, kv *etcdkv.EtcdK
|
||||
}
|
||||
|
||||
option := &minioKV.Option{
|
||||
Address: Params.MinioEndPoint,
|
||||
AccessKeyID: Params.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinioSecretAccessKey,
|
||||
UseSSL: Params.MinioUseSSLStr,
|
||||
Address: Params.QueryCoordCfg.MinioEndPoint,
|
||||
AccessKeyID: Params.QueryCoordCfg.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.QueryCoordCfg.MinioSecretAccessKey,
|
||||
UseSSL: Params.QueryCoordCfg.MinioUseSSLStr,
|
||||
CreateBucket: true,
|
||||
BucketName: Params.MinioBucketName,
|
||||
BucketName: Params.QueryCoordCfg.MinioBucketName,
|
||||
}
|
||||
|
||||
c.dataKV, err = minioKV.NewMinIOKV(ctx, option)
|
||||
|
||||
@ -335,11 +335,11 @@ func generateIndex(segmentID UniqueID) ([]string, error) {
|
||||
}
|
||||
|
||||
option := &minioKV.Option{
|
||||
Address: Params.MinioEndPoint,
|
||||
AccessKeyID: Params.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinioSecretAccessKey,
|
||||
UseSSL: Params.MinioUseSSLStr,
|
||||
BucketName: Params.MinioBucketName,
|
||||
Address: Params.QueryCoordCfg.MinioEndPoint,
|
||||
AccessKeyID: Params.QueryCoordCfg.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.QueryCoordCfg.MinioSecretAccessKey,
|
||||
UseSSL: Params.QueryCoordCfg.MinioUseSSLStr,
|
||||
BucketName: Params.QueryCoordCfg.MinioBucketName,
|
||||
CreateBucket: true,
|
||||
}
|
||||
|
||||
@ -393,10 +393,10 @@ func TestReloadClusterFromKV(t *testing.T) {
|
||||
t.Run("Test LoadOnlineNodes", func(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx := context.Background()
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.Address, true)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
||||
clusterSession.Register()
|
||||
cluster := &queryNodeCluster{
|
||||
ctx: baseCtx,
|
||||
@ -422,10 +422,10 @@ func TestReloadClusterFromKV(t *testing.T) {
|
||||
|
||||
t.Run("Test LoadOfflineNodes", func(t *testing.T) {
|
||||
refreshParams()
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.Address, true)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
||||
clusterSession.Register()
|
||||
cluster := &queryNodeCluster{
|
||||
client: kv,
|
||||
@ -459,14 +459,14 @@ func TestReloadClusterFromKV(t *testing.T) {
|
||||
func TestGrpcRequest(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.Address, true)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
||||
clusterSession.Register()
|
||||
factory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"PulsarAddress": Params.PulsarAddress,
|
||||
"PulsarAddress": Params.QueryCoordCfg.PulsarAddress,
|
||||
"ReceiveBufSize": 1024,
|
||||
"PulsarBufSize": 1024}
|
||||
err = factory.SetParams(m)
|
||||
@ -650,12 +650,12 @@ func TestEstimateSegmentSize(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
option := &minioKV.Option{
|
||||
Address: Params.MinioEndPoint,
|
||||
AccessKeyID: Params.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinioSecretAccessKey,
|
||||
UseSSL: Params.MinioUseSSLStr,
|
||||
Address: Params.QueryCoordCfg.MinioEndPoint,
|
||||
AccessKeyID: Params.QueryCoordCfg.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.QueryCoordCfg.MinioSecretAccessKey,
|
||||
UseSSL: Params.QueryCoordCfg.MinioUseSSLStr,
|
||||
CreateBucket: true,
|
||||
BucketName: Params.MinioBucketName,
|
||||
BucketName: Params.QueryCoordCfg.MinioBucketName,
|
||||
}
|
||||
|
||||
dataKV, err := minioKV.NewMinIOKV(baseCtx, option)
|
||||
|
||||
@ -71,7 +71,7 @@ func (qc *QueryCoord) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringR
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
},
|
||||
Value: Params.TimeTickChannelName,
|
||||
Value: Params.QueryCoordCfg.TimeTickChannelName,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -83,7 +83,7 @@ func (qc *QueryCoord) GetStatisticsChannel(ctx context.Context) (*milvuspb.Strin
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
},
|
||||
Value: Params.StatsChannelName,
|
||||
Value: Params.QueryCoordCfg.StatsChannelName,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -896,7 +896,7 @@ func (qc *QueryCoord) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRe
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
},
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, Params.QueryCoordID),
|
||||
ComponentName: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, Params.QueryCoordCfg.QueryCoordID),
|
||||
}
|
||||
|
||||
if qc.stateCode.Load() != internalpb.StateCode_Healthy {
|
||||
|
||||
@ -127,7 +127,7 @@ func (ic *IndexChecker) reloadFromKV() error {
|
||||
log.Error("reloadFromKV: unmarshal failed", zap.Any("error", err.Error()))
|
||||
return err
|
||||
}
|
||||
if ic.verifyHandoffReqValid(segmentInfo) && Params.AutoHandoff {
|
||||
if ic.verifyHandoffReqValid(segmentInfo) && Params.QueryCoordCfg.AutoHandoff {
|
||||
// push the req to handoffReqChan and then wait to load after index created
|
||||
// in case handoffReqChan is full, and block start process
|
||||
go ic.enqueueHandoffReq(segmentInfo)
|
||||
@ -196,7 +196,7 @@ func (ic *IndexChecker) checkIndexLoop() {
|
||||
// TODO:: check whether the index exists in parallel, in case indexCoord cannot create the index normally, and then block the loop
|
||||
log.Debug("checkIndexLoop: start check index for handoff segment", zap.Int64("segmentID", segmentInfo.SegmentID))
|
||||
for {
|
||||
if ic.verifyHandoffReqValid(segmentInfo) && Params.AutoHandoff {
|
||||
if ic.verifyHandoffReqValid(segmentInfo) && Params.QueryCoordCfg.AutoHandoff {
|
||||
indexInfo, err := getIndexInfo(ic.ctx, segmentInfo, ic.rootCoord, ic.indexCoord)
|
||||
if err != nil {
|
||||
continue
|
||||
|
||||
@ -34,7 +34,7 @@ import (
|
||||
func TestReloadFromKV(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
meta, err := newMeta(baseCtx, kv, nil, nil)
|
||||
assert.Nil(t, err)
|
||||
@ -88,7 +88,7 @@ func TestReloadFromKV(t *testing.T) {
|
||||
func TestCheckIndexLoop(t *testing.T) {
|
||||
refreshParams()
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
meta, err := newMeta(ctx, kv, nil, nil)
|
||||
assert.Nil(t, err)
|
||||
@ -152,7 +152,7 @@ func TestCheckIndexLoop(t *testing.T) {
|
||||
func TestProcessHandoffAfterIndexDone(t *testing.T) {
|
||||
refreshParams()
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
meta, err := newMeta(ctx, kv, nil, nil)
|
||||
assert.Nil(t, err)
|
||||
@ -162,7 +162,7 @@ func TestProcessHandoffAfterIndexDone(t *testing.T) {
|
||||
client: kv,
|
||||
triggerTaskQueue: NewTaskQueue(),
|
||||
}
|
||||
idAllocatorKV, err := tsoutil.NewTSOKVBase(Params.EtcdEndpoints, Params.KvRootPath, "queryCoordTaskID")
|
||||
idAllocatorKV, err := tsoutil.NewTSOKVBase(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
|
||||
assert.Nil(t, err)
|
||||
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
|
||||
err = idAllocator.Initialize()
|
||||
|
||||
@ -834,8 +834,8 @@ func (m *MetaReplica) createQueryChannel(collectionID UniqueID) *querypb.QueryCh
|
||||
// all collection use the same query channel
|
||||
colIDForAssignChannel := UniqueID(0)
|
||||
|
||||
searchPrefix := Params.SearchChannelPrefix
|
||||
searchResultPrefix := Params.SearchResultChannelPrefix
|
||||
searchPrefix := Params.QueryCoordCfg.SearchChannelPrefix
|
||||
searchResultPrefix := Params.QueryCoordCfg.SearchResultChannelPrefix
|
||||
allocatedQueryChannel := searchPrefix + "-" + strconv.FormatInt(colIDForAssignChannel, 10)
|
||||
allocatedQueryResultChannel := searchResultPrefix + "-" + strconv.FormatInt(colIDForAssignChannel, 10)
|
||||
log.Debug("query coordinator create query channel", zap.String("queryChannelName", allocatedQueryChannel), zap.String("queryResultChannelName", allocatedQueryResultChannel))
|
||||
|
||||
@ -63,7 +63,7 @@ func (tk *testKv) Load(key string) (string, error) {
|
||||
|
||||
func TestReplica_Release(t *testing.T) {
|
||||
refreshParams()
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
meta, err := newMeta(context.Background(), etcdKV, nil, nil)
|
||||
assert.Nil(t, err)
|
||||
@ -93,7 +93,7 @@ func TestReplica_Release(t *testing.T) {
|
||||
|
||||
func TestMetaFunc(t *testing.T) {
|
||||
refreshParams()
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
|
||||
nodeID := defaultQueryNodeID
|
||||
@ -285,7 +285,7 @@ func TestMetaFunc(t *testing.T) {
|
||||
|
||||
func TestReloadMetaFromKV(t *testing.T) {
|
||||
refreshParams()
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
meta := &MetaReplica{
|
||||
client: kv,
|
||||
|
||||
@ -41,7 +41,7 @@ func getSystemInfoMetrics(
|
||||
clusterTopology := metricsinfo.QueryClusterTopology{
|
||||
Self: metricsinfo.QueryCoordInfos{
|
||||
BaseComponentInfos: metricsinfo.BaseComponentInfos{
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, Params.QueryCoordID),
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, Params.QueryCoordCfg.QueryCoordID),
|
||||
HardwareInfos: metricsinfo.HardwareMetrics{
|
||||
IP: qc.session.Address,
|
||||
CPUCoreCount: metricsinfo.GetCPUCoreCount(false),
|
||||
@ -52,14 +52,14 @@ func getSystemInfoMetrics(
|
||||
DiskUsage: metricsinfo.GetDiskUsage(),
|
||||
},
|
||||
SystemInfo: metricsinfo.DeployMetrics{},
|
||||
CreatedTime: Params.CreatedTime.String(),
|
||||
UpdatedTime: Params.UpdatedTime.String(),
|
||||
CreatedTime: Params.QueryCoordCfg.CreatedTime.String(),
|
||||
UpdatedTime: Params.QueryCoordCfg.UpdatedTime.String(),
|
||||
Type: typeutil.QueryCoordRole,
|
||||
ID: qc.session.ServerID,
|
||||
},
|
||||
SystemConfigurations: metricsinfo.QueryCoordConfiguration{
|
||||
SearchChannelPrefix: Params.SearchChannelPrefix,
|
||||
SearchResultChannelPrefix: Params.SearchResultChannelPrefix,
|
||||
SearchChannelPrefix: Params.QueryCoordCfg.SearchChannelPrefix,
|
||||
SearchResultChannelPrefix: Params.QueryCoordCfg.SearchResultChannelPrefix,
|
||||
},
|
||||
},
|
||||
ConnectedNodes: make([]metricsinfo.QueryNodeInfos, 0),
|
||||
@ -119,7 +119,7 @@ func getSystemInfoMetrics(
|
||||
coordTopology := metricsinfo.QueryCoordTopology{
|
||||
Cluster: clusterTopology,
|
||||
Connections: metricsinfo.ConnTopology{
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, Params.QueryCoordID),
|
||||
Name: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, Params.QueryCoordCfg.QueryCoordID),
|
||||
// TODO(dragondriver): fill ConnectedComponents if necessary
|
||||
ConnectedComponents: []metricsinfo.ConnectionInfo{},
|
||||
},
|
||||
|
||||
@ -311,11 +311,11 @@ func newDataCoordMock(ctx context.Context) (*dataCoordMock, error) {
|
||||
|
||||
// create minio client
|
||||
option := &minioKV.Option{
|
||||
Address: Params.MinioEndPoint,
|
||||
AccessKeyID: Params.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinioSecretAccessKey,
|
||||
UseSSL: Params.MinioUseSSLStr,
|
||||
BucketName: Params.MinioBucketName,
|
||||
Address: Params.QueryCoordCfg.MinioEndPoint,
|
||||
AccessKeyID: Params.QueryCoordCfg.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.QueryCoordCfg.MinioSecretAccessKey,
|
||||
UseSSL: Params.QueryCoordCfg.MinioUseSSLStr,
|
||||
BucketName: Params.QueryCoordCfg.MinioBucketName,
|
||||
CreateBucket: true,
|
||||
}
|
||||
kv, err := minioKV.NewMinIOKV(ctx, option)
|
||||
|
||||
@ -100,8 +100,8 @@ func newQueryNodeServerMock(ctx context.Context) *queryNodeServerMock {
|
||||
}
|
||||
|
||||
func (qs *queryNodeServerMock) Register() error {
|
||||
log.Debug("query node session info", zap.String("metaPath", Params.MetaRootPath), zap.Strings("etcdEndPoints", Params.EtcdEndpoints))
|
||||
qs.session = sessionutil.NewSession(qs.ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
log.Debug("query node session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath), zap.Strings("etcdEndPoints", Params.QueryCoordCfg.EtcdEndpoints))
|
||||
qs.session = sessionutil.NewSession(qs.ctx, Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
||||
qs.session.Init(typeutil.QueryNodeRole, qs.queryNodeIP+":"+strconv.FormatInt(qs.queryNodePort, 10), false)
|
||||
qs.queryNodeID = qs.session.ServerID
|
||||
log.Debug("query nodeID", zap.Int64("nodeID", qs.queryNodeID))
|
||||
@ -113,7 +113,7 @@ func (qs *queryNodeServerMock) Register() error {
|
||||
|
||||
func (qs *queryNodeServerMock) init() error {
|
||||
qs.queryNodeIP = funcutil.GetLocalIP()
|
||||
grpcPort := Params.Port
|
||||
grpcPort := Params.QueryCoordCfg.Port
|
||||
|
||||
go func() {
|
||||
var lis net.Listener
|
||||
|
||||
@ -1,334 +0,0 @@
|
||||
// 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 querycoord
|
||||
|
||||
import (
|
||||
"path"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
)
|
||||
|
||||
// UniqueID is an alias for the Int64 type
|
||||
type UniqueID = typeutil.UniqueID
|
||||
|
||||
// ParamTable maintains some environment variables that are required for the QueryCoord runtime
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
|
||||
NodeID uint64
|
||||
|
||||
Address string
|
||||
Port int
|
||||
QueryCoordID UniqueID
|
||||
|
||||
// stats
|
||||
StatsChannelName string
|
||||
|
||||
// timetick
|
||||
TimeTickChannelName string
|
||||
|
||||
// channels
|
||||
ClusterChannelPrefix string
|
||||
SearchChannelPrefix string
|
||||
SearchResultChannelPrefix string
|
||||
|
||||
// --- etcd ---
|
||||
EtcdEndpoints []string
|
||||
MetaRootPath string
|
||||
KvRootPath string
|
||||
|
||||
//--- Minio ---
|
||||
MinioEndPoint string
|
||||
MinioAccessKeyID string
|
||||
MinioSecretAccessKey string
|
||||
MinioUseSSLStr bool
|
||||
MinioBucketName string
|
||||
|
||||
CreatedTime time.Time
|
||||
UpdatedTime time.Time
|
||||
|
||||
DmlChannelPrefix string
|
||||
DeltaChannelPrefix string
|
||||
|
||||
// --- Pulsar ---
|
||||
PulsarAddress string
|
||||
|
||||
//---- Handoff ---
|
||||
AutoHandoff bool
|
||||
|
||||
//---- Balance ---
|
||||
AutoBalance bool
|
||||
OverloadedMemoryThresholdPercentage float64
|
||||
BalanceIntervalSeconds int64
|
||||
MemoryUsageMaxDifferencePercentage float64
|
||||
}
|
||||
|
||||
// Params are variables of the ParamTable type
|
||||
var Params ParamTable
|
||||
var once sync.Once
|
||||
|
||||
// InitOnce guarantees that variables are initialized only once
|
||||
func (p *ParamTable) InitOnce() {
|
||||
once.Do(func() {
|
||||
p.Init()
|
||||
})
|
||||
}
|
||||
|
||||
//Init is used to initialize params
|
||||
func (p *ParamTable) Init() {
|
||||
p.BaseTable.Init()
|
||||
|
||||
p.initRoleName()
|
||||
|
||||
// --- Channels ---
|
||||
p.initClusterMsgChannelPrefix()
|
||||
p.initSearchChannelPrefix()
|
||||
p.initSearchResultChannelPrefix()
|
||||
p.initStatsChannelName()
|
||||
p.initTimeTickChannelName()
|
||||
|
||||
// --- etcd ---
|
||||
p.initEtcdEndpoints()
|
||||
p.initMetaRootPath()
|
||||
p.initKvRootPath()
|
||||
|
||||
//--- Minio ----
|
||||
p.initMinioEndPoint()
|
||||
p.initMinioAccessKeyID()
|
||||
p.initMinioSecretAccessKey()
|
||||
p.initMinioUseSSLStr()
|
||||
p.initMinioBucketName()
|
||||
|
||||
//--- Pulsar ----
|
||||
p.initPulsarAddress()
|
||||
|
||||
//---- Handoff ---
|
||||
p.initAutoHandoff()
|
||||
|
||||
p.initDmlChannelName()
|
||||
p.initDeltaChannelName()
|
||||
|
||||
//---- Balance ---
|
||||
p.initAutoBalance()
|
||||
p.initOverloadedMemoryThresholdPercentage()
|
||||
p.initBalanceIntervalSeconds()
|
||||
p.initMemoryUsageMaxDifferencePercentage()
|
||||
}
|
||||
|
||||
func (p *ParamTable) initClusterMsgChannelPrefix() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.cluster")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.ClusterChannelPrefix = config
|
||||
}
|
||||
|
||||
func (p *ParamTable) initSearchChannelPrefix() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.search")
|
||||
if err != nil {
|
||||
log.Error(err.Error())
|
||||
}
|
||||
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.SearchChannelPrefix = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initSearchResultChannelPrefix() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.searchResult")
|
||||
if err != nil {
|
||||
log.Error(err.Error())
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.SearchResultChannelPrefix = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initStatsChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.queryNodeStats")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.StatsChannelName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initTimeTickChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.queryTimeTick")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.TimeTickChannelName = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initEtcdEndpoints() {
|
||||
endpoints, err := p.Load("_EtcdEndpoints")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.EtcdEndpoints = strings.Split(endpoints, ",")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMetaRootPath() {
|
||||
rootPath, err := p.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := p.Load("etcd.metaSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MetaRootPath = path.Join(rootPath, subPath)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initKvRootPath() {
|
||||
rootPath, err := p.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
subPath, err := p.Load("etcd.kvSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.KvRootPath = path.Join(rootPath, subPath)
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioEndPoint() {
|
||||
url, err := p.Load("_MinioAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioEndPoint = url
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioAccessKeyID() {
|
||||
id, err := p.Load("minio.accessKeyID")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioAccessKeyID = id
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioSecretAccessKey() {
|
||||
key, err := p.Load("minio.secretAccessKey")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioSecretAccessKey = key
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioUseSSLStr() {
|
||||
ssl, err := p.Load("minio.useSSL")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
sslBoolean, err := strconv.ParseBool(ssl)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioUseSSLStr = sslBoolean
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMinioBucketName() {
|
||||
bucketName, err := p.Load("minio.bucketName")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MinioBucketName = bucketName
|
||||
}
|
||||
|
||||
func (p *ParamTable) initRoleName() {
|
||||
p.RoleName = "querycoord"
|
||||
}
|
||||
|
||||
func (p *ParamTable) initPulsarAddress() {
|
||||
addr, err := p.Load("_PulsarAddress")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.PulsarAddress = addr
|
||||
}
|
||||
|
||||
func (p *ParamTable) initAutoHandoff() {
|
||||
handoff, err := p.Load("queryCoord.autoHandoff")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.AutoHandoff, err = strconv.ParseBool(handoff)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (p *ParamTable) initAutoBalance() {
|
||||
balanceStr := p.LoadWithDefault("queryCoord.autoBalance", "false")
|
||||
autoBalance, err := strconv.ParseBool(balanceStr)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.AutoBalance = autoBalance
|
||||
}
|
||||
|
||||
func (p *ParamTable) initOverloadedMemoryThresholdPercentage() {
|
||||
overloadedMemoryThresholdPercentage := p.LoadWithDefault("queryCoord.overloadedMemoryThresholdPercentage", "90")
|
||||
thresholdPercentage, err := strconv.ParseInt(overloadedMemoryThresholdPercentage, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.OverloadedMemoryThresholdPercentage = float64(thresholdPercentage) / 100
|
||||
}
|
||||
|
||||
func (p *ParamTable) initBalanceIntervalSeconds() {
|
||||
balanceInterval := p.LoadWithDefault("queryCoord.balanceIntervalSeconds", "60")
|
||||
interval, err := strconv.ParseInt(balanceInterval, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.BalanceIntervalSeconds = interval
|
||||
}
|
||||
|
||||
func (p *ParamTable) initMemoryUsageMaxDifferencePercentage() {
|
||||
maxDiff := p.LoadWithDefault("queryCoord.memoryUsageMaxDifferencePercentage", "30")
|
||||
diffPercentage, err := strconv.ParseInt(maxDiff, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.MemoryUsageMaxDifferencePercentage = float64(diffPercentage) / 100
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDmlChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.rootCoordDml")
|
||||
if err != nil {
|
||||
config = "rootcoord-dml"
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.DmlChannelPrefix = strings.Join(s, "-")
|
||||
}
|
||||
|
||||
func (p *ParamTable) initDeltaChannelName() {
|
||||
config, err := p.Load("msgChannel.chanNamePrefix.rootCoordDelta")
|
||||
if err != nil {
|
||||
config = "rootcoord-delta"
|
||||
}
|
||||
s := []string{p.ClusterChannelPrefix, config}
|
||||
p.DeltaChannelPrefix = strings.Join(s, "-")
|
||||
}
|
||||
@ -1,40 +0,0 @@
|
||||
// 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 querycoord
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
//TODO add more test for other parameters
|
||||
func TestParamTable(t *testing.T) {
|
||||
Params.Init()
|
||||
|
||||
assert.Equal(t, Params.SearchChannelPrefix, "by-dev-search")
|
||||
t.Logf("query coord search channel = %s", Params.SearchChannelPrefix)
|
||||
|
||||
assert.Equal(t, Params.SearchResultChannelPrefix, "by-dev-searchResult")
|
||||
t.Logf("query coord search result channel = %s", Params.SearchResultChannelPrefix)
|
||||
|
||||
assert.Equal(t, Params.StatsChannelName, "by-dev-query-node-stats")
|
||||
t.Logf("query coord stats channel = %s", Params.StatsChannelName)
|
||||
|
||||
assert.Equal(t, Params.TimeTickChannelName, "by-dev-queryTimeTick")
|
||||
t.Logf("query coord time tick channel = %s", Params.TimeTickChannelName)
|
||||
}
|
||||
@ -30,9 +30,6 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
@ -42,16 +39,22 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/paramtable"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
const (
|
||||
handoffSegmentPrefix = "querycoord-handoff"
|
||||
)
|
||||
|
||||
// UniqueID is an alias for the Int64 type
|
||||
type UniqueID = typeutil.UniqueID
|
||||
|
||||
// Timestamp is an alias for the Int64 type
|
||||
type Timestamp = typeutil.Timestamp
|
||||
|
||||
@ -60,6 +63,8 @@ type queryChannelInfo struct {
|
||||
responseChannel string
|
||||
}
|
||||
|
||||
var Params paramtable.GlobalParamTable
|
||||
|
||||
// QueryCoord is the coordinator of queryNodes
|
||||
type QueryCoord struct {
|
||||
loopCtx context.Context
|
||||
@ -107,23 +112,23 @@ func (qc *QueryCoord) Register() error {
|
||||
}
|
||||
|
||||
func (qc *QueryCoord) initSession() error {
|
||||
qc.session = sessionutil.NewSession(qc.loopCtx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
qc.session = sessionutil.NewSession(qc.loopCtx, Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
||||
if qc.session == nil {
|
||||
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
||||
}
|
||||
qc.session.Init(typeutil.QueryCoordRole, Params.Address, true)
|
||||
Params.NodeID = uint64(qc.session.ServerID)
|
||||
Params.SetLogger(qc.session.ServerID)
|
||||
qc.session.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
||||
Params.QueryCoordCfg.NodeID = uint64(qc.session.ServerID)
|
||||
Params.BaseParams.SetLogger(qc.session.ServerID)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler
|
||||
func (qc *QueryCoord) Init() error {
|
||||
log.Debug("query coordinator start init, session info", zap.String("metaPath", Params.MetaRootPath),
|
||||
zap.Strings("etcdEndPoints", Params.EtcdEndpoints), zap.String("address", Params.Address))
|
||||
log.Debug("query coordinator start init, session info", zap.String("metaPath", Params.QueryCoordCfg.MetaRootPath),
|
||||
zap.Strings("etcdEndPoints", Params.QueryCoordCfg.EtcdEndpoints), zap.String("address", Params.QueryCoordCfg.Address))
|
||||
//connect etcd
|
||||
connectEtcdFn := func() error {
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
etcdKV, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -148,7 +153,7 @@ func (qc *QueryCoord) Init() error {
|
||||
|
||||
// init id allocator
|
||||
var idAllocatorKV *etcdkv.EtcdKV
|
||||
idAllocatorKV, initError = tsoutil.NewTSOKVBase(Params.EtcdEndpoints, Params.KvRootPath, "queryCoordTaskID")
|
||||
idAllocatorKV, initError = tsoutil.NewTSOKVBase(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.KvRootPath, "queryCoordTaskID")
|
||||
if initError != nil {
|
||||
log.Debug("query coordinator idAllocatorKV initialize failed", zap.Error(initError))
|
||||
return
|
||||
@ -200,7 +205,7 @@ func (qc *QueryCoord) Init() error {
|
||||
// Start function starts the goroutines to watch the meta and node updates
|
||||
func (qc *QueryCoord) Start() error {
|
||||
m := map[string]interface{}{
|
||||
"PulsarAddress": Params.PulsarAddress,
|
||||
"PulsarAddress": Params.QueryCoordCfg.PulsarAddress,
|
||||
"ReceiveBufSize": 1024,
|
||||
"PulsarBufSize": 1024}
|
||||
err := qc.msFactory.SetParams(m)
|
||||
@ -213,8 +218,8 @@ func (qc *QueryCoord) Start() error {
|
||||
qc.indexChecker.start()
|
||||
log.Debug("start index checker ...")
|
||||
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
Params.QueryCoordCfg.CreatedTime = time.Now()
|
||||
Params.QueryCoordCfg.UpdatedTime = time.Now()
|
||||
|
||||
qc.loopWg.Add(1)
|
||||
go qc.watchNodeLoop()
|
||||
@ -222,7 +227,7 @@ func (qc *QueryCoord) Start() error {
|
||||
qc.loopWg.Add(1)
|
||||
go qc.watchHandoffSegmentLoop()
|
||||
|
||||
if Params.AutoBalance {
|
||||
if Params.QueryCoordCfg.AutoBalance {
|
||||
qc.loopWg.Add(1)
|
||||
go qc.loadBalanceSegmentLoop()
|
||||
}
|
||||
@ -257,8 +262,8 @@ func NewQueryCoord(ctx context.Context, factory msgstream.Factory) (*QueryCoord,
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
queryChannels := make([]*queryChannelInfo, 0)
|
||||
channelID := len(queryChannels)
|
||||
searchPrefix := Params.SearchChannelPrefix
|
||||
searchResultPrefix := Params.SearchResultChannelPrefix
|
||||
searchPrefix := Params.QueryCoordCfg.SearchChannelPrefix
|
||||
searchResultPrefix := Params.QueryCoordCfg.SearchResultChannelPrefix
|
||||
allocatedQueryChannel := searchPrefix + "-" + strconv.FormatInt(int64(channelID), 10)
|
||||
allocatedQueryResultChannel := searchResultPrefix + "-" + strconv.FormatInt(int64(channelID), 10)
|
||||
|
||||
@ -419,7 +424,7 @@ func (qc *QueryCoord) watchHandoffSegmentLoop() {
|
||||
}
|
||||
switch event.Type {
|
||||
case mvccpb.PUT:
|
||||
if Params.AutoHandoff && qc.indexChecker.verifyHandoffReqValid(segmentInfo) {
|
||||
if Params.QueryCoordCfg.AutoHandoff && qc.indexChecker.verifyHandoffReqValid(segmentInfo) {
|
||||
qc.indexChecker.enqueueHandoffReq(segmentInfo)
|
||||
log.Debug("watchHandoffSegmentLoop: enqueue a handoff request to index checker", zap.Any("segment info", segmentInfo))
|
||||
} else {
|
||||
@ -445,7 +450,7 @@ func (qc *QueryCoord) loadBalanceSegmentLoop() {
|
||||
defer qc.loopWg.Done()
|
||||
log.Debug("QueryCoord start load balance segment loop")
|
||||
|
||||
timer := time.NewTicker(time.Duration(Params.BalanceIntervalSeconds) * time.Second)
|
||||
timer := time.NewTicker(time.Duration(Params.QueryCoordCfg.BalanceIntervalSeconds) * time.Second)
|
||||
|
||||
for {
|
||||
select {
|
||||
@ -510,8 +515,8 @@ func (qc *QueryCoord) loadBalanceSegmentLoop() {
|
||||
memUsageRateDiff := nodeID2MemUsageRate[sourceNodeID] - nodeID2MemUsageRate[dstNodeID]
|
||||
// if memoryUsageRate of source node is greater than 90%, and the max memUsageDiff is greater than 30%
|
||||
// then migrate the segments on source node to other query nodes
|
||||
if nodeID2MemUsageRate[sourceNodeID] > Params.OverloadedMemoryThresholdPercentage ||
|
||||
memUsageRateDiff > Params.MemoryUsageMaxDifferencePercentage {
|
||||
if nodeID2MemUsageRate[sourceNodeID] > Params.QueryCoordCfg.OverloadedMemoryThresholdPercentage ||
|
||||
memUsageRateDiff > Params.QueryCoordCfg.MemoryUsageMaxDifferencePercentage {
|
||||
segmentInfos := nodeID2SegmentInfos[sourceNodeID]
|
||||
// select the segment that needs balance on the source node
|
||||
selectedSegmentInfo, err := chooseSegmentToBalance(sourceNodeID, dstNodeID, segmentInfos, nodeID2MemUsage, nodeID2TotalMem, nodeID2MemUsageRate)
|
||||
@ -594,7 +599,7 @@ func chooseSegmentToBalance(sourceNodeID int64, dstNodeID int64,
|
||||
dstNodeMemUsageAfterBalance := nodeID2MemUsage[dstNodeID] + uint64(info.MemSize)
|
||||
dstNodeMemUsageRateAfterBalance := float64(dstNodeMemUsageAfterBalance) / float64(nodeID2TotalMem[dstNodeID])
|
||||
// if memUsageRate of dstNode is greater than OverloadedMemoryThresholdPercentage after balance, than can't balance
|
||||
if dstNodeMemUsageRateAfterBalance < Params.OverloadedMemoryThresholdPercentage {
|
||||
if dstNodeMemUsageRateAfterBalance < Params.QueryCoordCfg.OverloadedMemoryThresholdPercentage {
|
||||
memoryInsufficient = false
|
||||
sourceNodeMemUsageAfterBalance := nodeID2MemUsage[sourceNodeID] - uint64(info.MemSize)
|
||||
sourceNodeMemUsageRateAfterBalance := float64(sourceNodeMemUsageAfterBalance) / float64(nodeID2TotalMem[sourceNodeID])
|
||||
|
||||
@ -45,11 +45,11 @@ func setup() {
|
||||
func refreshParams() {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
suffix := "-test-query-Coord" + strconv.FormatInt(rand.Int63(), 10)
|
||||
Params.StatsChannelName = Params.StatsChannelName + suffix
|
||||
Params.TimeTickChannelName = Params.TimeTickChannelName + suffix
|
||||
Params.MetaRootPath = Params.MetaRootPath + suffix
|
||||
Params.DmlChannelPrefix = "Dml"
|
||||
Params.DeltaChannelPrefix = "delta"
|
||||
Params.QueryCoordCfg.StatsChannelName = Params.QueryCoordCfg.StatsChannelName + suffix
|
||||
Params.QueryCoordCfg.TimeTickChannelName = Params.QueryCoordCfg.TimeTickChannelName + suffix
|
||||
Params.QueryCoordCfg.MetaRootPath = Params.QueryCoordCfg.MetaRootPath + suffix
|
||||
Params.QueryCoordCfg.DmlChannelPrefix = "Dml"
|
||||
Params.QueryCoordCfg.DeltaChannelPrefix = "delta"
|
||||
GlobalSegmentInfos = make(map[UniqueID]*querypb.SegmentInfo)
|
||||
}
|
||||
|
||||
@ -150,7 +150,7 @@ func TestWatchNodeLoop(t *testing.T) {
|
||||
|
||||
t.Run("Test OfflineNodes", func(t *testing.T) {
|
||||
refreshParams()
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
|
||||
kvs := make(map[string]string)
|
||||
@ -514,7 +514,7 @@ func TestHandoffSegmentLoop(t *testing.T) {
|
||||
|
||||
func TestLoadBalanceSegmentLoop(t *testing.T) {
|
||||
refreshParams()
|
||||
Params.BalanceIntervalSeconds = 10
|
||||
Params.QueryCoordCfg.BalanceIntervalSeconds = 10
|
||||
baseCtx := context.Background()
|
||||
|
||||
queryCoord, err := startQueryCoord(baseCtx)
|
||||
@ -555,7 +555,7 @@ func TestLoadBalanceSegmentLoop(t *testing.T) {
|
||||
waitTaskFinalState(loadPartitionTask, taskExpired)
|
||||
nodeInfo, err := queryCoord.cluster.getNodeInfoByID(queryNode1.queryNodeID)
|
||||
assert.Nil(t, err)
|
||||
if nodeInfo.(*queryNode).memUsageRate >= Params.OverloadedMemoryThresholdPercentage {
|
||||
if nodeInfo.(*queryNode).memUsageRate >= Params.QueryCoordCfg.OverloadedMemoryThresholdPercentage {
|
||||
break
|
||||
}
|
||||
partitionID++
|
||||
|
||||
@ -36,7 +36,7 @@ import (
|
||||
//func waitQueryNodeOnline(cluster *queryNodeCluster, nodeID int64)
|
||||
|
||||
func removeNodeSession(id int64) error {
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -48,7 +48,7 @@ func removeNodeSession(id int64) error {
|
||||
}
|
||||
|
||||
func removeAllSession() error {
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -187,7 +187,7 @@ func TestQueryNode_getMetrics(t *testing.T) {
|
||||
func TestNewQueryNode(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
|
||||
queryNode1, err := startQueryNodeServer(baseCtx)
|
||||
@ -211,7 +211,7 @@ func TestNewQueryNode(t *testing.T) {
|
||||
func TestReleaseCollectionOnOfflineNode(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
|
||||
node, err := newQueryNode(baseCtx, "test", 100, kv)
|
||||
@ -280,7 +280,7 @@ func TestSealedSegmentChangeAfterQueryNodeStop(t *testing.T) {
|
||||
func TestGrpcRequestWithNodeOffline(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
nodeServer, err := startQueryNodeServer(baseCtx)
|
||||
assert.Nil(t, err)
|
||||
|
||||
@ -160,7 +160,7 @@ func shuffleSegmentsToQueryNodeV2(ctx context.Context, reqs []*querypb.LoadSegme
|
||||
}
|
||||
queryNodeInfo := nodeInfo.(*queryNode)
|
||||
// avoid allocate segment to node which memUsageRate is high
|
||||
if queryNodeInfo.memUsageRate >= Params.OverloadedMemoryThresholdPercentage {
|
||||
if queryNodeInfo.memUsageRate >= Params.QueryCoordCfg.OverloadedMemoryThresholdPercentage {
|
||||
log.Debug("shuffleSegmentsToQueryNodeV2: queryNode memUsageRate large than MaxMemUsagePerNode", zap.Int64("nodeID", nodeID), zap.Float64("current rate", queryNodeInfo.memUsageRate))
|
||||
continue
|
||||
}
|
||||
@ -182,7 +182,7 @@ func shuffleSegmentsToQueryNodeV2(ctx context.Context, reqs []*querypb.LoadSegme
|
||||
for _, nodeID := range availableNodeIDs {
|
||||
memUsageAfterLoad := memUsage[nodeID] + uint64(sizeOfReq)
|
||||
memUsageRateAfterLoad := float64(memUsageAfterLoad) / float64(totalMem[nodeID])
|
||||
if memUsageRateAfterLoad > Params.OverloadedMemoryThresholdPercentage {
|
||||
if memUsageRateAfterLoad > Params.QueryCoordCfg.OverloadedMemoryThresholdPercentage {
|
||||
continue
|
||||
}
|
||||
reqs[offset].DstNodeID = nodeID
|
||||
|
||||
@ -32,10 +32,10 @@ import (
|
||||
func TestShuffleSegmentsToQueryNode(t *testing.T) {
|
||||
refreshParams()
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.Address, true)
|
||||
clusterSession := sessionutil.NewSession(context.Background(), Params.QueryCoordCfg.MetaRootPath, Params.QueryCoordCfg.EtcdEndpoints)
|
||||
clusterSession.Init(typeutil.QueryCoordRole, Params.QueryCoordCfg.Address, true)
|
||||
meta, err := newMeta(baseCtx, kv, nil, nil)
|
||||
assert.Nil(t, err)
|
||||
cluster := &queryNodeCluster{
|
||||
@ -50,12 +50,12 @@ func TestShuffleSegmentsToQueryNode(t *testing.T) {
|
||||
}
|
||||
|
||||
option := &minioKV.Option{
|
||||
Address: Params.MinioEndPoint,
|
||||
AccessKeyID: Params.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.MinioSecretAccessKey,
|
||||
UseSSL: Params.MinioUseSSLStr,
|
||||
Address: Params.QueryCoordCfg.MinioEndPoint,
|
||||
AccessKeyID: Params.QueryCoordCfg.MinioAccessKeyID,
|
||||
SecretAccessKeyID: Params.QueryCoordCfg.MinioSecretAccessKey,
|
||||
UseSSL: Params.QueryCoordCfg.MinioUseSSLStr,
|
||||
CreateBucket: true,
|
||||
BucketName: Params.MinioBucketName,
|
||||
BucketName: Params.QueryCoordCfg.MinioBucketName,
|
||||
}
|
||||
|
||||
cluster.dataKV, err = minioKV.NewMinIOKV(baseCtx, option)
|
||||
|
||||
@ -2113,7 +2113,7 @@ func getSizeOfLoadSegmentReq(req *querypb.LoadSegmentsRequest) int {
|
||||
}
|
||||
|
||||
func generateWatchDeltaChannelInfo(info *datapb.VchannelInfo) (*datapb.VchannelInfo, error) {
|
||||
deltaChannelName, err := rootcoord.ConvertChannelName(info.ChannelName, Params.DmlChannelPrefix, Params.DeltaChannelPrefix)
|
||||
deltaChannelName, err := rootcoord.ConvertChannelName(info.ChannelName, Params.QueryCoordCfg.DmlChannelPrefix, Params.QueryCoordCfg.DeltaChannelPrefix)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@ -208,7 +208,7 @@ func TestWatchQueryChannel_ClearEtcdInfoAfterAssignedNodeDown(t *testing.T) {
|
||||
|
||||
func TestUnMarshalTask(t *testing.T) {
|
||||
refreshParams()
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
taskScheduler := &TaskScheduler{
|
||||
@ -454,7 +454,7 @@ func TestUnMarshalTask(t *testing.T) {
|
||||
|
||||
func TestReloadTaskFromKV(t *testing.T) {
|
||||
refreshParams()
|
||||
kv, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
||||
kv, err := etcdkv.NewEtcdKV(Params.QueryCoordCfg.EtcdEndpoints, Params.QueryCoordCfg.MetaRootPath)
|
||||
assert.Nil(t, err)
|
||||
baseCtx, cancel := context.WithCancel(context.Background())
|
||||
taskScheduler := &TaskScheduler{
|
||||
|
||||
@ -710,7 +710,7 @@ func getSegmentInfo(segment *Segment) *querypb.SegmentInfo {
|
||||
SegmentID: segment.ID(),
|
||||
CollectionID: segment.collectionID,
|
||||
PartitionID: segment.partitionID,
|
||||
NodeID: Params.QueryNodeID,
|
||||
NodeID: Params.QueryNodeCfg.QueryNodeID,
|
||||
MemSize: segment.getMemSize(),
|
||||
NumRows: segment.getRowCount(),
|
||||
IndexName: indexName,
|
||||
|
||||
@ -143,8 +143,8 @@ func (dNode *deleteNode) delete(deleteData *deleteData, segmentID UniqueID, wg *
|
||||
|
||||
// newDeleteNode returns a new deleteNode
|
||||
func newDeleteNode(historicalReplica ReplicaInterface) *deleteNode {
|
||||
maxQueueLength := Params.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.FlowGraphMaxParallelism
|
||||
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism
|
||||
|
||||
baseNode := baseNode{}
|
||||
baseNode.SetMaxQueueLength(maxQueueLength)
|
||||
|
||||
@ -115,8 +115,8 @@ func (fddNode *filterDeleteNode) filterInvalidDeleteMessage(msg *msgstream.Delet
|
||||
// newFilteredDeleteNode returns a new filterDeleteNode
|
||||
func newFilteredDeleteNode(replica ReplicaInterface, collectionID UniqueID) *filterDeleteNode {
|
||||
|
||||
maxQueueLength := Params.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.FlowGraphMaxParallelism
|
||||
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism
|
||||
|
||||
baseNode := baseNode{}
|
||||
baseNode.SetMaxQueueLength(maxQueueLength)
|
||||
|
||||
@ -236,8 +236,8 @@ func (fdmNode *filterDmNode) filterInvalidInsertMessage(msg *msgstream.InsertMsg
|
||||
// newFilteredDmNode returns a new filterDmNode
|
||||
func newFilteredDmNode(replica ReplicaInterface, collectionID UniqueID) *filterDmNode {
|
||||
|
||||
maxQueueLength := Params.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.FlowGraphMaxParallelism
|
||||
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism
|
||||
|
||||
baseNode := baseNode{}
|
||||
baseNode.SetMaxQueueLength(maxQueueLength)
|
||||
|
||||
@ -410,8 +410,8 @@ func getPrimaryKeys(msg *msgstream.InsertMsg, streamingReplica ReplicaInterface)
|
||||
|
||||
// newInsertNode returns a new insertNode
|
||||
func newInsertNode(streamingReplica ReplicaInterface) *insertNode {
|
||||
maxQueueLength := Params.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.FlowGraphMaxParallelism
|
||||
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism
|
||||
|
||||
baseNode := baseNode{}
|
||||
baseNode.SetMaxQueueLength(maxQueueLength)
|
||||
|
||||
@ -182,8 +182,8 @@ func (q *queryNodeFlowGraph) newDmInputNode(ctx context.Context, factory msgstre
|
||||
q.dmlStream = insertStream
|
||||
}
|
||||
|
||||
maxQueueLength := Params.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.FlowGraphMaxParallelism
|
||||
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength
|
||||
maxParallelism := Params.QueryNodeCfg.FlowGraphMaxParallelism
|
||||
|
||||
node := flowgraph.NewInputNode(insertStream, "dmlInputNode", maxQueueLength, maxParallelism)
|
||||
return node
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user