Use GlobalParamTable for all components (#13885)

Signed-off-by: yudong.cai <yudong.cai@zilliz.com>
This commit is contained in:
Cai Yudong 2021-12-23 18:39:11 +08:00 committed by GitHub
parent 6c3f1a9162
commit 891c202b73
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
139 changed files with 3442 additions and 4078 deletions

View File

@ -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 {

View File

@ -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,
})

View File

@ -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) {

View File

@ -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,

View File

@ -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()

View File

@ -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
}

View File

@ -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,

View File

@ -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

View File

@ -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,
},
}

View File

@ -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)
}

View File

@ -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)
}

View File

@ -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
}

View File

@ -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,
},
}

View File

@ -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
}

View File

@ -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,

View File

@ -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)

View File

@ -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
}

View File

@ -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
}

View File

@ -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

View File

@ -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,
})

View File

@ -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)

View File

@ -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++ {

View File

@ -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))

View File

@ -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)

View File

@ -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,
}

View File

@ -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,

View File

@ -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

View File

@ -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,

View File

@ -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

View File

@ -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()

View File

@ -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

View File

@ -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,

View File

@ -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,

View File

@ -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
}

View File

@ -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
}

View File

@ -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, "-")
}

View File

@ -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)
})
}

View File

@ -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)

View File

@ -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)

View File

@ -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 {

View File

@ -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 {

View File

@ -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)

View File

@ -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))

View File

@ -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() {

View File

@ -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
}

View File

@ -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)

View File

@ -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)

View File

@ -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

View File

@ -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)

View File

@ -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)
}

View File

@ -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()

View File

@ -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))

View File

@ -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
}

View File

@ -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) {

View File

@ -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),

View File

@ -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"
}

View File

@ -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()
// })
// })
//}

View File

@ -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))

View File

@ -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
}

View File

@ -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
}

View File

@ -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))
}

View File

@ -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()
// })
// })
//}

View File

@ -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 {

View File

@ -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,

View File

@ -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
}

View File

@ -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
}

View File

@ -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()
})
}

View File

@ -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()))

View File

@ -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)
})

View File

@ -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

View File

@ -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 {

View File

@ -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,

View File

@ -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)

View File

@ -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))
}

View File

@ -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)

View File

@ -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)

View File

@ -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)

View File

@ -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 {

View File

@ -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

View File

@ -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()

View File

@ -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))

View File

@ -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,

View File

@ -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{},
},

View File

@ -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)

View File

@ -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

View File

@ -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, "-")
}

View File

@ -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)
}

View File

@ -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])

View File

@ -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++

View File

@ -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)

View File

@ -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

View File

@ -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)

View File

@ -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
}

View File

@ -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{

View File

@ -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,

View File

@ -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)

View File

@ -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)

View File

@ -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)

View File

@ -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)

View File

@ -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