Add event dispatcher for config (#20393)

Signed-off-by: Enwei Jiao <enwei.jiao@zilliz.com>

Signed-off-by: Enwei Jiao <enwei.jiao@zilliz.com>
This commit is contained in:
Enwei Jiao 2022-11-17 18:59:09 +08:00 committed by GitHub
parent cc371d6801
commit c05b9ad539
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
99 changed files with 1111 additions and 824 deletions

View File

@ -226,7 +226,7 @@ func (c *mck) connectEctd() {
log.Fatal("failed to connect to etcd", zap.Error(err)) log.Fatal("failed to connect to etcd", zap.Error(err))
} }
rootPath := getConfigValue(c.ectdRootPath, c.params.EtcdCfg.MetaRootPath, "ectd_root_path") rootPath := getConfigValue(c.ectdRootPath, c.params.EtcdCfg.MetaRootPath.GetValue(), "ectd_root_path")
c.etcdKV = etcdkv.NewEtcdKV(etcdCli, rootPath) c.etcdKV = etcdkv.NewEtcdKV(etcdCli, rootPath)
log.Info("Etcd root path", zap.String("root_path", rootPath)) log.Info("Etcd root path", zap.String("root_path", rootPath))
} }

View File

@ -221,7 +221,7 @@ func (mr *MilvusRoles) Run(local bool, alias string) {
defer stopRocksmq() defer stopRocksmq()
} }
if params.EtcdCfg.UseEmbedEtcd { if params.EtcdCfg.UseEmbedEtcd.GetAsBool() {
// Start etcd server. // Start etcd server.
etcd.InitEtcdServer(&params.EtcdCfg) etcd.InitEtcdServer(&params.EtcdCfg)
defer etcd.StopEtcdServer() defer etcd.StopEtcdServer()

View File

@ -23,7 +23,7 @@ func newEtcdBasedBackend(cfg *configs.MilvusConfig) (*etcdBasedBackend, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
txn := etcdkv.NewEtcdKV(etcdCli, cfg.EtcdCfg.MetaRootPath) txn := etcdkv.NewEtcdKV(etcdCli, cfg.EtcdCfg.MetaRootPath.GetValue())
b := &etcdBasedBackend{cfg: cfg, etcdCli: etcdCli, txn: txn} b := &etcdBasedBackend{cfg: cfg, etcdCli: etcdCli, txn: txn}
return b, nil return b, nil
} }

View File

@ -413,7 +413,7 @@ func (b etcd210) Backup(meta *meta.Meta, backupFile string) error {
saves := meta.Meta210.GenerateSaves() saves := meta.Meta210.GenerateSaves()
codec := NewBackupCodec() codec := NewBackupCodec()
var instance, metaPath string var instance, metaPath string
metaRootPath := b.cfg.EtcdCfg.MetaRootPath metaRootPath := b.cfg.EtcdCfg.MetaRootPath.GetValue()
parts := strings.Split(metaRootPath, "/") parts := strings.Split(metaRootPath, "/")
if len(parts) > 1 { if len(parts) > 1 {
metaPath = parts[len(parts)-1] metaPath = parts[len(parts)-1]
@ -439,7 +439,7 @@ func (b etcd210) Backup(meta *meta.Meta, backupFile string) error {
func (b etcd210) BackupV2(file string) error { func (b etcd210) BackupV2(file string) error {
var instance, metaPath string var instance, metaPath string
metaRootPath := b.cfg.EtcdCfg.MetaRootPath metaRootPath := b.cfg.EtcdCfg.MetaRootPath.GetValue()
parts := strings.Split(metaRootPath, "/") parts := strings.Split(metaRootPath, "/")
if len(parts) > 1 { if len(parts) > 1 {
metaPath = parts[len(parts)-1] metaPath = parts[len(parts)-1]

View File

@ -89,8 +89,7 @@ func (c *MilvusConfig) init(base *paramtable.BaseTable) {
default: default:
} }
c.EtcdCfg.Base = base c.EtcdCfg.Init(base)
c.EtcdCfg.LoadCfgToMemory()
} }
func (c *MilvusConfig) String() string { func (c *MilvusConfig) String() string {

View File

@ -81,7 +81,7 @@ func (r *Runner) initEtcdCli() {
func (r *Runner) init() { func (r *Runner) init() {
r.initEtcdCli() r.initEtcdCli()
r.session = sessionutil.NewSession(r.ctx, r.cfg.EtcdCfg.MetaRootPath, r.etcdCli, r.session = sessionutil.NewSession(r.ctx, r.cfg.EtcdCfg.MetaRootPath.GetValue(), r.etcdCli,
sessionutil.WithCustomConfigEnable(), sessionutil.WithTTL(60), sessionutil.WithRetryTimes(30)) sessionutil.WithCustomConfigEnable(), sessionutil.WithTTL(60), sessionutil.WithRetryTimes(30))
// address not important here. // address not important here.
address := time.Now().String() address := time.Now().String()

View File

@ -34,7 +34,10 @@ func Init(opts ...Option) (*Manager, error) {
} }
sourceManager := NewManager() sourceManager := NewManager()
if o.File != nil { if o.File != nil {
sourceManager.AddSource(NewFileSource(*o.File)) s := NewFileSource(o.File)
sourceManager.AddSource(s)
s.SetEventHandler(sourceManager)
} }
if o.EnvKeyFormatter != nil { if o.EnvKeyFormatter != nil {
sourceManager.AddSource(NewEnvSource(o.EnvKeyFormatter)) sourceManager.AddSource(NewEnvSource(o.EnvKeyFormatter))
@ -44,8 +47,8 @@ func Init(opts ...Option) (*Manager, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
s.eh = sourceManager
sourceManager.AddSource(s) sourceManager.AddSource(s)
s.SetEventHandler(sourceManager)
} }
return sourceManager, nil return sourceManager, nil

View File

@ -57,11 +57,10 @@ func TestConfigFromRemote(t *testing.T) {
t.Setenv("TMP_KEY", "1") t.Setenv("TMP_KEY", "1")
t.Setenv("log.level", "info") t.Setenv("log.level", "info")
mgr, _ := Init(WithEnvSource(formatKey), mgr, _ := Init(WithEnvSource(formatKey),
WithFilesSource("../../configs/milvus.yaml"), WithFilesSource(&FileInfo{"../../configs/milvus.yaml", -1}),
WithEtcdSource(&EtcdInfo{ WithEtcdSource(&EtcdInfo{
Endpoints: []string{cfg.ACUrls[0].Host}, Endpoints: []string{cfg.ACUrls[0].Host},
KeyPrefix: "test", KeyPrefix: "test",
RefreshMode: ModeInterval,
RefreshInterval: 10 * time.Millisecond, RefreshInterval: 10 * time.Millisecond,
})) }))
ctx := context.Background() ctx := context.Background()

View File

@ -23,14 +23,7 @@ import (
"sync" "sync"
"time" "time"
"github.com/milvus-io/milvus/internal/log"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
)
const (
ModeWatch = iota
ModeInterval
) )
type EtcdSource struct { type EtcdSource struct {
@ -39,10 +32,8 @@ type EtcdSource struct {
ctx context.Context ctx context.Context
currentConfig map[string]string currentConfig map[string]string
keyPrefix string keyPrefix string
refreshMode int
refreshInterval time.Duration configRefresher refresher
intervalDone chan bool
intervalInitOnce sync.Once
eh EventHandler eh EventHandler
} }
@ -54,15 +45,14 @@ func NewEtcdSource(remoteInfo *EtcdInfo) (*EtcdSource, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
return &EtcdSource{ es := &EtcdSource{
etcdCli: etcdCli, etcdCli: etcdCli,
ctx: context.Background(), ctx: context.Background(),
currentConfig: make(map[string]string), currentConfig: make(map[string]string),
keyPrefix: remoteInfo.KeyPrefix, keyPrefix: remoteInfo.KeyPrefix,
refreshMode: remoteInfo.RefreshMode, }
refreshInterval: remoteInfo.RefreshInterval, es.configRefresher = newRefresher(remoteInfo.RefreshInterval, es.refreshConfigurations)
intervalDone: make(chan bool, 1), return es, nil
}, nil
} }
// GetConfigurationByKey implements ConfigSource // GetConfigurationByKey implements ConfigSource
@ -83,12 +73,7 @@ func (es *EtcdSource) GetConfigurations() (map[string]string, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
if es.refreshMode == ModeInterval { es.configRefresher.start()
es.intervalInitOnce.Do(func() {
go es.refreshConfigurationsPeriodically()
})
}
es.RLock() es.RLock()
for key, value := range es.currentConfig { for key, value := range es.currentConfig {
configMap[key] = value configMap[key] = value
@ -109,7 +94,11 @@ func (es *EtcdSource) GetSourceName() string {
} }
func (es *EtcdSource) Close() { func (es *EtcdSource) Close() {
es.intervalDone <- true es.configRefresher.stop()
}
func (es *EtcdSource) SetEventHandler(eh EventHandler) {
es.configRefresher.eh = eh
} }
func (es *EtcdSource) refreshConfigurations() error { func (es *EtcdSource) refreshConfigurations() error {
@ -125,42 +114,12 @@ func (es *EtcdSource) refreshConfigurations() error {
newConfig[key] = string(kv.Value) newConfig[key] = string(kv.Value)
newConfig[formatKey(key)] = string(kv.Value) newConfig[formatKey(key)] = string(kv.Value)
} }
return es.updateConfigurationAndFireEvent(newConfig)
}
func (es *EtcdSource) refreshConfigurationsPeriodically() {
ticker := time.NewTicker(es.refreshInterval)
log.Info("start refreshing configurations")
for {
select {
case <-ticker.C:
err := es.refreshConfigurations()
if err != nil {
log.Error("can not pull configs", zap.Error(err))
es.intervalDone <- true
}
case <-es.intervalDone:
log.Info("stop refreshing configurations")
return
}
}
}
func (es *EtcdSource) updateConfigurationAndFireEvent(config map[string]string) error {
es.Lock() es.Lock()
defer es.Unlock() defer es.Unlock()
//Populate the events based on the changed value between current config and newly received Config err = es.configRefresher.fireEvents(es.GetSourceName(), es.currentConfig, newConfig)
events, err := PopulateEvents(es.GetSourceName(), es.currentConfig, config)
if err != nil { if err != nil {
log.Warn("generating event error", zap.Error(err))
return err return err
} }
es.currentConfig = config es.currentConfig = newConfig
//Generate OnEvent Callback based on the events created
if es.eh != nil {
for _, e := range events {
es.eh.OnEvent(e)
}
}
return nil return nil
} }

View File

@ -0,0 +1,66 @@
// 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 config
type EventDispatcher struct {
registry map[string][]EventHandler
}
func NewEventDispatcher() *EventDispatcher {
return &EventDispatcher{
registry: make(map[string][]EventHandler),
}
}
func (ed *EventDispatcher) Get(key string) []EventHandler {
return ed.registry[formatKey(key)]
}
func (ed *EventDispatcher) Dispatch(event *Event) {
hs, ok := ed.registry[event.Key]
if !ok {
return
}
for _, h := range hs {
h.OnEvent(event)
}
}
// register a handler to watch specific config changed
func (ed *EventDispatcher) Register(key string, handler EventHandler) {
key = formatKey(key)
v, ok := ed.registry[key]
if ok {
ed.registry[key] = append(v, handler)
} else {
ed.registry[key] = []EventHandler{handler}
}
}
func (ed *EventDispatcher) Unregister(key string, handler EventHandler) {
v, ok := ed.registry[key]
if !ok {
return
}
newGroup := make([]EventHandler, 0)
for _, eh := range v {
if eh.GetIdentifier() == handler.GetIdentifier() {
continue
}
newGroup = append(newGroup, eh)
}
ed.registry[key] = newGroup
}

View File

@ -32,17 +32,24 @@ type FileSource struct {
sync.RWMutex sync.RWMutex
file string file string
configs map[string]string configs map[string]string
configRefresher refresher
} }
func NewFileSource(file string) *FileSource { func NewFileSource(fileInfo *FileInfo) *FileSource {
fs := &FileSource{file: file, configs: make(map[string]string)} fs := &FileSource{
fs.loadFromFile() file: fileInfo.Filepath,
configs: make(map[string]string),
}
fs.configRefresher = newRefresher(fileInfo.RefreshInterval, fs.loadFromFile)
return fs return fs
} }
// GetConfigurationByKey implements ConfigSource // GetConfigurationByKey implements ConfigSource
func (fs *FileSource) GetConfigurationByKey(key string) (string, error) { func (fs *FileSource) GetConfigurationByKey(key string) (string, error) {
fs.RLock()
v, ok := fs.configs[key] v, ok := fs.configs[key]
fs.RUnlock()
if !ok { if !ok {
return "", fmt.Errorf("key not found: %s", key) return "", fmt.Errorf("key not found: %s", key)
} }
@ -52,11 +59,19 @@ func (fs *FileSource) GetConfigurationByKey(key string) (string, error) {
// GetConfigurations implements ConfigSource // GetConfigurations implements ConfigSource
func (fs *FileSource) GetConfigurations() (map[string]string, error) { func (fs *FileSource) GetConfigurations() (map[string]string, error) {
configMap := make(map[string]string) configMap := make(map[string]string)
fs.Lock()
defer fs.Unlock() err := fs.loadFromFile()
if err != nil {
return nil, err
}
fs.configRefresher.start()
fs.RLock()
for k, v := range fs.configs { for k, v := range fs.configs {
configMap[k] = v configMap[k] = v
} }
fs.RUnlock()
return configMap, nil return configMap, nil
} }
@ -71,8 +86,12 @@ func (fs *FileSource) GetSourceName() string {
} }
func (fs *FileSource) Close() { func (fs *FileSource) Close() {
fs.configRefresher.stop()
} }
func (fs *FileSource) SetEventHandler(eh EventHandler) {
fs.configRefresher.eh = eh
}
func (fs *FileSource) loadFromFile() error { func (fs *FileSource) loadFromFile() error {
yamlReader := viper.New() yamlReader := viper.New()
configFile := fs.file configFile := fs.file
@ -86,6 +105,7 @@ func (fs *FileSource) loadFromFile() error {
return err return err
} }
newConfig := make(map[string]string)
for _, key := range yamlReader.AllKeys() { for _, key := range yamlReader.AllKeys() {
val := yamlReader.Get(key) val := yamlReader.Get(key)
str, err := cast.ToStringE(val) str, err := cast.ToStringE(val)
@ -110,11 +130,17 @@ func (fs *FileSource) loadFromFile() error {
continue continue
} }
} }
fs.Lock() newConfig[key] = str
fs.configs[key] = str newConfig[formatKey(key)] = str
fs.configs[formatKey(key)] = str
fs.Unlock()
} }
fs.Lock()
defer fs.Unlock()
err := fs.configRefresher.fireEvents(fs.GetSourceName(), fs.configs, newConfig)
if err != nil {
return err
}
fs.configs = newConfig
return nil return nil
} }

View File

@ -26,19 +26,20 @@ import (
) )
const ( const (
TombValue = "TOMB_VALUE" TombValue = "TOMB_VAULE"
CustomSourceName = "CustomSource"
) )
type Manager struct { type Manager struct {
sync.RWMutex sync.RWMutex
Dispatcher *EventDispatcher
sources map[string]Source sources map[string]Source
keySourceMap map[string]string keySourceMap map[string]string
overlayConfigs map[string]string // store the configs setted or deleted by user overlayConfigs map[string]string
} }
func NewManager() *Manager { func NewManager() *Manager {
return &Manager{ return &Manager{
Dispatcher: NewEventDispatcher(),
sources: make(map[string]Source), sources: make(map[string]Source),
keySourceMap: make(map[string]string), keySourceMap: make(map[string]string),
overlayConfigs: make(map[string]string), overlayConfigs: make(map[string]string),
@ -52,7 +53,7 @@ func (m *Manager) GetConfig(key string) (string, error) {
v, ok := m.overlayConfigs[realKey] v, ok := m.overlayConfigs[realKey]
if ok { if ok {
if v == TombValue { if v == TombValue {
return "", fmt.Errorf("key not found: %s", key) return "", fmt.Errorf("key not found %s", key)
} }
return v, nil return v, nil
} }
@ -94,8 +95,8 @@ func (m *Manager) GetConfigsByPattern(pattern string, withPrefix bool) map[strin
return matchedConfig return matchedConfig
} }
// Configs returns all the key values // GetConfigs returns all the key values
func (m *Manager) Configs() map[string]string { func (m *Manager) GetConfigs() map[string]string {
m.RLock() m.RLock()
defer m.RUnlock() defer m.RUnlock()
config := make(map[string]string) config := make(map[string]string)
@ -111,24 +112,6 @@ func (m *Manager) Configs() map[string]string {
return config return config
} }
// For compatible reason, only visiable for Test
func (m *Manager) SetConfig(key, value string) {
m.Lock()
defer m.Unlock()
realKey := formatKey(key)
m.overlayConfigs[realKey] = value
m.updateEvent(newEvent(CustomSourceName, CreateType, realKey, value))
}
// For compatible reason, only visiable for Test
func (m *Manager) DeleteConfig(key string) {
m.Lock()
defer m.Unlock()
realKey := formatKey(key)
m.overlayConfigs[realKey] = TombValue
m.updateEvent(newEvent(realKey, DeleteType, realKey, ""))
}
func (m *Manager) Close() { func (m *Manager) Close() {
for _, s := range m.sources { for _, s := range m.sources {
s.Close() s.Close()
@ -156,6 +139,20 @@ func (m *Manager) AddSource(source Source) error {
return nil return nil
} }
// For compatible reason, only visiable for Test
func (m *Manager) SetConfig(key, value string) {
m.Lock()
defer m.Unlock()
m.overlayConfigs[formatKey(key)] = value
}
// For compatible reason, only visiable for Test
func (m *Manager) DeleteConfig(key string) {
m.Lock()
defer m.Unlock()
m.overlayConfigs[formatKey(key)] = TombValue
}
// Do not use it directly, only used when add source and unittests. // Do not use it directly, only used when add source and unittests.
func (m *Manager) pullSourceConfigs(source string) error { func (m *Manager) pullSourceConfigs(source string) error {
configSource, ok := m.sources[source] configSource, ok := m.sources[source]
@ -260,7 +257,11 @@ func (m *Manager) OnEvent(event *Event) {
return return
} }
// m.dispatcher.DispatchEvent(event) m.Dispatcher.Dispatch(event)
}
func (m *Manager) GetIdentifier() string {
return "Manager"
} }
func (m *Manager) findNextBestSource(key string, sourceName string) Source { func (m *Manager) findNextBestSource(key string, sourceName string) Source {

View File

@ -25,11 +25,11 @@ import (
func TestAllConfigFromManager(t *testing.T) { func TestAllConfigFromManager(t *testing.T) {
mgr, _ := Init() mgr, _ := Init()
all := mgr.Configs() all := mgr.GetConfigs()
assert.Equal(t, 0, len(all)) assert.Equal(t, 0, len(all))
mgr, _ = Init(WithEnvSource(formatKey)) mgr, _ = Init(WithEnvSource(formatKey))
all = mgr.Configs() all = mgr.GetConfigs()
assert.Less(t, 0, len(all)) assert.Less(t, 0, len(all))
} }

View File

@ -0,0 +1,87 @@
// 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 config
import (
"sync"
"time"
"github.com/milvus-io/milvus/internal/log"
"go.uber.org/zap"
)
type refresher struct {
refreshInterval time.Duration
intervalDone chan bool
intervalInitOnce sync.Once
eh EventHandler
fetchFunc func() error
}
func newRefresher(interval time.Duration, fetchFunc func() error) refresher {
return refresher{
refreshInterval: interval,
intervalDone: make(chan bool, 1),
fetchFunc: fetchFunc,
}
}
func (r refresher) start() {
if r.refreshInterval > 0 {
r.intervalInitOnce.Do(func() {
go r.refreshPeriodically()
})
}
}
func (r refresher) stop() {
r.intervalDone <- true
}
func (r refresher) refreshPeriodically() {
ticker := time.NewTicker(r.refreshInterval)
log.Info("start refreshing configurations")
for {
select {
case <-ticker.C:
err := r.fetchFunc()
if err != nil {
log.Error("can not pull configs", zap.Error(err))
r.intervalDone <- true
}
case <-r.intervalDone:
log.Info("stop refreshing configurations")
return
}
}
}
func (r refresher) fireEvents(name string, source, target map[string]string) error {
events, err := PopulateEvents(name, source, target)
if err != nil {
log.Warn("generating event error", zap.Error(err))
return err
}
//Generate OnEvent Callback based on the events created
if r.eh != nil {
for _, e := range events {
r.eh.OnEvent(e)
}
}
return nil
}

View File

@ -34,6 +34,7 @@ type Source interface {
// EventHandler handles config change event // EventHandler handles config change event
type EventHandler interface { type EventHandler interface {
OnEvent(event *Event) OnEvent(event *Event)
GetIdentifier() string
} }
// EtcdInfo has attribute for config center source initialization // EtcdInfo has attribute for config center source initialization
@ -41,14 +42,19 @@ type EtcdInfo struct {
Endpoints []string Endpoints []string
KeyPrefix string KeyPrefix string
RefreshMode int
//Pull Configuration interval, unit is second //Pull Configuration interval, unit is second
RefreshInterval time.Duration RefreshInterval time.Duration
} }
// FileInfo has attribute for file source
type FileInfo struct {
Filepath string
RefreshInterval time.Duration
}
//Options hold options //Options hold options
type Options struct { type Options struct {
File *string File *FileInfo
EtcdInfo *EtcdInfo EtcdInfo *EtcdInfo
EnvKeyFormatter func(string) string EnvKeyFormatter func(string) string
} }
@ -57,9 +63,9 @@ type Options struct {
type Option func(options *Options) type Option func(options *Options)
//WithRequiredFiles tell archaius to manage files, if not exist will return error //WithRequiredFiles tell archaius to manage files, if not exist will return error
func WithFilesSource(f string) Option { func WithFilesSource(fi *FileInfo) Option {
return func(options *Options) { return func(options *Options) {
options.File = &f options.File = fi
} }
} }

View File

@ -24,12 +24,12 @@ import (
func TestLoadFromFileSource(t *testing.T) { func TestLoadFromFileSource(t *testing.T) {
t.Run("file not exist", func(t *testing.T) { t.Run("file not exist", func(t *testing.T) {
fs := NewFileSource("file_not_exist.yaml") fs := NewFileSource(&FileInfo{"file_not_exist.yaml", -1})
err := fs.loadFromFile() err := fs.loadFromFile()
assert.Error(t, err, "cannot access config file: file_not_exist.yaml") assert.Error(t, err, "cannot access config file: file_not_exist.yaml")
}) })
t.Run("file type not support", func(t *testing.T) { t.Run("file type not support", func(t *testing.T) {
fs := NewFileSource("../../go.mod") fs := NewFileSource(&FileInfo{"../../go.mod", -1})
err := fs.loadFromFile() err := fs.loadFromFile()
assert.Error(t, err) assert.Error(t, err)
}) })

View File

@ -50,7 +50,7 @@ func Test_garbageCollector_basic(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
segRefer, err := NewSegmentReferenceManager(etcdKV, nil) segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err) assert.NoError(t, err)
assert.NotNil(t, segRefer) assert.NotNil(t, segRefer)
@ -112,7 +112,7 @@ func Test_garbageCollector_scan(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
segRefer, err := NewSegmentReferenceManager(etcdKV, nil) segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err) assert.NoError(t, err)
assert.NotNil(t, segRefer) assert.NotNil(t, segRefer)
@ -276,9 +276,9 @@ func Test_garbageCollector_scan(t *testing.T) {
// initialize unit test sso env // initialize unit test sso env
func initUtOSSEnv(bucket, root string, n int) (mcm *storage.MinioChunkManager, inserts []string, stats []string, delta []string, other []string, err error) { func initUtOSSEnv(bucket, root string, n int) (mcm *storage.MinioChunkManager, inserts []string, stats []string, delta []string, other []string, err error) {
Params.Init() Params.Init()
cli, err := minio.New(Params.MinioCfg.Address, &minio.Options{ cli, err := minio.New(Params.MinioCfg.Address.GetValue(), &minio.Options{
Creds: credentials.NewStaticV4(Params.MinioCfg.AccessKeyID, Params.MinioCfg.SecretAccessKey, ""), Creds: credentials.NewStaticV4(Params.MinioCfg.AccessKeyID.GetValue(), Params.MinioCfg.SecretAccessKey.GetValue(), ""),
Secure: Params.MinioCfg.UseSSL, Secure: Params.MinioCfg.UseSSL.GetAsBool(),
}) })
if err != nil { if err != nil {
return nil, nil, nil, nil, nil, err return nil, nil, nil, nil, nil, err

View File

@ -238,7 +238,7 @@ func (s *Server) Register() error {
} }
func (s *Server) initSession() error { func (s *Server) initSession() error {
s.session = sessionutil.NewSession(s.ctx, Params.EtcdCfg.MetaRootPath, s.etcdCli) s.session = sessionutil.NewSession(s.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if s.session == nil { if s.session == nil {
return errors.New("failed to initialize session") return errors.New("failed to initialize session")
} }
@ -455,7 +455,7 @@ func (s *Server) initSegmentManager() {
} }
func (s *Server) initMeta(chunkManagerRootPath string) error { func (s *Server) initMeta(chunkManagerRootPath string) error {
etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
s.kvClient = etcdKV s.kvClient = etcdKV
reloadEtcdFn := func() error { reloadEtcdFn := func() error {
var err error var err error
@ -803,7 +803,7 @@ func (s *Server) handleFlushingSegments(ctx context.Context) {
func (s *Server) initRootCoordClient() error { func (s *Server) initRootCoordClient() error {
var err error var err error
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.EtcdCfg.MetaRootPath, s.etcdCli); err != nil { if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli); err != nil {
return err return err
} }
if err = s.rootCoordClient.Init(); err != nil { if err = s.rootCoordClient.Init(); err != nil {

View File

@ -917,7 +917,7 @@ func TestServer_watchQueryCoord(t *testing.T) {
Params.Init() Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
assert.NotNil(t, etcdKV) assert.NotNil(t, etcdKV)
factory := dependency.NewDefaultFactory(true) factory := dependency.NewDefaultFactory(true)
svr := CreateServer(context.TODO(), factory) svr := CreateServer(context.TODO(), factory)
@ -3082,7 +3082,7 @@ func TestDataCoord_Import(t *testing.T) {
t.Run("no datanode available", func(t *testing.T) { t.Run("no datanode available", func(t *testing.T) {
svr := newTestServer(t, nil) svr := newTestServer(t, nil)
Params.MinioCfg.Address = "minio:9000" Params.BaseTable.Save("minio.address", "minio:9000")
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{ resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
ImportTask: &datapb.ImportTask{ ImportTask: &datapb.ImportTask{
CollectionId: 100, CollectionId: 100,
@ -3363,7 +3363,7 @@ func newTestServer(t *testing.T, receiveCh chan any, opts ...Option) *Server {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
assert.Nil(t, err) assert.Nil(t, err)
@ -3402,7 +3402,7 @@ func newTestServerWithMeta(t *testing.T, receiveCh chan any, meta *meta, opts ..
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
assert.Nil(t, err) assert.Nil(t, err)
@ -3450,15 +3450,15 @@ func newTestServer2(t *testing.T, receiveCh chan any, opts ...Option) *Server {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
assert.Nil(t, err) assert.Nil(t, err)
icSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli) icSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
icSession.Init(typeutil.IndexCoordRole, "localhost:31000", true, true) icSession.Init(typeutil.IndexCoordRole, "localhost:31000", true, true)
icSession.Register() icSession.Register()
qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli) qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true) qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true)
qcSession.Register() qcSession.Register()
@ -3567,7 +3567,7 @@ func Test_initServiceDiscovery(t *testing.T) {
}) })
assert.Nil(t, err) assert.Nil(t, err)
qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, server.etcdCli) qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath.GetValue(), server.etcdCli)
qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true) qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true)
qcSession.Register() qcSession.Register()
req := &datapb.AcquireSegmentLockRequest{ req := &datapb.AcquireSegmentLockRequest{
@ -3578,7 +3578,7 @@ func Test_initServiceDiscovery(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetErrorCode()) assert.Equal(t, commonpb.ErrorCode_Success, resp.GetErrorCode())
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot, typeutil.QueryCoordRole) sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.QueryCoordRole)
_, err = server.etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) _, err = server.etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
assert.Nil(t, err) assert.Nil(t, err)
@ -3596,7 +3596,9 @@ func Test_newChunkManagerFactory(t *testing.T) {
Params.DataCoordCfg.EnableGarbageCollection = true Params.DataCoordCfg.EnableGarbageCollection = true
t.Run("err_minio_bad_address", func(t *testing.T) { t.Run("err_minio_bad_address", func(t *testing.T) {
Params.MinioCfg.Address = "host:9000:bad" os.Setenv("minio.address", "host:9000:bad")
defer os.Unsetenv("minio.address")
Params.Init()
storageCli, err := server.newChunkManagerFactory() storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli) assert.Nil(t, storageCli)
assert.Error(t, err) assert.Error(t, err)
@ -3629,7 +3631,9 @@ func Test_initGarbageCollection(t *testing.T) {
}) })
t.Run("err_minio_bad_address", func(t *testing.T) { t.Run("err_minio_bad_address", func(t *testing.T) {
Params.CommonCfg.StorageType = "minio" Params.CommonCfg.StorageType = "minio"
Params.MinioCfg.Address = "host:9000:bad" os.Setenv("minio.address", "host:9000:bad")
defer os.Unsetenv("minio.address")
Params.Init()
storageCli, err := server.newChunkManagerFactory() storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli) assert.Nil(t, storageCli)
assert.Error(t, err) assert.Error(t, err)
@ -3644,7 +3648,7 @@ func testDataCoordBase(t *testing.T, opts ...Option) *Server {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -211,7 +211,7 @@ func (node *DataNode) Register() error {
} }
func (node *DataNode) initSession() error { func (node *DataNode) initSession() error {
node.session = sessionutil.NewSession(node.ctx, Params.EtcdCfg.MetaRootPath, node.etcdCli) node.session = sessionutil.NewSession(node.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), node.etcdCli)
if node.session == nil { if node.session == nil {
return errors.New("failed to initialize session") return errors.New("failed to initialize session")
} }
@ -493,7 +493,7 @@ func (node *DataNode) Start() error {
} }
connectEtcdFn := func() error { connectEtcdFn := func() error {
etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
node.watchKv = etcdKV node.watchKv = etcdKV
return nil return nil
} }

View File

@ -832,7 +832,7 @@ func TestWatchChannel(t *testing.T) {
t.Run("test watch channel", func(t *testing.T) { t.Run("test watch channel", func(t *testing.T) {
// GOOSE TODO // GOOSE TODO
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid" oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh) path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh)
err = kv.Save(path, string([]byte{23})) err = kv.Save(path, string([]byte{23}))
@ -891,7 +891,7 @@ func TestWatchChannel(t *testing.T) {
}) })
t.Run("Test release channel", func(t *testing.T) { t.Run("Test release channel", func(t *testing.T) {
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid" oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh) path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh)
err = kv.Save(path, string([]byte{23})) err = kv.Save(path, string([]byte{23}))

View File

@ -246,7 +246,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
chanName := "datanode-test-FlowGraphDeletenode-operate" chanName := "datanode-test-FlowGraphDeletenode-operate"
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
assert.NoError(t, clearEtcd(testPath)) assert.NoError(t, clearEtcd(testPath))
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
c := &nodeConfig{ c := &nodeConfig{
channel: channel, channel: channel,
@ -269,7 +269,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
chanName := "datanode-test-FlowGraphDeletenode-operate" chanName := "datanode-test-FlowGraphDeletenode-operate"
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
assert.NoError(t, clearEtcd(testPath)) assert.NoError(t, clearEtcd(testPath))
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
c := &nodeConfig{ c := &nodeConfig{
channel: channel, channel: channel,
@ -298,7 +298,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
chanName := "datanode-test-FlowGraphDeletenode-operate" chanName := "datanode-test-FlowGraphDeletenode-operate"
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
assert.NoError(t, clearEtcd(testPath)) assert.NoError(t, clearEtcd(testPath))
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
c := &nodeConfig{ c := &nodeConfig{
channel: channel, channel: channel,
@ -333,7 +333,6 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
chanName := "datanode-test-FlowGraphDeletenode-issue18565" chanName := "datanode-test-FlowGraphDeletenode-issue18565"
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
assert.NoError(t, clearEtcd(testPath)) assert.NoError(t, clearEtcd(testPath))
Params.EtcdCfg.MetaRootPath = testPath
channel := &ChannelMeta{ channel := &ChannelMeta{
segments: make(map[UniqueID]*Segment), segments: make(map[UniqueID]*Segment),
@ -387,7 +386,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
chanName := "datanode-test-FlowGraphDeletenode-autoflush" chanName := "datanode-test-FlowGraphDeletenode-autoflush"
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
assert.NoError(t, clearEtcd(testPath)) assert.NoError(t, clearEtcd(testPath))
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
c := &nodeConfig{ c := &nodeConfig{
channel: channel, channel: channel,
@ -474,7 +473,7 @@ func TestFlowGraphDeleteNode_showDelBuf(t *testing.T) {
chanName := "datanode-test-FlowGraphDeletenode-showDelBuf" chanName := "datanode-test-FlowGraphDeletenode-showDelBuf"
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
assert.NoError(t, clearEtcd(testPath)) assert.NoError(t, clearEtcd(testPath))
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
channel := &ChannelMeta{ channel := &ChannelMeta{
segments: make(map[UniqueID]*Segment), segments: make(map[UniqueID]*Segment),
@ -518,7 +517,7 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) {
chanName := "datanode-test-FlowGraphDeletenode-showDelBuf" chanName := "datanode-test-FlowGraphDeletenode-showDelBuf"
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
assert.NoError(t, clearEtcd(testPath)) assert.NoError(t, clearEtcd(testPath))
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
channel := ChannelMeta{ channel := ChannelMeta{
segments: make(map[UniqueID]*Segment), segments: make(map[UniqueID]*Segment),

View File

@ -70,7 +70,7 @@ func TestFlowGraphInsertBufferNodeCreate(t *testing.T) {
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
err := clearEtcd(testPath) err := clearEtcd(testPath)
require.NoError(t, err) require.NoError(t, err)
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
Factory := &MetaFactory{} Factory := &MetaFactory{}
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64) collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
@ -163,7 +163,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
err := clearEtcd(testPath) err := clearEtcd(testPath)
require.NoError(t, err) require.NoError(t, err)
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
Factory := &MetaFactory{} Factory := &MetaFactory{}
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64) collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
@ -312,7 +312,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
err := clearEtcd(testPath) err := clearEtcd(testPath)
require.NoError(t, err) require.NoError(t, err)
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
Factory := &MetaFactory{} Factory := &MetaFactory{}
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64) collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
@ -550,7 +550,7 @@ func TestRollBF(t *testing.T) {
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
err := clearEtcd(testPath) err := clearEtcd(testPath)
require.NoError(t, err) require.NoError(t, err)
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
Factory := &MetaFactory{} Factory := &MetaFactory{}
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64) collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
@ -904,7 +904,7 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
testPath := "/test/datanode/root/meta" testPath := "/test/datanode/root/meta"
err := clearEtcd(testPath) err := clearEtcd(testPath)
require.NoError(t, err) require.NoError(t, err)
Params.EtcdCfg.MetaRootPath = testPath Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root")
Factory := &MetaFactory{} Factory := &MetaFactory{}
tests := []struct { tests := []struct {

View File

@ -35,7 +35,7 @@ func Test_NewClient(t *testing.T) {
ctx := context.Background() ctx := context.Background()
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
client, err := NewClient(ctx, proxy.Params.EtcdCfg.MetaRootPath, etcdCli) client, err := NewClient(ctx, proxy.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, client) assert.NotNil(t, client)

View File

@ -100,7 +100,7 @@ func (s *Server) init() error {
if s.indexCoord == nil { if s.indexCoord == nil {
var err error var err error
log.Debug("create IndexCoord client for DataCoord") log.Debug("create IndexCoord client for DataCoord")
s.indexCoord, err = icc.NewClient(s.ctx, Params.EtcdCfg.MetaRootPath, etcdCli) s.indexCoord, err = icc.NewClient(s.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create IndexCoord client for DataCoord", zap.Error(err)) log.Warn("failed to create IndexCoord client for DataCoord", zap.Error(err))
return err return err

View File

@ -250,7 +250,7 @@ func (s *Server) init() error {
// --- RootCoord Client --- // --- RootCoord Client ---
if s.newRootCoordClient != nil { if s.newRootCoordClient != nil {
log.Info("initializing RootCoord client for DataNode") log.Info("initializing RootCoord client for DataNode")
rootCoordClient, err := s.newRootCoordClient(dn.Params.EtcdCfg.MetaRootPath, s.etcdCli) rootCoordClient, err := s.newRootCoordClient(dn.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil { if err != nil {
log.Error("failed to create new RootCoord client", zap.Error(err)) log.Error("failed to create new RootCoord client", zap.Error(err))
panic(err) panic(err)
@ -276,7 +276,7 @@ func (s *Server) init() error {
// --- DataCoord Client --- // --- DataCoord Client ---
if s.newDataCoordClient != nil { if s.newDataCoordClient != nil {
log.Debug("starting DataCoord client for DataNode") log.Debug("starting DataCoord client for DataNode")
dataCoordClient, err := s.newDataCoordClient(dn.Params.EtcdCfg.MetaRootPath, s.etcdCli) dataCoordClient, err := s.newDataCoordClient(dn.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil { if err != nil {
log.Error("failed to create new DataCoord client", zap.Error(err)) log.Error("failed to create new DataCoord client", zap.Error(err))
panic(err) panic(err)

View File

@ -52,13 +52,13 @@ func TestIndexCoordClient(t *testing.T) {
address = addr address = addr
} }
icm.CallRegister = func() error { icm.CallRegister = func() error {
session := sessionutil.NewSession(context.Background(), indexcoord.Params.EtcdCfg.MetaRootPath, etcdCli) session := sessionutil.NewSession(context.Background(), indexcoord.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
session.Init(typeutil.IndexCoordRole, address, true, false) session.Init(typeutil.IndexCoordRole, address, true, false)
session.Register() session.Register()
return err return err
} }
icm.CallStop = func() error { icm.CallStop = func() error {
etcdKV := etcdkv.NewEtcdKV(etcdCli, indexcoord.Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, indexcoord.Params.EtcdCfg.MetaRootPath.GetValue())
err = etcdKV.RemoveWithPrefix("session/" + typeutil.IndexCoordRole) err = etcdKV.RemoveWithPrefix("session/" + typeutil.IndexCoordRole)
return err return err
} }
@ -76,7 +76,7 @@ func TestIndexCoordClient(t *testing.T) {
// //
//etcdCli, err := etcd.GetEtcdClient(&indexcoord.Params.EtcdCfg) //etcdCli, err := etcd.GetEtcdClient(&indexcoord.Params.EtcdCfg)
//assert.NoError(t, err) //assert.NoError(t, err)
icc, err := NewClient(ctx, indexcoord.Params.EtcdCfg.MetaRootPath, etcdCli) icc, err := NewClient(ctx, indexcoord.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
assert.NoError(t, err) assert.NoError(t, err)
assert.NotNil(t, icc) assert.NotNil(t, icc)

View File

@ -120,7 +120,7 @@ func (s *Server) init() error {
// --- RootCoord --- // --- RootCoord ---
if s.rootCoord == nil { if s.rootCoord == nil {
s.rootCoord, err = rcc.NewClient(s.loopCtx, ic.Params.EtcdCfg.MetaRootPath, s.etcdCli) s.rootCoord, err = rcc.NewClient(s.loopCtx, ic.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil { if err != nil {
log.Debug("IndexCoord try to new RootCoord client failed", zap.Error(err)) log.Debug("IndexCoord try to new RootCoord client failed", zap.Error(err))
panic(err) panic(err)
@ -148,7 +148,7 @@ func (s *Server) init() error {
// --- DataCoord --- // --- DataCoord ---
if s.dataCoord == nil { if s.dataCoord == nil {
s.dataCoord, err = dcc.NewClient(s.loopCtx, ic.Params.EtcdCfg.MetaRootPath, s.etcdCli) s.dataCoord, err = dcc.NewClient(s.loopCtx, ic.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil { if err != nil {
log.Debug("IndexCoord try to new DataCoord client failed", zap.Error(err)) log.Debug("IndexCoord try to new DataCoord client failed", zap.Error(err))
panic(err) panic(err)

View File

@ -358,7 +358,7 @@ func (s *Server) init() error {
if s.rootCoordClient == nil { if s.rootCoordClient == nil {
var err error var err error
log.Debug("create RootCoord client for Proxy") log.Debug("create RootCoord client for Proxy")
s.rootCoordClient, err = rcc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath, etcdCli) s.rootCoordClient, err = rcc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create RootCoord client for Proxy", zap.Error(err)) log.Warn("failed to create RootCoord client for Proxy", zap.Error(err))
return err return err
@ -387,7 +387,7 @@ func (s *Server) init() error {
if s.dataCoordClient == nil { if s.dataCoordClient == nil {
var err error var err error
log.Debug("create DataCoord client for Proxy") log.Debug("create DataCoord client for Proxy")
s.dataCoordClient, err = dcc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath, etcdCli) s.dataCoordClient, err = dcc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create DataCoord client for Proxy", zap.Error(err)) log.Warn("failed to create DataCoord client for Proxy", zap.Error(err))
return err return err
@ -416,7 +416,7 @@ func (s *Server) init() error {
if s.indexCoordClient == nil { if s.indexCoordClient == nil {
var err error var err error
log.Debug("create IndexCoord client for Proxy") log.Debug("create IndexCoord client for Proxy")
s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath, etcdCli) s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err)) log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err))
return err return err
@ -445,7 +445,7 @@ func (s *Server) init() error {
if s.queryCoordClient == nil { if s.queryCoordClient == nil {
var err error var err error
log.Debug("create QueryCoord client for Proxy") log.Debug("create QueryCoord client for Proxy")
s.queryCoordClient, err = qcc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath, etcdCli) s.queryCoordClient, err = qcc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
if err != nil { if err != nil {
log.Warn("failed to create QueryCoord client for Proxy", zap.Error(err)) log.Warn("failed to create QueryCoord client for Proxy", zap.Error(err))
return err return err

View File

@ -37,7 +37,7 @@ func Test_NewClient(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg)
assert.NoError(t, err) assert.NoError(t, err)
client, err := NewClient(ctx, proxy.Params.EtcdCfg.MetaRootPath, etcdCli) client, err := NewClient(ctx, proxy.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, client) assert.NotNil(t, client)

View File

@ -134,7 +134,7 @@ func (s *Server) init() error {
// --- Master Server Client --- // --- Master Server Client ---
if s.rootCoord == nil { if s.rootCoord == nil {
s.rootCoord, err = rcc.NewClient(s.loopCtx, qc.Params.EtcdCfg.MetaRootPath, s.etcdCli) s.rootCoord, err = rcc.NewClient(s.loopCtx, qc.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil { if err != nil {
log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err)) log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err))
panic(err) panic(err)
@ -165,7 +165,7 @@ func (s *Server) init() error {
// --- Data service client --- // --- Data service client ---
if s.dataCoord == nil { if s.dataCoord == nil {
s.dataCoord, err = dcc.NewClient(s.loopCtx, qc.Params.EtcdCfg.MetaRootPath, s.etcdCli) s.dataCoord, err = dcc.NewClient(s.loopCtx, qc.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil { if err != nil {
log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err)) log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err))
panic(err) panic(err)
@ -193,7 +193,7 @@ func (s *Server) init() error {
// --- IndexCoord --- // --- IndexCoord ---
if s.indexCoord == nil { if s.indexCoord == nil {
s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.EtcdCfg.MetaRootPath, s.etcdCli) s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil { if err != nil {
log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err)) log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err))
panic(err) panic(err)

View File

@ -37,7 +37,7 @@ func Test_NewClient(t *testing.T) {
ctx := context.Background() ctx := context.Background()
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg)
assert.NoError(t, err) assert.NoError(t, err)
client, err := NewClient(ctx, proxy.Params.EtcdCfg.MetaRootPath, etcdCli) client, err := NewClient(ctx, proxy.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
assert.Nil(t, err) assert.Nil(t, err)
assert.NotNil(t, client) assert.NotNil(t, client)

View File

@ -180,7 +180,7 @@ func (s *Server) init() error {
if s.newDataCoordClient != nil { if s.newDataCoordClient != nil {
log.Debug("RootCoord start to create DataCoord client") log.Debug("RootCoord start to create DataCoord client")
dataCoord := s.newDataCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath, s.etcdCli) dataCoord := s.newDataCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err := s.rootCoord.SetDataCoord(s.ctx, dataCoord); err != nil { if err := s.rootCoord.SetDataCoord(s.ctx, dataCoord); err != nil {
panic(err) panic(err)
} }
@ -188,7 +188,7 @@ func (s *Server) init() error {
} }
if s.newIndexCoordClient != nil { if s.newIndexCoordClient != nil {
log.Debug("RootCoord start to create IndexCoord client") log.Debug("RootCoord start to create IndexCoord client")
indexCoord := s.newIndexCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath, s.etcdCli) indexCoord := s.newIndexCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err := s.rootCoord.SetIndexCoord(indexCoord); err != nil { if err := s.rootCoord.SetIndexCoord(indexCoord); err != nil {
panic(err) panic(err)
} }
@ -196,7 +196,7 @@ func (s *Server) init() error {
} }
if s.newQueryCoordClient != nil { if s.newQueryCoordClient != nil {
log.Debug("RootCoord start to create QueryCoord client") log.Debug("RootCoord start to create QueryCoord client")
queryCoord := s.newQueryCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath, s.etcdCli) queryCoord := s.newQueryCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err := s.rootCoord.SetQueryCoord(queryCoord); err != nil { if err := s.rootCoord.SetQueryCoord(queryCoord); err != nil {
panic(err) panic(err)
} }

View File

@ -180,12 +180,13 @@ func TestRun(t *testing.T) {
rand.Seed(time.Now().UnixNano()) rand.Seed(time.Now().UnixNano())
randVal := rand.Int() randVal := rand.Int()
rootPath := fmt.Sprintf("/%d/test", randVal)
rootcoord.Params.BaseTable.Save("etcd.rootPath", rootPath)
rootcoord.Params.Init() rootcoord.Params.Init()
rootcoord.Params.EtcdCfg.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) assert.Nil(t, err)
sessKey := path.Join(rootcoord.Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(rootcoord.Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot)
_, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) _, err = etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
assert.Nil(t, err) assert.Nil(t, err)
err = svr.Run() err = svr.Run()

View File

@ -276,19 +276,19 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
var storageConfig *indexpb.StorageConfig var storageConfig *indexpb.StorageConfig
if Params.CommonCfg.StorageType == "local" { if Params.CommonCfg.StorageType == "local" {
storageConfig = &indexpb.StorageConfig{ storageConfig = &indexpb.StorageConfig{
RootPath: Params.LocalStorageCfg.Path, RootPath: Params.LocalStorageCfg.Path.GetValue(),
StorageType: Params.CommonCfg.StorageType, StorageType: Params.CommonCfg.StorageType,
} }
} else { } else {
storageConfig = &indexpb.StorageConfig{ storageConfig = &indexpb.StorageConfig{
Address: Params.MinioCfg.Address, Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID, AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: Params.MinioCfg.SecretAccessKey, SecretAccessKey: Params.MinioCfg.SecretAccessKey.GetValue(),
UseSSL: Params.MinioCfg.UseSSL, UseSSL: Params.MinioCfg.UseSSL.GetAsBool(),
BucketName: Params.MinioCfg.BucketName, BucketName: Params.MinioCfg.BucketName.GetValue(),
RootPath: Params.MinioCfg.RootPath, RootPath: Params.MinioCfg.RootPath.GetValue(),
UseIAM: Params.MinioCfg.UseIAM, UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint, IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
StorageType: Params.CommonCfg.StorageType, StorageType: Params.CommonCfg.StorageType,
} }
} }

View File

@ -157,7 +157,7 @@ func (i *IndexCoord) Register() error {
} }
func (i *IndexCoord) initSession() error { func (i *IndexCoord) initSession() error {
i.session = sessionutil.NewSession(i.loopCtx, Params.EtcdCfg.MetaRootPath, i.etcdCli) i.session = sessionutil.NewSession(i.loopCtx, Params.EtcdCfg.MetaRootPath.GetValue(), i.etcdCli)
if i.session == nil { if i.session == nil {
return errors.New("failed to initialize session") return errors.New("failed to initialize session")
} }
@ -183,7 +183,7 @@ func (i *IndexCoord) Init() error {
} }
connectEtcdFn := func() error { connectEtcdFn := func() error {
i.etcdKV = etcdkv.NewEtcdKV(i.etcdCli, Params.EtcdCfg.MetaRootPath) i.etcdKV = etcdkv.NewEtcdKV(i.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
i.metaTable, err = NewMetaTable(i.etcdKV) i.metaTable, err = NewMetaTable(i.etcdKV)
return err return err
} }

View File

@ -49,11 +49,11 @@ import (
func TestMockEtcd(t *testing.T) { func TestMockEtcd(t *testing.T) {
Params.InitOnce() Params.InitOnce()
Params.EtcdCfg.MetaRootPath = "indexcoord-mock" Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root/indexcoord-mock")
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.NoError(t, err) assert.NoError(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
mockEtcd := NewMockEtcdKVWithReal(etcdKV) mockEtcd := NewMockEtcdKVWithReal(etcdKV)
key := "foo" key := "foo"
@ -94,7 +94,7 @@ func TestMockEtcd(t *testing.T) {
func testIndexCoord(t *testing.T) { func testIndexCoord(t *testing.T) {
ctx := context.Background() ctx := context.Background()
Params.EtcdCfg.MetaRootPath = "indexcoord-ut" Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root/indexcoord-ut")
// first start an IndexNode // first start an IndexNode
inm0 := indexnode.NewIndexNodeMock() inm0 := indexnode.NewIndexNodeMock()
@ -495,13 +495,14 @@ func testIndexCoord(t *testing.T) {
err = ic.Stop() err = ic.Stop()
assert.NoError(t, err) assert.NoError(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
err = etcdKV.RemoveWithPrefix("") err = etcdKV.RemoveWithPrefix("")
assert.NoError(t, err) assert.NoError(t, err)
} }
func TestIndexCoord_DisableActiveStandby(t *testing.T) { func TestIndexCoord_DisableActiveStandby(t *testing.T) {
Params.InitOnce() Params.InitOnce()
indexnode.Params.InitOnce()
Params.IndexCoordCfg.EnableActiveStandby = false Params.IndexCoordCfg.EnableActiveStandby = false
testIndexCoord(t) testIndexCoord(t)
} }
@ -509,6 +510,7 @@ func TestIndexCoord_DisableActiveStandby(t *testing.T) {
// make sure the main functions work well when EnableActiveStandby=true // make sure the main functions work well when EnableActiveStandby=true
func TestIndexCoord_EnableActiveStandby(t *testing.T) { func TestIndexCoord_EnableActiveStandby(t *testing.T) {
Params.InitOnce() Params.InitOnce()
indexnode.Params.InitOnce()
Params.IndexCoordCfg.EnableActiveStandby = true Params.IndexCoordCfg.EnableActiveStandby = true
testIndexCoord(t) testIndexCoord(t)
} }

View File

@ -80,7 +80,7 @@ func getSystemInfoMetrics(
ID: coord.session.ServerID, ID: coord.session.ServerID,
}, },
SystemConfigurations: metricsinfo.IndexCoordConfiguration{ SystemConfigurations: metricsinfo.IndexCoordConfiguration{
MinioBucketName: Params.MinioCfg.BucketName, MinioBucketName: Params.MinioCfg.BucketName.GetValue(),
}, },
}, },
ConnectedNodes: make([]metricsinfo.IndexNodeInfos, 0), ConnectedNodes: make([]metricsinfo.IndexNodeInfos, 0),

View File

@ -164,14 +164,14 @@ func (i *IndexNode) initKnowhere() {
cThreadCoreCoefficient := C.int64_t(Params.CommonCfg.ThreadCoreCoefficient) cThreadCoreCoefficient := C.int64_t(Params.CommonCfg.ThreadCoreCoefficient)
C.InitThreadCoreCoefficient(cThreadCoreCoefficient) C.InitThreadCoreCoefficient(cThreadCoreCoefficient)
cCpuNum := C.int(hardware.GetCPUNum()) cCPUNum := C.int(hardware.GetCPUNum())
C.InitCpuNum(cCpuNum) C.InitCpuNum(cCPUNum)
initcore.InitLocalStorageConfig(Params) initcore.InitLocalStorageConfig(Params)
} }
func (i *IndexNode) initSession() error { func (i *IndexNode) initSession() error {
i.session = sessionutil.NewSession(i.loopCtx, Params.EtcdCfg.MetaRootPath, i.etcdCli) i.session = sessionutil.NewSession(i.loopCtx, Params.EtcdCfg.MetaRootPath.GetValue(), i.etcdCli)
if i.session == nil { if i.session == nil {
return errors.New("failed to initialize session") return errors.New("failed to initialize session")
} }

View File

@ -237,7 +237,7 @@ func getMockSystemInfoMetrics(
Type: typeutil.IndexNodeRole, Type: typeutil.IndexNodeRole,
}, },
SystemConfigurations: metricsinfo.IndexNodeConfiguration{ SystemConfigurations: metricsinfo.IndexNodeConfiguration{
MinioBucketName: Params.MinioCfg.BucketName, MinioBucketName: Params.MinioCfg.BucketName.GetValue(),
SimdType: Params.CommonCfg.SimdType, SimdType: Params.CommonCfg.SimdType,
}, },
} }

View File

@ -20,14 +20,14 @@ import (
func genStorageConfig() *indexpb.StorageConfig { func genStorageConfig() *indexpb.StorageConfig {
return &indexpb.StorageConfig{ return &indexpb.StorageConfig{
Address: Params.MinioCfg.Address, Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID, AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: Params.MinioCfg.SecretAccessKey, SecretAccessKey: Params.MinioCfg.SecretAccessKey.GetValue(),
BucketName: Params.MinioCfg.BucketName, BucketName: Params.MinioCfg.BucketName.GetValue(),
RootPath: Params.MinioCfg.RootPath, RootPath: Params.MinioCfg.RootPath.GetValue(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint, IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
UseSSL: Params.MinioCfg.UseSSL, UseSSL: Params.MinioCfg.UseSSL.GetAsBool(),
UseIAM: Params.MinioCfg.UseIAM, UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
} }
} }

View File

@ -76,7 +76,7 @@ func getSystemInfoMetrics(
ID: node.session.ServerID, ID: node.session.ServerID,
}, },
SystemConfigurations: metricsinfo.IndexNodeConfiguration{ SystemConfigurations: metricsinfo.IndexNodeConfiguration{
MinioBucketName: Params.MinioCfg.BucketName, MinioBucketName: Params.MinioCfg.BucketName.GetValue(),
SimdType: Params.CommonCfg.SimdType, SimdType: Params.CommonCfg.SimdType,
}, },
} }

View File

@ -31,10 +31,9 @@ import (
func TestEtcdRestartLoad(te *testing.T) { func TestEtcdRestartLoad(te *testing.T) {
etcdDataDir := "/tmp/_etcd_data" etcdDataDir := "/tmp/_etcd_data"
te.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode) te.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode)
te.Setenv("ETCD_USE_EMBED", "true")
param := new(paramtable.ServiceParam) param := new(paramtable.ServiceParam)
param.Init() param.Init()
param.BaseTable.Save("etcd.use.embed", "true")
// TODO, not sure if the relative path works for ci environment
param.BaseTable.Save("etcd.config.path", "../../../configs/advanced/etcd.yaml") param.BaseTable.Save("etcd.config.path", "../../../configs/advanced/etcd.yaml")
param.BaseTable.Save("etcd.data.dir", etcdDataDir) param.BaseTable.Save("etcd.data.dir", etcdDataDir)
//clean up data //clean up data
@ -42,7 +41,6 @@ func TestEtcdRestartLoad(te *testing.T) {
err := os.RemoveAll(etcdDataDir) err := os.RemoveAll(etcdDataDir)
assert.NoError(te, err) assert.NoError(te, err)
}() }()
param.EtcdCfg.LoadCfgToMemory()
te.Run("EtcdKV SaveRestartAndLoad", func(t *testing.T) { te.Run("EtcdKV SaveRestartAndLoad", func(t *testing.T) {
rootPath := "/etcd/test/root/saveRestartAndLoad" rootPath := "/etcd/test/root/saveRestartAndLoad"
metaKv, err := embed_etcd_kv.NewMetaKvFactory(rootPath, &param.EtcdCfg) metaKv, err := embed_etcd_kv.NewMetaKvFactory(rootPath, &param.EtcdCfg)

View File

@ -30,9 +30,9 @@ import (
func NewMetaKvFactory(rootPath string, etcdCfg *paramtable.EtcdConfig) (kv.MetaKv, error) { func NewMetaKvFactory(rootPath string, etcdCfg *paramtable.EtcdConfig) (kv.MetaKv, error) {
log.Info("start etcd with rootPath", log.Info("start etcd with rootPath",
zap.String("rootpath", rootPath), zap.String("rootpath", rootPath),
zap.Bool("isEmbed", etcdCfg.UseEmbedEtcd)) zap.Bool("isEmbed", etcdCfg.UseEmbedEtcd.GetAsBool()))
if etcdCfg.UseEmbedEtcd { if etcdCfg.UseEmbedEtcd.GetAsBool() {
path := etcdCfg.ConfigPath path := etcdCfg.ConfigPath.GetValue()
var cfg *embed.Config var cfg *embed.Config
if len(path) > 0 { if len(path) > 0 {
cfgFromFile, err := embed.ConfigFromFile(path) cfgFromFile, err := embed.ConfigFromFile(path)
@ -43,7 +43,7 @@ func NewMetaKvFactory(rootPath string, etcdCfg *paramtable.EtcdConfig) (kv.MetaK
} else { } else {
cfg = embed.NewConfig() cfg = embed.NewConfig()
} }
cfg.Dir = etcdCfg.DataDir cfg.Dir = etcdCfg.DataDir.GetValue()
metaKv, err := NewEmbededEtcdKV(cfg, rootPath) metaKv, err := NewEmbededEtcdKV(cfg, rootPath)
if err != nil { if err != nil {
return nil, err return nil, err

View File

@ -53,12 +53,12 @@ func NewPmsFactory(config *paramtable.PulsarConfig) *PmsFactory {
return &PmsFactory{ return &PmsFactory{
PulsarBufSize: 1024, PulsarBufSize: 1024,
ReceiveBufSize: 1024, ReceiveBufSize: 1024,
PulsarAddress: config.Address, PulsarAddress: config.Address.GetValue(),
PulsarWebAddress: config.WebAddress, PulsarWebAddress: config.WebAddress.GetValue(),
PulsarAuthPlugin: config.AuthPlugin, PulsarAuthPlugin: config.AuthPlugin.GetValue(),
PulsarAuthParams: config.AuthParams, PulsarAuthParams: config.AuthParams.GetValue(),
PulsarTenant: config.Tenant, PulsarTenant: config.Tenant.GetValue(),
PulsarNameSpace: config.Namespace, PulsarNameSpace: config.Namespace.GetValue(),
} }
} }

View File

@ -21,7 +21,6 @@ import (
"os" "os"
"testing" "testing"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -43,14 +42,13 @@ func TestPmsFactory(t *testing.T) {
} }
func TestPmsFactoryWithAuth(t *testing.T) { func TestPmsFactoryWithAuth(t *testing.T) {
config := &paramtable.PulsarConfig{ config := &Params.PulsarCfg
Address: Params.PulsarCfg.Address, Params.Save(Params.PulsarCfg.AuthPlugin.Key, "token")
WebAddress: Params.PulsarCfg.WebAddress, Params.Save(Params.PulsarCfg.AuthParams.Key, "token:fake_token")
MaxMessageSize: Params.PulsarCfg.MaxMessageSize, defer func() {
AuthPlugin: "token", Params.Save(Params.PulsarCfg.AuthPlugin.Key, "")
AuthParams: "{\"token\":\"fake_token\"}", Params.Save(Params.PulsarCfg.AuthParams.Key, "")
} }()
pmsFactory := NewPmsFactory(config) pmsFactory := NewPmsFactory(config)
ctx := context.Background() ctx := context.Background()
@ -63,7 +61,7 @@ func TestPmsFactoryWithAuth(t *testing.T) {
_, err = pmsFactory.NewQueryMsgStream(ctx) _, err = pmsFactory.NewQueryMsgStream(ctx)
assert.Nil(t, err) assert.Nil(t, err)
config.AuthParams = "" Params.Save(Params.PulsarCfg.AuthParams.Key, "")
pmsFactory = NewPmsFactory(config) pmsFactory = NewPmsFactory(config)
ctx = context.Background() ctx = context.Background()

View File

@ -47,18 +47,18 @@ func NewKafkaClientInstanceWithConfigMap(config kafka.ConfigMap, extraConsumerCo
} }
func NewKafkaClientInstanceWithConfig(config *paramtable.KafkaConfig) *kafkaClient { func NewKafkaClientInstanceWithConfig(config *paramtable.KafkaConfig) *kafkaClient {
kafkaConfig := getBasicConfig(config.Address) kafkaConfig := getBasicConfig(config.Address.GetValue())
if (config.SaslUsername == "" && config.SaslPassword != "") || if (config.SaslUsername.GetValue() == "" && config.SaslPassword.GetValue() != "") ||
(config.SaslUsername != "" && config.SaslPassword == "") { (config.SaslUsername.GetValue() != "" && config.SaslPassword.GetValue() == "") {
panic("enable security mode need config username and password at the same time!") panic("enable security mode need config username and password at the same time!")
} }
if config.SaslUsername != "" && config.SaslPassword != "" { if config.SaslUsername.GetValue() != "" && config.SaslPassword.GetValue() != "" {
kafkaConfig.SetKey("sasl.mechanisms", config.SaslMechanisms) kafkaConfig.SetKey("sasl.mechanisms", config.SaslMechanisms.GetValue())
kafkaConfig.SetKey("security.protocol", config.SecurityProtocol) kafkaConfig.SetKey("security.protocol", config.SecurityProtocol.GetValue())
kafkaConfig.SetKey("sasl.username", config.SaslUsername) kafkaConfig.SetKey("sasl.username", config.SaslUsername.GetValue())
kafkaConfig.SetKey("sasl.password", config.SaslPassword) kafkaConfig.SetKey("sasl.password", config.SaslPassword.GetValue())
} }
specExtraConfig := func(config map[string]string) kafka.ConfigMap { specExtraConfig := func(config map[string]string) kafka.ConfigMap {
@ -69,7 +69,7 @@ func NewKafkaClientInstanceWithConfig(config *paramtable.KafkaConfig) *kafkaClie
return kafkaConfigMap return kafkaConfigMap
} }
return NewKafkaClientInstanceWithConfigMap(kafkaConfig, specExtraConfig(config.ConsumerExtraConfig), specExtraConfig(config.ProducerExtraConfig)) return NewKafkaClientInstanceWithConfigMap(kafkaConfig, specExtraConfig(config.ConsumerExtraConfig.GetValue()), specExtraConfig(config.ProducerExtraConfig.GetValue()))
} }

View File

@ -298,20 +298,35 @@ func TestKafkaClient_MsgSerializAndDeserialize(t *testing.T) {
} }
func TestKafkaClient_NewKafkaClientInstanceWithConfig(t *testing.T) { func TestKafkaClient_NewKafkaClientInstanceWithConfig(t *testing.T) {
config1 := &paramtable.KafkaConfig{Address: "addr", SaslPassword: "password"} config1 := &paramtable.KafkaConfig{
Address: paramtable.ParamItem{Formatter: func(originValue string) string { return "addr" }},
SaslPassword: paramtable.ParamItem{Formatter: func(originValue string) string { return "password" }},
}
assert.Panics(t, func() { NewKafkaClientInstanceWithConfig(config1) }) assert.Panics(t, func() { NewKafkaClientInstanceWithConfig(config1) })
config2 := &paramtable.KafkaConfig{Address: "addr", SaslUsername: "username"} config2 := &paramtable.KafkaConfig{
Address: paramtable.ParamItem{Formatter: func(originValue string) string { return "addr" }},
SaslUsername: paramtable.ParamItem{Formatter: func(originValue string) string { return "username" }},
}
assert.Panics(t, func() { NewKafkaClientInstanceWithConfig(config2) }) assert.Panics(t, func() { NewKafkaClientInstanceWithConfig(config2) })
config3 := &paramtable.KafkaConfig{Address: "addr", SaslUsername: "username", SaslPassword: "password"} config3 := &paramtable.KafkaConfig{
Address: paramtable.ParamItem{Formatter: func(originValue string) string { return "addr" }},
SaslUsername: paramtable.ParamItem{Formatter: func(originValue string) string { return "username" }},
SaslPassword: paramtable.ParamItem{Formatter: func(originValue string) string { return "password" }},
}
client := NewKafkaClientInstanceWithConfig(config3) client := NewKafkaClientInstanceWithConfig(config3)
assert.NotNil(t, client) assert.NotNil(t, client)
assert.NotNil(t, client.basicConfig) assert.NotNil(t, client.basicConfig)
consumerConfig := make(map[string]string) consumerConfig := make(map[string]string)
consumerConfig["client.id"] = "dc" consumerConfig["client.id"] = "dc"
config4 := &paramtable.KafkaConfig{Address: "addr", SaslUsername: "username", SaslPassword: "password", ConsumerExtraConfig: consumerConfig} config4 := &paramtable.KafkaConfig{
Address: paramtable.ParamItem{Formatter: func(originValue string) string { return "addr" }},
SaslUsername: paramtable.ParamItem{Formatter: func(originValue string) string { return "username" }},
SaslPassword: paramtable.ParamItem{Formatter: func(originValue string) string { return "password" }},
ConsumerExtraConfig: paramtable.ParamGroup{GetFunc: func() map[string]string { return consumerConfig }},
}
client4 := NewKafkaClientInstanceWithConfig(config4) client4 := NewKafkaClientInstanceWithConfig(config4)
assert.Equal(t, "dc", client4.consumerConfig["client.id"]) assert.Equal(t, "dc", client4.consumerConfig["client.id"])
@ -322,7 +337,12 @@ func TestKafkaClient_NewKafkaClientInstanceWithConfig(t *testing.T) {
producerConfig := make(map[string]string) producerConfig := make(map[string]string)
producerConfig["client.id"] = "dc1" producerConfig["client.id"] = "dc1"
config5 := &paramtable.KafkaConfig{Address: "addr", SaslUsername: "username", SaslPassword: "password", ProducerExtraConfig: producerConfig} config5 := &paramtable.KafkaConfig{
Address: paramtable.ParamItem{Formatter: func(originValue string) string { return "addr" }},
SaslUsername: paramtable.ParamItem{Formatter: func(originValue string) string { return "username" }},
SaslPassword: paramtable.ParamItem{Formatter: func(originValue string) string { return "password" }},
ProducerExtraConfig: paramtable.ParamGroup{GetFunc: func() map[string]string { return producerConfig }},
}
client5 := NewKafkaClientInstanceWithConfig(config5) client5 := NewKafkaClientInstanceWithConfig(config5)
assert.Equal(t, "dc1", client5.producerConfig["client.id"]) assert.Equal(t, "dc1", client5.producerConfig["client.id"])

View File

@ -171,13 +171,13 @@ func TestRotateLogger_BasicError(t *testing.T) {
} }
func TestRotateLogger_InitError(t *testing.T) { func TestRotateLogger_InitError(t *testing.T) {
var Params paramtable.ComponentParam var params paramtable.ComponentParam
Params.Init() params.Init()
testPath := "" testPath := ""
Params.ProxyCfg.AccessLog.LocalPath = testPath params.ProxyCfg.AccessLog.LocalPath = testPath
Params.ProxyCfg.AccessLog.MinioEnable = true params.ProxyCfg.AccessLog.MinioEnable = true
Params.MinioCfg.Address = "" params.Save(params.MinioCfg.Address.Key, "")
//init err with invalid minio address //init err with invalid minio address
_, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) _, err := NewRotateLogger(&params.ProxyCfg.AccessLog, &params.MinioCfg)
assert.Error(t, err) assert.Error(t, err)
} }

View File

@ -62,14 +62,14 @@ type minioHandler struct {
func NewMinioHandler(ctx context.Context, cfg *paramtable.MinioConfig, rootPath string, queueLen int) (*minioHandler, error) { func NewMinioHandler(ctx context.Context, cfg *paramtable.MinioConfig, rootPath string, queueLen int) (*minioHandler, error) {
handlerCfg := config{ handlerCfg := config{
address: cfg.Address, address: cfg.Address.GetValue(),
bucketName: cfg.BucketName, bucketName: cfg.BucketName.GetValue(),
accessKeyID: cfg.AccessKeyID, accessKeyID: cfg.AccessKeyID.GetValue(),
secretAccessKeyID: cfg.SecretAccessKey, secretAccessKeyID: cfg.SecretAccessKey.GetValue(),
useSSL: cfg.UseSSL, useSSL: cfg.UseSSL.GetAsBool(),
createBucket: true, createBucket: true,
useIAM: cfg.UseIAM, useIAM: cfg.UseIAM.GetAsBool(),
iamEndpoint: cfg.IAMEndpoint, iamEndpoint: cfg.IAMEndpoint.GetValue(),
} }
client, err := newMinioClient(ctx, handlerCfg) client, err := newMinioClient(ctx, handlerCfg)

View File

@ -26,19 +26,19 @@ import (
) )
func TestMinioHandler_ConnectError(t *testing.T) { func TestMinioHandler_ConnectError(t *testing.T) {
var Params paramtable.ComponentParam var params paramtable.ComponentParam
Params.Init() params.Init()
testPath := "/tme/miniotest" testPath := "/tme/miniotest"
Params.ProxyCfg.AccessLog.LocalPath = testPath params.ProxyCfg.AccessLog.LocalPath = testPath
Params.MinioCfg.UseIAM = true params.Save(params.MinioCfg.UseIAM.Key, "true")
Params.MinioCfg.Address = "" params.Save(params.MinioCfg.Address.Key, "")
defer os.RemoveAll(testPath) defer os.RemoveAll(testPath)
_, err := NewMinioHandler( _, err := NewMinioHandler(
context.Background(), context.Background(),
&Params.MinioCfg, &params.MinioCfg,
Params.ProxyCfg.AccessLog.RemotePath, params.ProxyCfg.AccessLog.RemotePath,
Params.ProxyCfg.AccessLog.MaxBackups, params.ProxyCfg.AccessLog.MaxBackups,
) )
assert.Error(t, err) assert.Error(t, err)
} }

View File

@ -104,7 +104,6 @@ func (node *Proxy) InvalidateCollectionMetaCache(ctx context.Context, request *p
if !node.checkHealthy() { if !node.checkHealthy() {
return unhealthyStatus(), nil return unhealthyStatus(), nil
} }
ctx = logutil.WithModule(ctx, moduleName) ctx = logutil.WithModule(ctx, moduleName)
sp, ctx := trace.StartSpanFromContextWithOperationName(ctx, "Proxy-InvalidateCollectionMetaCache") sp, ctx := trace.StartSpanFromContextWithOperationName(ctx, "Proxy-InvalidateCollectionMetaCache")
defer sp.Finish() defer sp.Finish()

View File

@ -150,7 +150,7 @@ func (node *Proxy) Register() error {
// initSession initialize the session of Proxy. // initSession initialize the session of Proxy.
func (node *Proxy) initSession() error { func (node *Proxy) initSession() error {
node.session = sessionutil.NewSession(node.ctx, Params.EtcdCfg.MetaRootPath, node.etcdCli) node.session = sessionutil.NewSession(node.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), node.etcdCli)
if node.session == nil { if node.session == nil {
return errors.New("new session failed, maybe etcd cannot be connected") return errors.New("new session failed, maybe etcd cannot be connected")
} }
@ -184,7 +184,7 @@ func (node *Proxy) Init() error {
log.Info("init session for Proxy done") log.Info("init session for Proxy done")
node.factory.Init(Params) node.factory.Init(Params)
log.Debug("init parameters for factory", zap.String("role", typeutil.ProxyRole), zap.Any("parameters", Params.ServiceParam)) log.Debug("init parameters for factory", zap.String("role", typeutil.ProxyRole), zap.Any("parameters", Params.BaseTable.GetAll()))
accesslog.SetupAccseeLog(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) accesslog.SetupAccseeLog(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
log.Debug("init access log for Proxy done") log.Debug("init access log for Proxy done")

View File

@ -458,7 +458,7 @@ func TestProxy(t *testing.T) {
} }
qn := runQueryNode(ctx, localMsg, alias) qn := runQueryNode(ctx, localMsg, alias)
log.Info("running query node ...") log.Info("running QueryNode ...")
if qn != nil { if qn != nil {
defer func() { defer func() {
@ -506,7 +506,7 @@ func TestProxy(t *testing.T) {
go testServer.startGrpc(ctx, &wg) go testServer.startGrpc(ctx, &wg)
assert.NoError(t, testServer.waitForGrpcReady()) assert.NoError(t, testServer.waitForGrpcReady())
rootCoordClient, err := rcc.NewClient(ctx, Params.EtcdCfg.MetaRootPath, etcdcli) rootCoordClient, err := rcc.NewClient(ctx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdcli)
assert.NoError(t, err) assert.NoError(t, err)
err = rootCoordClient.Init() err = rootCoordClient.Init()
assert.NoError(t, err) assert.NoError(t, err)
@ -515,7 +515,7 @@ func TestProxy(t *testing.T) {
proxy.SetRootCoordClient(rootCoordClient) proxy.SetRootCoordClient(rootCoordClient)
log.Info("Proxy set root coordinator client") log.Info("Proxy set root coordinator client")
dataCoordClient, err := grpcdatacoordclient2.NewClient(ctx, Params.EtcdCfg.MetaRootPath, etcdcli) dataCoordClient, err := grpcdatacoordclient2.NewClient(ctx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdcli)
assert.NoError(t, err) assert.NoError(t, err)
err = dataCoordClient.Init() err = dataCoordClient.Init()
assert.NoError(t, err) assert.NoError(t, err)
@ -524,7 +524,7 @@ func TestProxy(t *testing.T) {
proxy.SetDataCoordClient(dataCoordClient) proxy.SetDataCoordClient(dataCoordClient)
log.Info("Proxy set data coordinator client") log.Info("Proxy set data coordinator client")
queryCoordClient, err := grpcquerycoordclient.NewClient(ctx, Params.EtcdCfg.MetaRootPath, etcdcli) queryCoordClient, err := grpcquerycoordclient.NewClient(ctx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdcli)
assert.NoError(t, err) assert.NoError(t, err)
err = queryCoordClient.Init() err = queryCoordClient.Init()
assert.NoError(t, err) assert.NoError(t, err)
@ -533,7 +533,7 @@ func TestProxy(t *testing.T) {
proxy.SetQueryCoordClient(queryCoordClient) proxy.SetQueryCoordClient(queryCoordClient)
log.Info("Proxy set query coordinator client") log.Info("Proxy set query coordinator client")
indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.EtcdCfg.MetaRootPath, etcdcli) indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdcli)
assert.NoError(t, err) assert.NoError(t, err)
err = indexCoordClient.Init() err = indexCoordClient.Init()
assert.NoError(t, err) assert.NoError(t, err)

View File

@ -260,7 +260,8 @@ func (it *insertTask) PreExecute(ctx context.Context) error {
} }
func (it *insertTask) assignSegmentID(channelNames []string) (*msgstream.MsgPack, error) { func (it *insertTask) assignSegmentID(channelNames []string) (*msgstream.MsgPack, error) {
threshold := Params.PulsarCfg.MaxMessageSize threshold := Params.PulsarCfg.MaxMessageSize.GetAsInt()
log.Debug("assign segmentid", zap.Int("threshold", threshold))
result := &msgstream.MsgPack{ result := &msgstream.MsgPack{
BeginTs: it.BeginTs(), BeginTs: it.BeginTs(),

View File

@ -43,9 +43,9 @@ func (suite *RowCountBasedBalancerTestSuite) SetupSuite() {
func (suite *RowCountBasedBalancerTestSuite) SetupTest() { func (suite *RowCountBasedBalancerTestSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
store := meta.NewMetaStore(suite.kv) store := meta.NewMetaStore(suite.kv)
idAllocator := RandomIncrementIDAllocator() idAllocator := RandomIncrementIDAllocator()

View File

@ -48,9 +48,9 @@ func (suite *ChannelCheckerTestSuite) SetupSuite() {
func (suite *ChannelCheckerTestSuite) SetupTest() { func (suite *ChannelCheckerTestSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
// meta // meta
store := meta.NewMetaStore(suite.kv) store := meta.NewMetaStore(suite.kv)

View File

@ -50,9 +50,9 @@ func (suite *SegmentCheckerTestSuite) SetupSuite() {
func (suite *SegmentCheckerTestSuite) SetupTest() { func (suite *SegmentCheckerTestSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
// meta // meta
store := meta.NewMetaStore(suite.kv) store := meta.NewMetaStore(suite.kv)

View File

@ -51,9 +51,9 @@ func (suite *DistControllerTestSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
// meta // meta
store := meta.NewMetaStore(suite.kv) store := meta.NewMetaStore(suite.kv)

View File

@ -116,9 +116,9 @@ func (suite *JobSuite) SetupSuite() {
func (suite *JobSuite) SetupTest() { func (suite *JobSuite) SetupTest() {
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
suite.store = meta.NewMetaStore(suite.kv) suite.store = meta.NewMetaStore(suite.kv)
suite.dist = meta.NewDistributionManager() suite.dist = meta.NewDistributionManager()

View File

@ -68,9 +68,9 @@ func (suite *CollectionManagerSuite) SetupSuite() {
func (suite *CollectionManagerSuite) SetupTest() { func (suite *CollectionManagerSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
suite.store = NewMetaStore(suite.kv) suite.store = NewMetaStore(suite.kv)
suite.mgr = NewCollectionManager(suite.store) suite.mgr = NewCollectionManager(suite.store)

View File

@ -51,9 +51,9 @@ func (suite *ReplicaManagerSuite) SetupSuite() {
func (suite *ReplicaManagerSuite) SetupTest() { func (suite *ReplicaManagerSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
suite.store = NewMetaStore(suite.kv) suite.store = NewMetaStore(suite.kv)
suite.idAllocator = RandomIncrementIDAllocator() suite.idAllocator = RandomIncrementIDAllocator()

View File

@ -87,9 +87,9 @@ func (suite *TargetManagerSuite) SetupSuite() {
func (suite *TargetManagerSuite) SetupTest() { func (suite *TargetManagerSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
// meta // meta
store := NewMetaStore(suite.kv) store := NewMetaStore(suite.kv)

View File

@ -58,7 +58,7 @@ func NewMockQueryNode(t *testing.T, etcdCli *clientv3.Client, nodeID int64) *Moc
MockQueryNodeServer: NewMockQueryNodeServer(t), MockQueryNodeServer: NewMockQueryNodeServer(t),
ctx: ctx, ctx: ctx,
cancel: cancel, cancel: cancel,
session: sessionutil.NewSession(ctx, Params.EtcdCfg.MetaRootPath, etcdCli), session: sessionutil.NewSession(ctx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli),
channels: make(map[int64][]string), channels: make(map[int64][]string),
segments: make(map[int64]map[string][]int64), segments: make(map[int64]map[string][]int64),
ID: nodeID, ID: nodeID,

View File

@ -145,9 +145,9 @@ func (suite *CollectionObserverSuite) SetupTest() {
suite.idAllocator = RandomIncrementIDAllocator() suite.idAllocator = RandomIncrementIDAllocator()
log.Debug("create embedded etcd KV...") log.Debug("create embedded etcd KV...")
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
client, err := etcd.GetEtcdClient(&config) client, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(client, Params.EtcdCfg.MetaRootPath+"-"+RandomMetaRootPath()) suite.kv = etcdkv.NewEtcdKV(client, Params.EtcdCfg.MetaRootPath.GetValue()+"-"+RandomMetaRootPath())
suite.Require().NoError(err) suite.Require().NoError(err)
log.Debug("create meta store...") log.Debug("create meta store...")
suite.store = meta.NewMetaStore(suite.kv) suite.store = meta.NewMetaStore(suite.kv)

View File

@ -53,9 +53,9 @@ func (suite *LeaderObserverTestSuite) SetupSuite() {
func (suite *LeaderObserverTestSuite) SetupTest() { func (suite *LeaderObserverTestSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
// meta // meta
store := meta.NewMetaStore(suite.kv) store := meta.NewMetaStore(suite.kv)

View File

@ -58,9 +58,9 @@ func (suite *TargetObserverSuite) SetupSuite() {
func (suite *TargetObserverSuite) SetupTest() { func (suite *TargetObserverSuite) SetupTest() {
var err error var err error
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
// meta // meta
store := meta.NewMetaStore(suite.kv) store := meta.NewMetaStore(suite.kv)

View File

@ -34,12 +34,13 @@ var (
// GenerateEtcdConfig returns a etcd config with a random root path, // GenerateEtcdConfig returns a etcd config with a random root path,
// NOTE: for test only // NOTE: for test only
func GenerateEtcdConfig() paramtable.EtcdConfig { func GenerateEtcdConfig() *paramtable.EtcdConfig {
config := Params.EtcdCfg config := &Params.EtcdCfg
rand.Seed(time.Now().UnixNano()) rand.Seed(time.Now().UnixNano())
suffix := "-test-querycoord" + strconv.FormatInt(rand.Int63(), 10) suffix := "-test-querycoord" + strconv.FormatInt(rand.Int63(), 10)
config.MetaRootPath = config.MetaRootPath + suffix
return config Params.BaseTable.Save("etcd.rootPath", config.MetaRootPath.GetValue()+suffix)
return &Params.EtcdCfg
} }
func RandomMetaRootPath() string { func RandomMetaRootPath() string {

View File

@ -146,11 +146,11 @@ func (s *Server) Register() error {
func (s *Server) Init() error { func (s *Server) Init() error {
log.Info("QueryCoord start init", log.Info("QueryCoord start init",
zap.String("meta-root-path", Params.EtcdCfg.MetaRootPath), zap.String("meta-root-path", Params.EtcdCfg.MetaRootPath.GetValue()),
zap.String("address", s.address)) zap.String("address", s.address))
// Init QueryCoord session // Init QueryCoord session
s.session = sessionutil.NewSession(s.ctx, Params.EtcdCfg.MetaRootPath, s.etcdCli) s.session = sessionutil.NewSession(s.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if s.session == nil { if s.session == nil {
return fmt.Errorf("failed to create session") return fmt.Errorf("failed to create session")
} }
@ -160,12 +160,12 @@ func (s *Server) Init() error {
s.factory.Init(Params) s.factory.Init(Params)
// Init KV // Init KV
etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
s.kv = etcdKV s.kv = etcdKV
log.Info("query coordinator try to connect etcd success") log.Info("query coordinator try to connect etcd success")
// Init ID allocator // Init ID allocator
idAllocatorKV := tsoutil.NewTSOKVBase(s.etcdCli, Params.EtcdCfg.KvRootPath, "querycoord-id-allocator") idAllocatorKV := tsoutil.NewTSOKVBase(s.etcdCli, Params.EtcdCfg.KvRootPath.GetValue(), "querycoord-id-allocator")
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV) idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
err := idAllocator.Initialize() err := idAllocator.Initialize()
if err != nil { if err != nil {

View File

@ -61,7 +61,7 @@ type ServerSuite struct {
func (suite *ServerSuite) SetupSuite() { func (suite *ServerSuite) SetupSuite() {
Params.Init() Params.Init()
Params.EtcdCfg = params.GenerateEtcdConfig() params.GenerateEtcdConfig()
suite.collections = []int64{1000, 1001} suite.collections = []int64{1000, 1001}
suite.partitions = map[int64][]int64{ suite.partitions = map[int64][]int64{

View File

@ -108,9 +108,9 @@ func (suite *ServiceSuite) SetupSuite() {
func (suite *ServiceSuite) SetupTest() { func (suite *ServiceSuite) SetupTest() {
config := params.GenerateEtcdConfig() config := params.GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
suite.store = meta.NewMetaStore(suite.kv) suite.store = meta.NewMetaStore(suite.kv)
suite.dist = meta.NewDistributionManager() suite.dist = meta.NewDistributionManager()
@ -135,7 +135,7 @@ func (suite *ServiceSuite) SetupTest() {
suite.server = &Server{ suite.server = &Server{
kv: suite.kv, kv: suite.kv,
store: suite.store, store: suite.store,
session: sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, cli), session: sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath.GetValue(), cli),
metricsCacheManager: metricsinfo.NewMetricsCacheManager(), metricsCacheManager: metricsinfo.NewMetricsCacheManager(),
dist: suite.dist, dist: suite.dist,
meta: suite.meta, meta: suite.meta,

View File

@ -118,10 +118,10 @@ func (suite *TaskSuite) SetupSuite() {
func (suite *TaskSuite) SetupTest() { func (suite *TaskSuite) SetupTest() {
config := GenerateEtcdConfig() config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config) cli, err := etcd.GetEtcdClient(config)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath) suite.kv = etcdkv.NewEtcdKV(cli, config.MetaRootPath.GetValue())
suite.store = meta.NewMetaStore(suite.kv) suite.store = meta.NewMetaStore(suite.kv)
suite.meta = meta.NewMeta(RandomIncrementIDAllocator(), suite.store) suite.meta = meta.NewMeta(RandomIncrementIDAllocator(), suite.store)
suite.dist = meta.NewDistributionManager() suite.dist = meta.NewDistributionManager()

View File

@ -451,7 +451,7 @@ func TestImpl_ShowConfigurations(t *testing.T) {
t.Run("test ShowConfigurations", func(t *testing.T) { t.Run("test ShowConfigurations", func(t *testing.T) {
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath, etcdCli) node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
pattern := "Cache" pattern := "Cache"
req := &internalpb.ShowConfigurationsRequest{ req := &internalpb.ShowConfigurationsRequest{
@ -467,7 +467,7 @@ func TestImpl_ShowConfigurations(t *testing.T) {
t.Run("test ShowConfigurations node failed", func(t *testing.T) { t.Run("test ShowConfigurations node failed", func(t *testing.T) {
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath, etcdCli) node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
node.UpdateStateCode(commonpb.StateCode_Abnormal) node.UpdateStateCode(commonpb.StateCode_Abnormal)
pattern := "Cache" pattern := "Cache"
@ -496,7 +496,7 @@ func TestImpl_GetMetrics(t *testing.T) {
defer wg.Done() defer wg.Done()
node, err := genSimpleQueryNode(ctx) node, err := genSimpleQueryNode(ctx)
assert.NoError(t, err) assert.NoError(t, err)
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath, etcdCli) node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
metricReq := make(map[string]string) metricReq := make(map[string]string)
metricReq[metricsinfo.MetricTypeKey] = "system_info" metricReq[metricsinfo.MetricTypeKey] = "system_info"

View File

@ -50,13 +50,13 @@ func newLoadIndexInfo() (*LoadIndexInfo, error) {
// TODO::xige-16 support embedded milvus // TODO::xige-16 support embedded milvus
storageType := "minio" storageType := "minio"
cAddress := C.CString(Params.MinioCfg.Address) cAddress := C.CString(Params.MinioCfg.Address.GetValue())
cBucketName := C.CString(Params.MinioCfg.BucketName) cBucketName := C.CString(Params.MinioCfg.BucketName.GetValue())
cAccessKey := C.CString(Params.MinioCfg.AccessKeyID) cAccessKey := C.CString(Params.MinioCfg.AccessKeyID.GetValue())
cAccessValue := C.CString(Params.MinioCfg.SecretAccessKey) cAccessValue := C.CString(Params.MinioCfg.SecretAccessKey.GetValue())
cRootPath := C.CString(Params.MinioCfg.RootPath) cRootPath := C.CString(Params.MinioCfg.RootPath.GetValue())
cStorageType := C.CString(storageType) cStorageType := C.CString(storageType)
cIamEndPoint := C.CString(Params.MinioCfg.IAMEndpoint) cIamEndPoint := C.CString(Params.MinioCfg.IAMEndpoint.GetValue())
defer C.free(unsafe.Pointer(cAddress)) defer C.free(unsafe.Pointer(cAddress))
defer C.free(unsafe.Pointer(cBucketName)) defer C.free(unsafe.Pointer(cBucketName))
defer C.free(unsafe.Pointer(cAccessKey)) defer C.free(unsafe.Pointer(cAccessKey))
@ -72,8 +72,8 @@ func newLoadIndexInfo() (*LoadIndexInfo, error) {
remote_root_path: cRootPath, remote_root_path: cRootPath,
storage_type: cStorageType, storage_type: cStorageType,
iam_endpoint: cIamEndPoint, iam_endpoint: cIamEndPoint,
useSSL: C.bool(Params.MinioCfg.UseSSL), useSSL: C.bool(Params.MinioCfg.UseSSL.GetAsBool()),
useIAM: C.bool(Params.MinioCfg.UseIAM), useIAM: C.bool(Params.MinioCfg.UseIAM.GetAsBool()),
} }
status := C.NewLoadIndexInfo(&cLoadIndexInfo, storageConfig) status := C.NewLoadIndexInfo(&cLoadIndexInfo, storageConfig)

View File

@ -42,7 +42,7 @@ func TestGetSystemInfoMetrics(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.NoError(t, err) assert.NoError(t, err)
defer etcdCli.Close() defer etcdCli.Close()
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath, etcdCli) node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
req := &milvuspb.GetMetricsRequest{ req := &milvuspb.GetMetricsRequest{
Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels, node.session.ServerID), Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels, node.session.ServerID),
@ -70,7 +70,7 @@ func TestGetComponentConfigurationsFailed(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.NoError(t, err) assert.NoError(t, err)
defer etcdCli.Close() defer etcdCli.Close()
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath, etcdCli) node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
req := &internalpb.ShowConfigurationsRequest{ req := &internalpb.ShowConfigurationsRequest{
Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels, node.session.ServerID), Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels, node.session.ServerID),

View File

@ -427,14 +427,14 @@ func generateAndSaveIndex(segmentID UniqueID, msgLength int, indexType, metricTy
func genStorageConfig() *indexpb.StorageConfig { func genStorageConfig() *indexpb.StorageConfig {
return &indexpb.StorageConfig{ return &indexpb.StorageConfig{
Address: Params.MinioCfg.Address, Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID, AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: Params.MinioCfg.SecretAccessKey, SecretAccessKey: Params.MinioCfg.SecretAccessKey.GetValue(),
BucketName: Params.MinioCfg.BucketName, BucketName: Params.MinioCfg.BucketName.GetValue(),
RootPath: Params.MinioCfg.RootPath, RootPath: Params.MinioCfg.RootPath.GetValue(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint, IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
UseSSL: Params.MinioCfg.UseSSL, UseSSL: Params.MinioCfg.UseSSL.GetAsBool(),
UseIAM: Params.MinioCfg.UseIAM, UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
} }
} }
@ -533,7 +533,7 @@ func genEtcdKV() (*etcdkv.EtcdKV, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
return etcdKV, nil return etcdKV, nil
} }
@ -560,11 +560,11 @@ func genLocalChunkManager() (storage.ChunkManager, error) {
func genRemoteChunkManager(ctx context.Context) (storage.ChunkManager, error) { func genRemoteChunkManager(ctx context.Context) (storage.ChunkManager, error) {
return storage.NewMinioChunkManager( return storage.NewMinioChunkManager(
ctx, ctx,
storage.Address(Params.MinioCfg.Address), storage.Address(Params.MinioCfg.Address.GetValue()),
storage.AccessKeyID(Params.MinioCfg.AccessKeyID), storage.AccessKeyID(Params.MinioCfg.AccessKeyID.GetValue()),
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey), storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey.GetValue()),
storage.UseSSL(Params.MinioCfg.UseSSL), storage.UseSSL(Params.MinioCfg.UseSSL.GetAsBool()),
storage.BucketName(Params.MinioCfg.BucketName), storage.BucketName(Params.MinioCfg.BucketName.GetValue()),
storage.CreateBucket(true)) storage.CreateBucket(true))
} }
@ -574,11 +574,11 @@ func genVectorChunkManager(ctx context.Context, col *Collection) (*storage.Vecto
rcm, err := storage.NewMinioChunkManager( rcm, err := storage.NewMinioChunkManager(
ctx, ctx,
storage.Address(Params.MinioCfg.Address), storage.Address(Params.MinioCfg.Address.GetValue()),
storage.AccessKeyID(Params.MinioCfg.AccessKeyID), storage.AccessKeyID(Params.MinioCfg.AccessKeyID.GetValue()),
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey), storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey.GetValue()),
storage.UseSSL(Params.MinioCfg.UseSSL), storage.UseSSL(Params.MinioCfg.UseSSL.GetAsBool()),
storage.BucketName(Params.MinioCfg.BucketName), storage.BucketName(Params.MinioCfg.BucketName.GetValue()),
storage.CreateBucket(true)) storage.CreateBucket(true))
if err != nil { if err != nil {
@ -1660,7 +1660,7 @@ func genSimpleQueryNodeWithMQFactory(ctx context.Context, fac dependency.Factory
log.Error("QueryNode init channel pool failed", zap.Error(err)) log.Error("QueryNode init channel pool failed", zap.Error(err))
return nil, err return nil, err
} }
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
node.etcdKV = etcdKV node.etcdKV = etcdKV
node.tSafeReplica = newTSafeReplica() node.tSafeReplica = newTSafeReplica()

View File

@ -144,7 +144,7 @@ func NewQueryNode(ctx context.Context, factory dependency.Factory) *QueryNode {
} }
func (node *QueryNode) initSession() error { func (node *QueryNode) initSession() error {
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath, node.etcdCli) node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.EtcdCfg.MetaRootPath.GetValue(), node.etcdCli)
if node.session == nil { if node.session == nil {
return fmt.Errorf("session is nil, the etcd client connection may have failed") return fmt.Errorf("session is nil, the etcd client connection may have failed")
} }
@ -216,8 +216,8 @@ func (node *QueryNode) InitSegcore() {
cThreadCoreCoefficient := C.int64_t(Params.CommonCfg.ThreadCoreCoefficient) cThreadCoreCoefficient := C.int64_t(Params.CommonCfg.ThreadCoreCoefficient)
C.InitThreadCoreCoefficient(cThreadCoreCoefficient) C.InitThreadCoreCoefficient(cThreadCoreCoefficient)
cCpuNum := C.int(hardware.GetCPUNum()) cCPUNum := C.int(hardware.GetCPUNum())
C.InitCpuNum(cCpuNum) C.InitCpuNum(cCPUNum)
initcore.InitLocalStorageConfig(Params) initcore.InitLocalStorageConfig(Params)
} }
@ -227,7 +227,7 @@ func (node *QueryNode) Init() error {
var initError error var initError error
node.initOnce.Do(func() { node.initOnce.Do(func() {
//ctx := context.Background() //ctx := context.Background()
log.Info("QueryNode session info", zap.String("metaPath", Params.EtcdCfg.MetaRootPath)) log.Info("QueryNode session info", zap.String("metaPath", Params.EtcdCfg.MetaRootPath.GetValue()))
err := node.initSession() err := node.initSession()
if err != nil { if err != nil {
log.Error("QueryNode init session failed", zap.Error(err)) log.Error("QueryNode init session failed", zap.Error(err))
@ -252,7 +252,7 @@ func (node *QueryNode) Init() error {
return return
} }
node.etcdKV = etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath) node.etcdKV = etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
log.Info("queryNode try to connect etcd success", zap.Any("MetaRootPath", Params.EtcdCfg.MetaRootPath)) log.Info("queryNode try to connect etcd success", zap.Any("MetaRootPath", Params.EtcdCfg.MetaRootPath))
cpuNum := runtime.GOMAXPROCS(0) cpuNum := runtime.GOMAXPROCS(0)

View File

@ -48,7 +48,8 @@ type queryCoordMock struct {
func setup() { func setup() {
os.Setenv("QUERY_NODE_ID", "1") os.Setenv("QUERY_NODE_ID", "1")
paramtable.Init() paramtable.Init()
Params.EtcdCfg.MetaRootPath = "/etcd/test/root/querynode" paramtable.Get().BaseTable.Save("etcd.rootPath", "/etcd/test/root")
paramtable.Get().BaseTable.Save("etcd.metaSubPath", "querynode")
} }
func initTestMeta(t *testing.T, node *QueryNode, collectionID UniqueID, segmentID UniqueID, optional ...bool) { func initTestMeta(t *testing.T, node *QueryNode, collectionID UniqueID, segmentID UniqueID, optional ...bool) {
@ -87,7 +88,7 @@ func newQueryNodeMock() *QueryNode {
if err != nil { if err != nil {
panic(err) panic(err)
} }
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
factory := newMessageStreamFactory() factory := newMessageStreamFactory()
svr := NewQueryNode(ctx, factory) svr := NewQueryNode(ctx, factory)

View File

@ -50,7 +50,7 @@ type queryShardService struct {
func newQueryShardService(ctx context.Context, metaReplica ReplicaInterface, tSafeReplica TSafeReplicaInterface, clusterService *ShardClusterService, factory dependency.Factory, scheduler *taskScheduler) (*queryShardService, error) { func newQueryShardService(ctx context.Context, metaReplica ReplicaInterface, tSafeReplica TSafeReplicaInterface, clusterService *ShardClusterService, factory dependency.Factory, scheduler *taskScheduler) (*queryShardService, error) {
// TODO we don't need the local chunk manager any more // TODO we don't need the local chunk manager any more
localChunkManager := storage.NewLocalChunkManager(storage.RootPath(Params.LocalStorageCfg.Path)) localChunkManager := storage.NewLocalChunkManager(storage.RootPath(Params.LocalStorageCfg.Path.GetValue()))
remoteChunkManager, err := factory.NewPersistentStorageChunkManager(ctx) remoteChunkManager, err := factory.NewPersistentStorageChunkManager(ctx)
if err != nil { if err != nil {
log.Ctx(ctx).Warn("failed to init remote chunk manager", zap.Error(err)) log.Ctx(ctx).Warn("failed to init remote chunk manager", zap.Error(err))

View File

@ -51,7 +51,7 @@ func newShardClusterService(client *clientv3.Client, session *sessionutil.Sessio
// addShardCluster adds shardCluster into service. // addShardCluster adds shardCluster into service.
func (s *ShardClusterService) addShardCluster(collectionID, replicaID int64, vchannelName string, version int64) { func (s *ShardClusterService) addShardCluster(collectionID, replicaID int64, vchannelName string, version int64) {
nodeDetector := NewEtcdShardNodeDetector(s.client, path.Join(Params.EtcdCfg.MetaRootPath, ReplicaMetaPrefix), nodeDetector := NewEtcdShardNodeDetector(s.client, path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), ReplicaMetaPrefix),
func() (map[int64]string, error) { func() (map[int64]string, error) {
result := make(map[int64]string) result := make(map[int64]string)
sessions, _, err := s.session.GetSessions(typeutil.QueryNodeRole) sessions, _, err := s.session.GetSessions(typeutil.QueryNodeRole)
@ -64,7 +64,7 @@ func (s *ShardClusterService) addShardCluster(collectionID, replicaID int64, vch
return result, nil return result, nil
}) })
segmentDetector := NewEtcdShardSegmentDetector(s.client, path.Join(Params.EtcdCfg.MetaRootPath, util.SegmentMetaPrefix, strconv.FormatInt(collectionID, 10))) segmentDetector := NewEtcdShardSegmentDetector(s.client, path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), util.SegmentMetaPrefix, strconv.FormatInt(collectionID, 10)))
cs := NewShardCluster(collectionID, replicaID, vchannelName, version, nodeDetector, segmentDetector, cs := NewShardCluster(collectionID, replicaID, vchannelName, version, nodeDetector, segmentDetector,
func(nodeID int64, addr string) shardQueryNode { func(nodeID int64, addr string) shardQueryNode {

View File

@ -76,8 +76,8 @@ func (p *proxyClientManager) AddProxyClient(session *sessionutil.Session) {
go p.connect(session) go p.connect(session)
} }
// GetProxyNumber returns number of proxy clients. // GetProxyCount returns number of proxy clients.
func (p *proxyClientManager) GetProxyNumber() int { func (p *proxyClientManager) GetProxyCount() int {
p.lock.Lock() p.lock.Lock()
defer p.lock.Unlock() defer p.lock.Unlock()

View File

@ -92,7 +92,7 @@ func (p *proxyManager) WatchProxy() error {
eventCh := p.etcdCli.Watch( eventCh := p.etcdCli.Watch(
p.ctx, p.ctx,
path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot, typeutil.ProxyRole), path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
clientv3.WithPrefix(), clientv3.WithPrefix(),
clientv3.WithCreatedNotify(), clientv3.WithCreatedNotify(),
clientv3.WithPrevKV(), clientv3.WithPrevKV(),
@ -182,7 +182,7 @@ func (p *proxyManager) parseSession(value []byte) (*sessionutil.Session, error)
func (p *proxyManager) getSessionsOnEtcd(ctx context.Context) ([]*sessionutil.Session, int64, error) { func (p *proxyManager) getSessionsOnEtcd(ctx context.Context) ([]*sessionutil.Session, int64, error) {
resp, err := p.etcdCli.Get( resp, err := p.etcdCli.Get(
ctx, ctx,
path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot, typeutil.ProxyRole), path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
clientv3.WithPrefix(), clientv3.WithPrefix(),
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend), clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend),
) )

View File

@ -40,7 +40,7 @@ func TestProxyManager(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel() defer cancel()
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot)
etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
defer etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix()) defer etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
s1 := sessionutil.Session{ s1 := sessionutil.Session{
@ -109,7 +109,7 @@ func TestProxyManager_ErrCompacted(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel() defer cancel()
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot) sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot)
f1 := func(sess []*sessionutil.Session) { f1 := func(sess []*sessionutil.Session) {
t.Log("get sessions num", len(sess)) t.Log("get sessions num", len(sess))
} }
@ -117,7 +117,7 @@ func TestProxyManager_ErrCompacted(t *testing.T) {
eventCh := pm.etcdCli.Watch( eventCh := pm.etcdCli.Watch(
pm.ctx, pm.ctx,
path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot, typeutil.ProxyRole), path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
clientv3.WithPrefix(), clientv3.WithPrefix(),
clientv3.WithCreatedNotify(), clientv3.WithCreatedNotify(),
clientv3.WithPrevKV(), clientv3.WithPrevKV(),

View File

@ -648,7 +648,7 @@ func (q *QuotaCenter) setRates() error {
switch q.rateAllocateStrategy { switch q.rateAllocateStrategy {
case Average: case Average:
map2List = func() []*internalpb.Rate { map2List = func() []*internalpb.Rate {
proxyNum := q.proxies.GetProxyNumber() proxyNum := q.proxies.GetProxyCount()
if proxyNum == 0 { if proxyNum == 0 {
return nil return nil
} }

View File

@ -331,7 +331,7 @@ func (c *Core) SetEtcdClient(etcdClient *clientv3.Client) {
} }
func (c *Core) initSession() error { func (c *Core) initSession() error {
c.session = sessionutil.NewSession(c.ctx, Params.EtcdCfg.MetaRootPath, c.etcdCli) c.session = sessionutil.NewSession(c.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), c.etcdCli)
if c.session == nil { if c.session == nil {
return fmt.Errorf("session is nil, the etcd client connection may have failed") return fmt.Errorf("session is nil, the etcd client connection may have failed")
} }
@ -357,11 +357,11 @@ func (c *Core) initMetaTable() error {
var ss *kvmetestore.SuffixSnapshot var ss *kvmetestore.SuffixSnapshot
var err error var err error
if metaKV, err = c.metaKVCreator(Params.EtcdCfg.MetaRootPath); err != nil { if metaKV, err = c.metaKVCreator(Params.EtcdCfg.MetaRootPath.GetValue()); err != nil {
return err return err
} }
if ss, err = kvmetestore.NewSuffixSnapshot(metaKV, kvmetestore.SnapshotsSep, Params.EtcdCfg.MetaRootPath, kvmetestore.SnapshotPrefix); err != nil { if ss, err = kvmetestore.NewSuffixSnapshot(metaKV, kvmetestore.SnapshotsSep, Params.EtcdCfg.MetaRootPath.GetValue(), kvmetestore.SnapshotPrefix); err != nil {
return err return err
} }
@ -389,7 +389,7 @@ func (c *Core) initMetaTable() error {
} }
func (c *Core) initIDAllocator() error { func (c *Core) initIDAllocator() error {
tsoKV := tsoutil.NewTSOKVBase(c.etcdCli, Params.EtcdCfg.KvRootPath, globalIDAllocatorSubPath) tsoKV := tsoutil.NewTSOKVBase(c.etcdCli, Params.EtcdCfg.KvRootPath.GetValue(), globalIDAllocatorSubPath)
idAllocator := allocator.NewGlobalIDAllocator(globalIDAllocatorKey, tsoKV) idAllocator := allocator.NewGlobalIDAllocator(globalIDAllocatorKey, tsoKV)
if err := idAllocator.Initialize(); err != nil { if err := idAllocator.Initialize(); err != nil {
return err return err
@ -399,7 +399,7 @@ func (c *Core) initIDAllocator() error {
} }
func (c *Core) initTSOAllocator() error { func (c *Core) initTSOAllocator() error {
tsoKV := tsoutil.NewTSOKVBase(c.etcdCli, Params.EtcdCfg.KvRootPath, globalTSOAllocatorSubPath) tsoKV := tsoutil.NewTSOKVBase(c.etcdCli, Params.EtcdCfg.KvRootPath.GetValue(), globalTSOAllocatorSubPath)
tsoAllocator := tso.NewGlobalTSOAllocator(globalTSOAllocatorKey, tsoKV) tsoAllocator := tso.NewGlobalTSOAllocator(globalTSOAllocatorKey, tsoKV)
if err := tsoAllocator.Initialize(); err != nil { if err := tsoAllocator.Initialize(); err != nil {
return err return err
@ -410,7 +410,7 @@ func (c *Core) initTSOAllocator() error {
} }
func (c *Core) initImportManager() error { func (c *Core) initImportManager() error {
impTaskKv, err := c.metaKVCreator(Params.EtcdCfg.KvRootPath) impTaskKv, err := c.metaKVCreator(Params.EtcdCfg.KvRootPath.GetValue())
if err != nil { if err != nil {
return err return err
} }

View File

@ -1324,13 +1324,12 @@ func TestCore_startTimeTickLoop(t *testing.T) {
// make sure the main functions work well when EnableActiveStandby=true // make sure the main functions work well when EnableActiveStandby=true
func TestRootcoord_EnableActiveStandby(t *testing.T) { func TestRootcoord_EnableActiveStandby(t *testing.T) {
Params.Init()
Params.RootCoordCfg.EnableActiveStandby = true
randVal := rand.Int() randVal := rand.Int()
Params.Init()
Params.BaseTable.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal))
Params.RootCoordCfg.EnableActiveStandby = true
Params.CommonCfg.RootCoordTimeTick = fmt.Sprintf("rootcoord-time-tick-%d", randVal) Params.CommonCfg.RootCoordTimeTick = fmt.Sprintf("rootcoord-time-tick-%d", randVal)
Params.CommonCfg.RootCoordStatistics = fmt.Sprintf("rootcoord-statistics-%d", randVal) Params.CommonCfg.RootCoordStatistics = fmt.Sprintf("rootcoord-statistics-%d", randVal)
Params.EtcdCfg.MetaRootPath = fmt.Sprintf("/%d/%s", randVal, Params.EtcdCfg.MetaRootPath)
Params.EtcdCfg.KvRootPath = fmt.Sprintf("/%d/%s", randVal, Params.EtcdCfg.KvRootPath)
Params.CommonCfg.RootCoordSubName = fmt.Sprintf("subname-%d", randVal) Params.CommonCfg.RootCoordSubName = fmt.Sprintf("subname-%d", randVal)
Params.CommonCfg.RootCoordDml = fmt.Sprintf("rootcoord-dml-test-%d", randVal) Params.CommonCfg.RootCoordDml = fmt.Sprintf("rootcoord-dml-test-%d", randVal)
Params.CommonCfg.RootCoordDelta = fmt.Sprintf("rootcoord-delta-test-%d", randVal) Params.CommonCfg.RootCoordDelta = fmt.Sprintf("rootcoord-delta-test-%d", randVal)

View File

@ -14,18 +14,18 @@ type ChunkManagerFactory struct {
func NewChunkManagerFactoryWithParam(params *paramtable.ComponentParam) *ChunkManagerFactory { func NewChunkManagerFactoryWithParam(params *paramtable.ComponentParam) *ChunkManagerFactory {
if params.CommonCfg.StorageType == "local" { if params.CommonCfg.StorageType == "local" {
return NewChunkManagerFactory("local", RootPath(params.LocalStorageCfg.Path)) return NewChunkManagerFactory("local", RootPath(params.LocalStorageCfg.Path.GetValue()))
} }
return NewChunkManagerFactory("minio", return NewChunkManagerFactory("minio",
RootPath(params.MinioCfg.RootPath), RootPath(params.MinioCfg.RootPath.GetValue()),
Address(params.MinioCfg.Address), Address(params.MinioCfg.Address.GetValue()),
AccessKeyID(params.MinioCfg.AccessKeyID), AccessKeyID(params.MinioCfg.AccessKeyID.GetValue()),
SecretAccessKeyID(params.MinioCfg.SecretAccessKey), SecretAccessKeyID(params.MinioCfg.SecretAccessKey.GetValue()),
UseSSL(params.MinioCfg.UseSSL), UseSSL(params.MinioCfg.UseSSL.GetAsBool()),
BucketName(params.MinioCfg.BucketName), BucketName(params.MinioCfg.BucketName.GetValue()),
UseIAM(params.MinioCfg.UseIAM), UseIAM(params.MinioCfg.UseIAM.GetAsBool()),
CloudProvider(params.MinioCfg.CloudProvider), CloudProvider(params.MinioCfg.CloudProvider.GetValue()),
IAMEndpoint(params.MinioCfg.IAMEndpoint), IAMEndpoint(params.MinioCfg.IAMEndpoint.GetValue()),
CreateBucket(true)) CreateBucket(true))
} }

View File

@ -29,7 +29,6 @@ import (
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/storage/gcp" "github.com/milvus-io/milvus/internal/storage/gcp"
"github.com/milvus-io/milvus/internal/util/errorutil" "github.com/milvus-io/milvus/internal/util/errorutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"github.com/minio/minio-go/v7" "github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials" "github.com/minio/minio-go/v7/pkg/credentials"
@ -41,6 +40,11 @@ var (
ErrNoSuchKey = errors.New("NoSuchKey") ErrNoSuchKey = errors.New("NoSuchKey")
) )
const (
CloudProviderGCP = "gcp"
CloudProviderAWS = "aws"
)
func WrapErrNoSuchKey(key string) error { func WrapErrNoSuchKey(key string) error {
return fmt.Errorf("%w(key=%s)", ErrNoSuchKey, key) return fmt.Errorf("%w(key=%s)", ErrNoSuchKey, key)
} }
@ -74,7 +78,7 @@ func newMinioChunkManagerWithConfig(ctx context.Context, c *config) (*MinioChunk
var newMinioFn = minio.New var newMinioFn = minio.New
switch c.cloudProvider { switch c.cloudProvider {
case paramtable.CloudProviderGCP: case CloudProviderGCP:
newMinioFn = gcp.NewMinioClient newMinioFn = gcp.NewMinioClient
if !c.useIAM { if !c.useIAM {
creds = credentials.NewStaticV2(c.accessKeyID, c.secretAccessKeyID, "") creds = credentials.NewStaticV2(c.accessKeyID, c.secretAccessKeyID, "")

View File

@ -26,10 +26,10 @@ func GetEmbedEtcdClient() (*clientv3.Client, error) {
// InitEtcdServer initializes embedded etcd server singleton. // InitEtcdServer initializes embedded etcd server singleton.
func InitEtcdServer(etcdCfg *paramtable.EtcdConfig) error { func InitEtcdServer(etcdCfg *paramtable.EtcdConfig) error {
if etcdCfg.UseEmbedEtcd { if etcdCfg.UseEmbedEtcd.GetAsBool() {
var initError error var initError error
initOnce.Do(func() { initOnce.Do(func() {
path := etcdCfg.ConfigPath path := etcdCfg.ConfigPath.GetValue()
var cfg *embed.Config var cfg *embed.Config
if len(path) > 0 { if len(path) > 0 {
cfgFromFile, err := embed.ConfigFromFile(path) cfgFromFile, err := embed.ConfigFromFile(path)
@ -40,16 +40,16 @@ func InitEtcdServer(etcdCfg *paramtable.EtcdConfig) error {
} else { } else {
cfg = embed.NewConfig() cfg = embed.NewConfig()
} }
cfg.Dir = etcdCfg.DataDir cfg.Dir = etcdCfg.DataDir.GetValue()
cfg.LogOutputs = []string{etcdCfg.EtcdLogPath} cfg.LogOutputs = []string{etcdCfg.EtcdLogPath.GetValue()}
cfg.LogLevel = etcdCfg.EtcdLogLevel cfg.LogLevel = etcdCfg.EtcdLogLevel.GetValue()
e, err := embed.StartEtcd(cfg) e, err := embed.StartEtcd(cfg)
if err != nil { if err != nil {
log.Error("failed to init embedded Etcd server", zap.Error(err)) log.Error("failed to init embedded Etcd server", zap.Error(err))
initError = err initError = err
} }
etcdServer = e etcdServer = e
log.Info("finish init Etcd config", zap.String("path", path), zap.String("data", etcdCfg.DataDir)) log.Info("finish init Etcd config", zap.String("path", path), zap.String("data", etcdCfg.DataDir.GetValue()))
}) })
return initError return initError
} }

View File

@ -35,13 +35,13 @@ var (
// GetEtcdClient returns etcd client // GetEtcdClient returns etcd client
func GetEtcdClient(cfg *paramtable.EtcdConfig) (*clientv3.Client, error) { func GetEtcdClient(cfg *paramtable.EtcdConfig) (*clientv3.Client, error) {
if cfg.UseEmbedEtcd { if cfg.UseEmbedEtcd.GetAsBool() {
return GetEmbedEtcdClient() return GetEmbedEtcdClient()
} }
if cfg.EtcdUseSSL { if cfg.EtcdUseSSL.GetAsBool() {
return GetRemoteEtcdSSLClient(cfg.Endpoints, cfg.EtcdTLSCert, cfg.EtcdTLSKey, cfg.EtcdTLSCACert, cfg.EtcdTLSMinVersion) return GetRemoteEtcdSSLClient(cfg.Endpoints.GetAsStrings(), cfg.EtcdTLSCert.GetValue(), cfg.EtcdTLSKey.GetValue(), cfg.EtcdTLSCACert.GetValue(), cfg.EtcdTLSMinVersion.GetValue())
} }
return GetRemoteEtcdClient(cfg.Endpoints) return GetRemoteEtcdClient(cfg.Endpoints.GetAsStrings())
} }
// GetRemoteEtcdClient returns client of remote etcd by given endpoints // GetRemoteEtcdClient returns client of remote etcd by given endpoints

View File

@ -23,6 +23,7 @@ import (
"path" "path"
"testing" "testing"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -30,12 +31,13 @@ import (
var Params paramtable.ServiceParam var Params paramtable.ServiceParam
func TestEtcd(t *testing.T) { func TestEtcd(t *testing.T) {
t.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode)
t.Setenv("ETCD_USE_EMBED", "true")
Params.Init() Params.Init()
Params.EtcdCfg.UseEmbedEtcd = true Params.BaseTable.Save("etcd.data.dir", "/tmp/data")
Params.EtcdCfg.DataDir = "/tmp/data"
err := InitEtcdServer(&Params.EtcdCfg) err := InitEtcdServer(&Params.EtcdCfg)
assert.NoError(t, err) assert.NoError(t, err)
defer os.RemoveAll(Params.EtcdCfg.DataDir) defer os.RemoveAll(Params.EtcdCfg.DataDir.GetValue())
defer StopEtcdServer() defer StopEtcdServer()
etcdCli, err := GetEtcdClient(&Params.EtcdCfg) etcdCli, err := GetEtcdClient(&Params.EtcdCfg)
@ -50,26 +52,26 @@ func TestEtcd(t *testing.T) {
assert.False(t, resp.Count < 1) assert.False(t, resp.Count < 1)
assert.Equal(t, string(resp.Kvs[0].Value), "value") assert.Equal(t, string(resp.Kvs[0].Value), "value")
Params.EtcdCfg.UseEmbedEtcd = false t.Setenv("ETCD_USE_EMBED", "false")
Params.EtcdCfg.EtcdUseSSL = true Params.Init()
Params.EtcdCfg.EtcdTLSMinVersion = "1.3" Params.BaseTable.Save("etcd.ssl.enabled", "true")
Params.EtcdCfg.EtcdTLSCACert = "../../../configs/cert/ca.pem" Params.BaseTable.Save("etcd.ssl.tlsMinVersion", "1.3")
Params.EtcdCfg.EtcdTLSCert = "../../../configs/cert/client.pem" Params.BaseTable.Save("etcd.ssl.tlsCACert", "../../../configs/cert/ca.pem")
Params.EtcdCfg.EtcdTLSKey = "../../../configs/cert/client.key" Params.BaseTable.Save("etcd.ssl.tlsCert", "../../../configs/cert/client.pem")
etcdCli, err = GetEtcdClient(&Params.EtcdCfg) Params.BaseTable.Save("etcd.ssl.tlsKey", "../../../configs/cert/client.key")
assert.NoError(t, err) assert.NoError(t, err)
Params.EtcdCfg.EtcdTLSMinVersion = "some not right word" Params.BaseTable.Save("etcd.ssl.tlsMinVersion", "some not right word")
etcdCli, err = GetEtcdClient(&Params.EtcdCfg) etcdCli, err = GetEtcdClient(&Params.EtcdCfg)
assert.NotNil(t, err) assert.NotNil(t, err)
Params.EtcdCfg.EtcdTLSMinVersion = "1.2" Params.BaseTable.Save("etcd.ssl.tlsMinVersion", "1.2")
Params.EtcdCfg.EtcdTLSCACert = "wrong/file" Params.BaseTable.Save("etcd.ssl.tlsCACert", "wrong/file")
etcdCli, err = GetEtcdClient(&Params.EtcdCfg) etcdCli, err = GetEtcdClient(&Params.EtcdCfg)
assert.NotNil(t, err) assert.NotNil(t, err)
Params.EtcdCfg.EtcdTLSCACert = "../../../configs/cert/ca.pem" Params.BaseTable.Save("etcd.ssl.tlsCACert", "../../../configs/cert/ca.pem")
Params.EtcdCfg.EtcdTLSCert = "wrong/file" Params.BaseTable.Save("etcd.ssl.tlsCert", "wrong/file")
assert.NotNil(t, err) assert.NotNil(t, err)
} }

View File

@ -295,14 +295,14 @@ func genStorageConfig() *indexpb.StorageConfig {
}) })
return &indexpb.StorageConfig{ return &indexpb.StorageConfig{
Address: Params.MinioCfg.Address, Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID, AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: Params.MinioCfg.SecretAccessKey, SecretAccessKey: Params.MinioCfg.SecretAccessKey.GetValue(),
BucketName: Params.MinioCfg.BucketName, BucketName: Params.MinioCfg.BucketName.GetValue(),
RootPath: Params.MinioCfg.RootPath, RootPath: Params.MinioCfg.RootPath.GetValue(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint, IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
UseSSL: Params.MinioCfg.UseSSL, UseSSL: Params.MinioCfg.UseSSL.GetAsBool(),
UseIAM: Params.MinioCfg.UseIAM, UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
} }
} }

View File

@ -83,7 +83,10 @@ type BaseTable struct {
// NewBaseTableFromYamlOnly only used in migration tool. // NewBaseTableFromYamlOnly only used in migration tool.
// Maybe we shouldn't limit the configDir internally. // Maybe we shouldn't limit the configDir internally.
func NewBaseTableFromYamlOnly(yaml string) *BaseTable { func NewBaseTableFromYamlOnly(yaml string) *BaseTable {
mgr, _ := config.Init(config.WithFilesSource(yaml)) mgr, _ := config.Init(config.WithFilesSource(&config.FileInfo{
Filepath: yaml,
RefreshInterval: 10 * time.Second,
}))
gp := &BaseTable{mgr: mgr, YamlFile: yaml} gp := &BaseTable{mgr: mgr, YamlFile: yaml}
return gp return gp
} }
@ -126,7 +129,12 @@ func (gp *BaseTable) initConfigsFromLocal(formatter func(key string) string) {
gp.configDir = gp.initConfPath() gp.configDir = gp.initConfPath()
configFilePath := gp.configDir + "/" + gp.YamlFile configFilePath := gp.configDir + "/" + gp.YamlFile
gp.mgr, err = config.Init(config.WithEnvSource(formatter), config.WithFilesSource(configFilePath)) gp.mgr, err = config.Init(
config.WithEnvSource(formatter),
config.WithFilesSource(&config.FileInfo{
Filepath: configFilePath,
RefreshInterval: 10 * time.Second,
}))
if err != nil { if err != nil {
log.Warn("init baseTable with file failed", zap.String("configFile", configFilePath), zap.Error(err)) log.Warn("init baseTable with file failed", zap.String("configFile", configFilePath), zap.Error(err))
return return
@ -147,11 +155,13 @@ func (gp *BaseTable) initConfigsFromRemote(formatter func(key string) string) {
configFilePath := gp.configDir + "/" + gp.YamlFile configFilePath := gp.configDir + "/" + gp.YamlFile
gp.mgr, err = config.Init(config.WithEnvSource(formatter), gp.mgr, err = config.Init(config.WithEnvSource(formatter),
config.WithFilesSource(configFilePath), config.WithFilesSource(&config.FileInfo{
Filepath: configFilePath,
RefreshInterval: 10 * time.Second,
}),
config.WithEtcdSource(&config.EtcdInfo{ config.WithEtcdSource(&config.EtcdInfo{
Endpoints: strings.Split(endpoints, ","), Endpoints: strings.Split(endpoints, ","),
KeyPrefix: rootPath, KeyPrefix: rootPath,
RefreshMode: config.ModeInterval,
RefreshInterval: 10 * time.Second, RefreshInterval: 10 * time.Second,
})) }))
if err != nil { if err != nil {
@ -183,7 +193,7 @@ func (gp *BaseTable) initConfPath() string {
} }
func (gp *BaseTable) Configs() map[string]string { func (gp *BaseTable) Configs() map[string]string {
return gp.mgr.Configs() return gp.mgr.GetConfigs()
} }
// Load loads an object with @key. // Load loads an object with @key.
@ -241,6 +251,10 @@ func (gp *BaseTable) GetConfigSubSet(pattern string) map[string]string {
return gp.mgr.GetConfigsByPattern(pattern, false) return gp.mgr.GetConfigsByPattern(pattern, false)
} }
func (gp *BaseTable) GetAll() map[string]string {
return gp.mgr.GetConfigs()
}
// For compatible reason, only visiable for Test // For compatible reason, only visiable for Test
func (gp *BaseTable) Remove(key string) error { func (gp *BaseTable) Remove(key string) error {
gp.mgr.DeleteConfig(key) gp.mgr.DeleteConfig(key)

View File

@ -29,7 +29,7 @@ func TestMain(m *testing.M) {
func TestBaseTable_GetConfigSubSet(t *testing.T) { func TestBaseTable_GetConfigSubSet(t *testing.T) {
prefix := "rootcoord." prefix := "rootcoord."
configs := baseParams.mgr.Configs() configs := baseParams.mgr.GetConfigs()
configsWithPrefix := make(map[string]string) configsWithPrefix := make(map[string]string)
for k, v := range configs { for k, v := range configs {

View File

@ -95,15 +95,15 @@ func (p *ComponentParam) Init() {
} }
func (p *ComponentParam) RocksmqEnable() bool { func (p *ComponentParam) RocksmqEnable() bool {
return p.RocksmqCfg.Path != "" return p.RocksmqCfg.Path.GetValue() != ""
} }
func (p *ComponentParam) PulsarEnable() bool { func (p *ComponentParam) PulsarEnable() bool {
return p.PulsarCfg.Address != "" return p.PulsarCfg.Address.GetValue() != ""
} }
func (p *ComponentParam) KafkaEnable() bool { func (p *ComponentParam) KafkaEnable() bool {
return p.KafkaCfg.Address != "" return p.KafkaCfg.Address.GetValue() != ""
} }
// ///////////////////////////////////////////////////////////////////////////// // /////////////////////////////////////////////////////////////////////////////

View File

@ -32,10 +32,10 @@ func TestComponentParam(t *testing.T) {
t.Run("test kafkaConfig", func(t *testing.T) { t.Run("test kafkaConfig", func(t *testing.T) {
params := params.ServiceParam.KafkaCfg params := params.ServiceParam.KafkaCfg
producerConfig := params.ProducerExtraConfig producerConfig := params.ProducerExtraConfig.GetValue()
assert.Equal(t, "dc", producerConfig["client.id"]) assert.Equal(t, "dc", producerConfig["client.id"])
consumerConfig := params.ConsumerExtraConfig consumerConfig := params.ConsumerExtraConfig.GetValue()
assert.Equal(t, "dc1", consumerConfig["client.id"]) assert.Equal(t, "dc1", consumerConfig["client.id"])
}) })

View File

@ -10,11 +10,100 @@
// or implied. See the License for the specific language governing permissions and limitations under the License. // or implied. See the License for the specific language governing permissions and limitations under the License.
package paramtable package paramtable
import (
"strconv"
"strings"
"github.com/milvus-io/milvus/internal/config"
)
type ParamItem struct { type ParamItem struct {
Key string Key string // which should be named as "A.B.C"
EnvKey string
Version string Version string
Doc string Doc string
DefaultValue string DefaultValue string
Refreshable bool PanicIfEmpty bool
Formatter func(originValue string) string
manager *config.Manager
}
func (pi *ParamItem) Init(manager *config.Manager) {
pi.manager = manager
}
// Get original value with error
func (pi *ParamItem) get() (string, error) {
ret, err := pi.manager.GetConfig(pi.Key)
if err != nil {
ret = pi.DefaultValue
}
if pi.Formatter == nil {
return ret, err
}
return pi.Formatter(ret), err
}
func (pi *ParamItem) GetValue() string {
v, _ := pi.get()
return v
}
func (pi *ParamItem) GetAsStrings() []string {
return getAndConvert(pi, func(value string) ([]string, error) {
return strings.Split(value, ","), nil
}, []string{})
}
func (pi *ParamItem) GetAsBool() bool {
return getAndConvert(pi, strconv.ParseBool, false)
}
func (pi *ParamItem) GetAsInt() int {
return getAndConvert(pi, strconv.Atoi, 0)
}
type CompositeParamItem struct {
Items []*ParamItem
Format func(map[string]string) string
}
func (cpi *CompositeParamItem) GetValue() string {
kvs := make(map[string]string, len(cpi.Items))
for _, v := range cpi.Items {
kvs[v.Key] = v.GetValue()
}
return cpi.Format(kvs)
}
type ParamGroup struct {
KeyPrefix string // which should be named as "A.B."
Version string
Doc string
GetFunc func() map[string]string
manager *config.Manager
}
func (pg *ParamGroup) Init(manager *config.Manager) {
pg.manager = manager
}
func (pg *ParamGroup) GetValue() map[string]string {
if pg.GetFunc != nil {
return pg.GetFunc()
}
values := pg.manager.GetConfigsByPattern(pg.KeyPrefix, false)
return values
}
func getAndConvert[T any](pi *ParamItem, converter func(input string) (T, error), defaultValue T) T {
v, _ := pi.get()
t, err := converter(v)
if err != nil {
return defaultValue
}
return t
} }

View File

@ -56,156 +56,178 @@ type ServiceParam struct {
func (p *ServiceParam) Init() { func (p *ServiceParam) Init() {
p.BaseTable.Init() p.BaseTable.Init()
p.LocalStorageCfg.init(&p.BaseTable) p.LocalStorageCfg.Init(&p.BaseTable)
p.MetaStoreCfg.init(&p.BaseTable) p.MetaStoreCfg.Init(&p.BaseTable)
p.EtcdCfg.init(&p.BaseTable) p.EtcdCfg.Init(&p.BaseTable)
if p.MetaStoreCfg.MetaStoreType == util.MetaStoreTypeMysql { if p.MetaStoreCfg.MetaStoreType == util.MetaStoreTypeMysql {
log.Debug("Mysql protocol is used as meta store") log.Debug("Mysql protocol is used as meta store")
p.DBCfg.init(&p.BaseTable) p.DBCfg.Init(&p.BaseTable)
} }
p.PulsarCfg.init(&p.BaseTable) p.PulsarCfg.Init(&p.BaseTable)
p.KafkaCfg.init(&p.BaseTable) p.KafkaCfg.Init(&p.BaseTable)
p.RocksmqCfg.init(&p.BaseTable) p.RocksmqCfg.Init(&p.BaseTable)
p.MinioCfg.init(&p.BaseTable) p.MinioCfg.Init(&p.BaseTable)
} }
// ///////////////////////////////////////////////////////////////////////////// // /////////////////////////////////////////////////////////////////////////////
// --- etcd --- // --- etcd ---
type EtcdConfig struct { type EtcdConfig struct {
Base *BaseTable
// --- ETCD --- // --- ETCD ---
Endpoints []string Endpoints ParamItem
MetaRootPath string RootPath ParamItem
KvRootPath string MetaSubPath ParamItem
EtcdLogLevel string KvSubPath ParamItem
EtcdLogPath string MetaRootPath CompositeParamItem
EtcdUseSSL bool KvRootPath CompositeParamItem
EtcdTLSCert string EtcdLogLevel ParamItem
EtcdTLSKey string EtcdLogPath ParamItem
EtcdTLSCACert string EtcdUseSSL ParamItem
EtcdTLSMinVersion string EtcdTLSCert ParamItem
EtcdTLSKey ParamItem
EtcdTLSCACert ParamItem
EtcdTLSMinVersion ParamItem
// --- Embed ETCD --- // --- Embed ETCD ---
UseEmbedEtcd bool UseEmbedEtcd ParamItem
ConfigPath string ConfigPath ParamItem
DataDir string DataDir ParamItem
} }
func (p *EtcdConfig) init(base *BaseTable) { func (p *EtcdConfig) Init(base *BaseTable) {
p.Base = base p.Endpoints = ParamItem{
p.LoadCfgToMemory() Key: "etcd.endpoints",
} Version: "2.0.0",
PanicIfEmpty: true,
func (p *EtcdConfig) LoadCfgToMemory() {
p.initUseEmbedEtcd()
if p.UseEmbedEtcd {
p.initConfigPath()
p.initDataDir()
} else {
p.initEndpoints()
} }
p.initMetaRootPath() p.Endpoints.Init(base.mgr)
p.initKvRootPath()
p.initEtcdLogLevel()
p.initEtcdLogPath()
p.initEtcdUseSSL()
p.initEtcdTLSCert()
p.initEtcdTLSKey()
p.initEtcdTLSCACert()
p.initEtcdTLSMinVersion()
}
func (p *EtcdConfig) initUseEmbedEtcd() { p.UseEmbedEtcd = ParamItem{
p.UseEmbedEtcd = p.Base.ParseBool("etcd.use.embed", false) Key: "etcd.use.embed",
if p.UseEmbedEtcd && (os.Getenv(metricsinfo.DeployModeEnvKey) != metricsinfo.StandaloneDeployMode) { DefaultValue: "false",
Version: "2.1.0",
}
p.UseEmbedEtcd.Init(base.mgr)
if p.UseEmbedEtcd.GetAsBool() && (os.Getenv(metricsinfo.DeployModeEnvKey) != metricsinfo.StandaloneDeployMode) {
panic("embedded etcd can not be used under distributed mode") panic("embedded etcd can not be used under distributed mode")
} }
}
func (p *EtcdConfig) initConfigPath() { if p.UseEmbedEtcd.GetAsBool() {
addr := p.Base.LoadWithDefault("etcd.config.path", "") p.ConfigPath = ParamItem{
p.ConfigPath = addr Key: "etcd.config.path",
} DefaultValue: "",
Version: "2.1.0",
func (p *EtcdConfig) initDataDir() {
addr := p.Base.LoadWithDefault("etcd.data.dir", "default.etcd")
p.DataDir = addr
}
func (p *EtcdConfig) initEndpoints() {
endpoints, err := p.Base.Load("etcd.endpoints")
if err != nil {
panic(err)
} }
p.Endpoints = strings.Split(endpoints, ",") p.ConfigPath.Init(base.mgr)
}
func (p *EtcdConfig) initMetaRootPath() { p.DataDir = ParamItem{
rootPath, err := p.Base.Load("etcd.rootPath") Key: "etcd.data.dir",
if err != nil { DefaultValue: "default.etcd",
panic(err) Version: "2.1.0",
} }
subPath, err := p.Base.Load("etcd.metaSubPath") p.DataDir.Init(base.mgr)
if err != nil { } else {
panic(err) p.Endpoints = ParamItem{
Key: "etcd.endpoints",
Version: "2.0.0",
PanicIfEmpty: true,
} }
p.MetaRootPath = path.Join(rootPath, subPath) p.Endpoints.Init(base.mgr)
}
func (p *EtcdConfig) initKvRootPath() {
rootPath, err := p.Base.Load("etcd.rootPath")
if err != nil {
panic(err)
} }
subPath, err := p.Base.Load("etcd.kvSubPath")
if err != nil { p.RootPath = ParamItem{
panic(err) Key: "etcd.rootPath",
Version: "2.0.0",
PanicIfEmpty: true,
} }
p.KvRootPath = path.Join(rootPath, subPath) p.RootPath.Init(base.mgr)
}
func (p *EtcdConfig) initEtcdLogLevel() { p.MetaSubPath = ParamItem{
p.EtcdLogLevel = p.Base.LoadWithDefault("etcd.log.level", defaultEtcdLogLevel) Key: "etcd.metaSubPath",
} Version: "2.0.0",
PanicIfEmpty: true,
}
p.MetaSubPath.Init(base.mgr)
func (p *EtcdConfig) initEtcdLogPath() { p.MetaRootPath = CompositeParamItem{
p.EtcdLogPath = p.Base.LoadWithDefault("etcd.log.path", defaultEtcdLogPath) Items: []*ParamItem{&p.RootPath, &p.MetaSubPath},
} Format: func(kvs map[string]string) string {
return path.Join(kvs["etcd.rootPath"], kvs["etcd.metaSubPath"])
},
}
func (p *EtcdConfig) initEtcdUseSSL() { p.KvSubPath = ParamItem{
p.EtcdUseSSL = p.Base.ParseBool("etcd.ssl.enabled", false) Key: "etcd.kvSubPath",
} Version: "2.0.0",
PanicIfEmpty: true,
}
p.KvSubPath.Init(base.mgr)
func (p *EtcdConfig) initEtcdTLSCert() { p.KvRootPath = CompositeParamItem{
p.EtcdTLSCert = p.Base.LoadWithDefault("etcd.ssl.tlsCert", "") Items: []*ParamItem{&p.RootPath, &p.KvSubPath},
} Format: func(kvs map[string]string) string {
return path.Join(kvs["etcd.rootPath"], kvs["etcd.kvSubPath"])
},
}
func (p *EtcdConfig) initEtcdTLSKey() { p.EtcdLogLevel = ParamItem{
p.EtcdTLSKey = p.Base.LoadWithDefault("etcd.ssl.tlsKey", "") Key: "etcd.log.level",
} DefaultValue: defaultEtcdLogLevel,
Version: "2.0.0",
}
p.EtcdLogLevel.Init(base.mgr)
func (p *EtcdConfig) initEtcdTLSCACert() { p.EtcdLogPath = ParamItem{
p.EtcdTLSCACert = p.Base.LoadWithDefault("etcd.ssl.tlsCACert", "") Key: "etcd.log.path",
} DefaultValue: defaultEtcdLogPath,
Version: "2.0.0",
}
p.EtcdLogPath.Init(base.mgr)
func (p *EtcdConfig) initEtcdTLSMinVersion() { p.EtcdUseSSL = ParamItem{
p.EtcdTLSMinVersion = p.Base.LoadWithDefault("etcd.ssl.tlsMinVersion", "1.3") Key: "etcd.ssl.enabled",
DefaultValue: "false",
Version: "2.0.0",
}
p.EtcdUseSSL.Init(base.mgr)
p.EtcdTLSCert = ParamItem{
Key: "etcd.ssl.tlsCert",
Version: "2.0.0",
}
p.EtcdTLSCert.Init(base.mgr)
p.EtcdTLSKey = ParamItem{
Key: "etcd.ssl.tlsKey",
Version: "2.0.0",
}
p.EtcdTLSKey.Init(base.mgr)
p.EtcdTLSCACert = ParamItem{
Key: "etcd.ssl.tlsCACert",
Version: "2.0.0",
}
p.EtcdTLSCACert.Init(base.mgr)
p.EtcdTLSMinVersion = ParamItem{
Key: "etcd.ssl.tlsMinVersion",
DefaultValue: "1.3",
Version: "2.0.0",
}
p.EtcdTLSMinVersion.Init(base.mgr)
} }
type LocalStorageConfig struct { type LocalStorageConfig struct {
Base *BaseTable Path ParamItem
Path string
} }
func (p *LocalStorageConfig) init(base *BaseTable) { func (p *LocalStorageConfig) Init(base *BaseTable) {
p.Base = base p.Path = ParamItem{
p.initPath() Key: "localStorage.path",
} Version: "2.0.0",
DefaultValue: "/var/lib/milvus/data",
func (p *LocalStorageConfig) initPath() { }
p.Path = p.Base.LoadWithDefault("localStorage.path", "/var/lib/milvus/data") p.Path.Init(base.mgr)
} }
type MetaStoreConfig struct { type MetaStoreConfig struct {
@ -214,7 +236,7 @@ type MetaStoreConfig struct {
MetaStoreType string MetaStoreType string
} }
func (p *MetaStoreConfig) init(base *BaseTable) { func (p *MetaStoreConfig) Init(base *BaseTable) {
p.Base = base p.Base = base
p.LoadCfgToMemory() p.LoadCfgToMemory()
} }
@ -241,7 +263,7 @@ type MetaDBConfig struct {
MaxIdleConns int MaxIdleConns int
} }
func (p *MetaDBConfig) init(base *BaseTable) { func (p *MetaDBConfig) Init(base *BaseTable) {
p.Base = base p.Base = base
p.LoadCfgToMemory() p.LoadCfgToMemory()
} }
@ -306,87 +328,101 @@ func (p *MetaDBConfig) initMaxIdleConns() {
// ///////////////////////////////////////////////////////////////////////////// // /////////////////////////////////////////////////////////////////////////////
// --- pulsar --- // --- pulsar ---
type PulsarConfig struct { type PulsarConfig struct {
Base *BaseTable Address ParamItem
Port ParamItem
Address string WebAddress ParamItem
WebAddress string WebPort ParamItem
MaxMessageSize int MaxMessageSize ParamItem
// support auth // support auth
AuthPlugin string AuthPlugin ParamItem
AuthParams string AuthParams ParamItem
// support tenant // support tenant
Tenant string Tenant ParamItem
Namespace string Namespace ParamItem
} }
func (p *PulsarConfig) init(base *BaseTable) { func (p *PulsarConfig) Init(base *BaseTable) {
p.Base = base p.Port = ParamItem{
Key: "pulsar.port",
p.initAddress() Version: "2.0.0",
p.initWebAddress() DefaultValue: "6650",
p.initMaxMessageSize()
p.initAuthPlugin()
p.initAuthParams()
p.initTenant()
p.initNamespace()
}
func (p *PulsarConfig) initAddress() {
pulsarHost := p.Base.LoadWithDefault("pulsar.address", "")
if strings.Contains(pulsarHost, ":") {
p.Address = pulsarHost
return
} }
p.Port.Init(base.mgr)
port := p.Base.LoadWithDefault("pulsar.port", "") p.Address = ParamItem{
if len(pulsarHost) != 0 && len(port) != 0 { Key: "pulsar.address",
p.Address = "pulsar://" + pulsarHost + ":" + port Version: "2.0.0",
DefaultValue: "localhost",
Formatter: func(addr string) string {
if addr == "" {
return ""
} }
} if strings.Contains(addr, ":") {
return addr
func (p *PulsarConfig) initWebAddress() {
if p.Address == "" {
return
} }
port, _ := p.Port.get()
return "pulsar://" + addr + ":" + port
},
}
p.Address.Init(base.mgr)
pulsarURL, err := url.ParseRequestURI(p.Address) p.WebPort = ParamItem{
Key: "pulsar.webport",
Version: "2.0.0",
DefaultValue: "80",
}
p.WebPort.Init(base.mgr)
p.WebAddress = ParamItem{
Key: "pulsar.webaddress",
Version: "2.0.0",
DefaultValue: "",
Formatter: func(add string) string {
pulsarURL, err := url.ParseRequestURI(p.Address.GetValue())
if err != nil { if err != nil {
p.WebAddress = ""
log.Info("failed to parse pulsar config, assume pulsar not used", zap.Error(err)) log.Info("failed to parse pulsar config, assume pulsar not used", zap.Error(err))
} else { return ""
webport := p.Base.LoadWithDefault("pulsar.webport", "80")
p.WebAddress = "http://" + pulsarURL.Hostname() + ":" + webport
} }
} return "http://" + pulsarURL.Hostname() + ":" + p.WebPort.GetValue()
},
func (p *PulsarConfig) initMaxMessageSize() {
maxMessageSizeStr, err := p.Base.Load("pulsar.maxMessageSize")
if err != nil {
p.MaxMessageSize = SuggestPulsarMaxMessageSize
} else {
maxMessageSize, err := strconv.Atoi(maxMessageSizeStr)
if err != nil {
p.MaxMessageSize = SuggestPulsarMaxMessageSize
} else {
p.MaxMessageSize = maxMessageSize
} }
p.WebAddress.Init(base.mgr)
p.MaxMessageSize = ParamItem{
Key: "pulsar.maxMessageSize",
Version: "2.0.0",
DefaultValue: strconv.Itoa(SuggestPulsarMaxMessageSize),
} }
} p.MaxMessageSize.Init(base.mgr)
func (p *PulsarConfig) initAuthPlugin() { p.Tenant = ParamItem{
p.AuthPlugin = p.Base.LoadWithDefault("pulsar.authPlugin", "") Key: "pulsar.tenant",
} Version: "2.2.0",
DefaultValue: "public",
}
p.Tenant.Init(base.mgr)
func (p *PulsarConfig) initAuthParams() { p.Namespace = ParamItem{
paramString := p.Base.LoadWithDefault("pulsar.authParams", "") Key: "pulsar.namespace",
Version: "2.2.0",
DefaultValue: "default",
}
p.Namespace.Init(base.mgr)
// need to parse params to json due to .yaml config file doesn't support json format config item p.AuthPlugin = ParamItem{
// official pulsar client JWT config : {"token","fake_token_string"} Key: "pulsar.authPlugin",
// milvus config: token:fake_token_string Version: "2.2.0",
}
p.AuthPlugin.Init(base.mgr)
p.AuthParams = ParamItem{
Key: "pulsar.authParams",
Version: "2.2.0",
Formatter: func(authParams string) string {
jsonMap := make(map[string]string) jsonMap := make(map[string]string)
params := strings.Split(paramString, ",") params := strings.Split(authParams, ",")
for _, param := range params { for _, param := range params {
kv := strings.Split(param, ":") kv := strings.Split(param, ":")
if len(kv) == 2 { if len(kv) == 2 {
@ -395,197 +431,184 @@ func (p *PulsarConfig) initAuthParams() {
} }
if len(jsonMap) == 0 { if len(jsonMap) == 0 {
p.AuthParams = "" return ""
} else {
jsonData, _ := json.Marshal(&jsonMap)
p.AuthParams = string(jsonData)
} }
} jsonData, _ := json.Marshal(&jsonMap)
return string(jsonData)
},
}
p.AuthParams.Init(base.mgr)
func (p *PulsarConfig) initTenant() {
p.Tenant = p.Base.LoadWithDefault("pulsar.tenant", "public")
}
func (p *PulsarConfig) initNamespace() {
p.Namespace = p.Base.LoadWithDefault("pulsar.namespace", "default")
} }
// --- kafka --- // --- kafka ---
type KafkaConfig struct { type KafkaConfig struct {
Base *BaseTable Address ParamItem
Address string SaslUsername ParamItem
SaslUsername string SaslPassword ParamItem
SaslPassword string SaslMechanisms ParamItem
SaslMechanisms string SecurityProtocol ParamItem
SecurityProtocol string ConsumerExtraConfig ParamGroup
ConsumerExtraConfig map[string]string ProducerExtraConfig ParamGroup
ProducerExtraConfig map[string]string
} }
func (k *KafkaConfig) init(base *BaseTable) { func (k *KafkaConfig) Init(base *BaseTable) {
k.Base = base k.Address = ParamItem{
k.initAddress() Key: "kafka.brokerList",
k.initSaslUsername() DefaultValue: "",
k.initSaslPassword() Version: "2.1.0",
k.initSaslMechanisms() }
k.initSecurityProtocol() k.Address.Init(base.mgr)
k.initExtraKafkaConfig()
}
func (k *KafkaConfig) initAddress() { k.SaslUsername = ParamItem{
k.Address = k.Base.LoadWithDefault("kafka.brokerList", "") Key: "kafka.saslUsername",
} DefaultValue: "",
Version: "2.1.0",
}
k.SaslUsername.Init(base.mgr)
func (k *KafkaConfig) initSaslUsername() { k.SaslPassword = ParamItem{
k.SaslUsername = k.Base.LoadWithDefault("kafka.saslUsername", "") Key: "kafka.saslPassword",
} DefaultValue: "",
Version: "2.1.0",
}
k.SaslPassword.Init(base.mgr)
func (k *KafkaConfig) initSaslPassword() { k.SaslMechanisms = ParamItem{
k.SaslPassword = k.Base.LoadWithDefault("kafka.saslPassword", "") Key: "kafka.saslMechanisms",
} DefaultValue: "PLAIN",
Version: "2.1.0",
}
k.SaslMechanisms.Init(base.mgr)
func (k *KafkaConfig) initSaslMechanisms() { k.SecurityProtocol = ParamItem{
k.SaslMechanisms = k.Base.LoadWithDefault("kafka.saslMechanisms", "PLAIN") Key: "kafka.securityProtocol",
} DefaultValue: "SASL_SSL",
Version: "2.1.0",
}
k.SecurityProtocol.Init(base.mgr)
func (k *KafkaConfig) initSecurityProtocol() { k.ConsumerExtraConfig = ParamGroup{
k.SecurityProtocol = k.Base.LoadWithDefault("kafka.securityProtocol", "SASL_SSL") KeyPrefix: "kafka.consumer.",
} Version: "2.2.0",
}
k.ConsumerExtraConfig.Init(base.mgr)
func (k *KafkaConfig) initExtraKafkaConfig() { k.ProducerExtraConfig = ParamGroup{
k.ConsumerExtraConfig = k.Base.GetConfigSubSet(KafkaConsumerConfigPrefix) KeyPrefix: "kafka.producer.",
k.ProducerExtraConfig = k.Base.GetConfigSubSet(KafkaProducerConfigPrefix) Version: "2.2.0",
}
k.ProducerExtraConfig.Init(base.mgr)
} }
// ///////////////////////////////////////////////////////////////////////////// // /////////////////////////////////////////////////////////////////////////////
// --- rocksmq --- // --- rocksmq ---
type RocksmqConfig struct { type RocksmqConfig struct {
Base *BaseTable Path ParamItem
Path string
} }
func (p *RocksmqConfig) init(base *BaseTable) { func (r *RocksmqConfig) Init(base *BaseTable) {
p.Base = base r.Path = ParamItem{
Key: "rocksmq.path",
p.initPath() DefaultValue: "",
} Version: "2.0.0",
}
func (p *RocksmqConfig) initPath() { r.Path.Init(base.mgr)
p.Path = p.Base.LoadWithDefault("rocksmq.path", "")
} }
// ///////////////////////////////////////////////////////////////////////////// // /////////////////////////////////////////////////////////////////////////////
// --- minio --- // --- minio ---
type MinioConfig struct { type MinioConfig struct {
Base *BaseTable Address ParamItem
Port ParamItem
Address string AccessKeyID ParamItem
AccessKeyID string SecretAccessKey ParamItem
SecretAccessKey string UseSSL ParamItem
UseSSL bool BucketName ParamItem
BucketName string RootPath ParamItem
RootPath string UseIAM ParamItem
UseIAM bool CloudProvider ParamItem
CloudProvider string IAMEndpoint ParamItem
IAMEndpoint string
} }
func (p *MinioConfig) init(base *BaseTable) { func (p *MinioConfig) Init(base *BaseTable) {
p.Base = base p.Port = ParamItem{
Key: "minio.port",
p.initAddress() DefaultValue: "9000",
p.initAccessKeyID() Version: "2.0.0",
p.initSecretAccessKey()
p.initUseSSL()
p.initBucketName()
p.initRootPath()
p.initUseIAM()
p.initCloudProvider()
p.initIAMEndpoint()
}
func (p *MinioConfig) initAddress() {
host, err := p.Base.Load("minio.Address")
if err != nil {
panic(err)
} }
// for compatible p.Port.Init(base.mgr)
if strings.Contains(host, ":") {
p.Address = host p.Address = ParamItem{
} else { Key: "minio.address",
port := p.Base.LoadWithDefault("minio.port", "9000") DefaultValue: "",
p.Address = host + ":" + port Version: "2.0.0",
Formatter: func(addr string) string {
if addr == "" {
return ""
} }
} if strings.Contains(addr, ":") {
return addr
func (p *MinioConfig) initAccessKeyID() {
keyID, err := p.Base.Load("minio.accessKeyID")
if err != nil {
panic(err)
} }
p.AccessKeyID = keyID port, _ := p.Port.get()
} return addr + ":" + port
},
func (p *MinioConfig) initSecretAccessKey() {
key, err := p.Base.Load("minio.secretAccessKey")
if err != nil {
panic(err)
} }
p.SecretAccessKey = key p.Address.Init(base.mgr)
}
func (p *MinioConfig) initUseSSL() { p.AccessKeyID = ParamItem{
usessl, err := p.Base.Load("minio.useSSL") Key: "minio.accessKeyID",
if err != nil { Version: "2.0.0",
panic(err) PanicIfEmpty: true,
} }
p.UseSSL, _ = strconv.ParseBool(usessl) p.AccessKeyID.Init(base.mgr)
}
func (p *MinioConfig) initBucketName() { p.SecretAccessKey = ParamItem{
bucketName, err := p.Base.Load("minio.bucketName") Key: "minio.secretAccessKey",
if err != nil { Version: "2.0.0",
panic(err) PanicIfEmpty: true,
} }
p.BucketName = bucketName p.SecretAccessKey.Init(base.mgr)
}
func (p *MinioConfig) initRootPath() { p.UseSSL = ParamItem{
rootPath, err := p.Base.Load("minio.rootPath") Key: "minio.useSSL",
if err != nil { Version: "2.0.0",
panic(err) PanicIfEmpty: true,
} }
p.RootPath = rootPath p.UseSSL.Init(base.mgr)
}
func (p *MinioConfig) initUseIAM() { p.BucketName = ParamItem{
useIAM := p.Base.LoadWithDefault("minio.useIAM", DefaultMinioUseIAM) Key: "minio.bucketName",
var err error Version: "2.0.0",
p.UseIAM, err = strconv.ParseBool(useIAM) PanicIfEmpty: true,
if err != nil {
panic("parse bool useIAM:" + err.Error())
} }
} p.BucketName.Init(base.mgr)
// CloudProvider supported p.RootPath = ParamItem{
const ( Key: "minio.rootPath",
CloudProviderAWS = "aws" Version: "2.0.0",
CloudProviderGCP = "gcp" PanicIfEmpty: true,
)
var supportedCloudProvider = map[string]bool{
CloudProviderAWS: true,
CloudProviderGCP: true,
}
func (p *MinioConfig) initCloudProvider() {
p.CloudProvider = p.Base.LoadWithDefault("minio.cloudProvider", DefaultMinioCloudProvider)
if !supportedCloudProvider[p.CloudProvider] {
panic("unsupported cloudProvider:" + p.CloudProvider)
} }
} p.RootPath.Init(base.mgr)
func (p *MinioConfig) initIAMEndpoint() { p.UseIAM = ParamItem{
p.IAMEndpoint = p.Base.LoadWithDefault("minio.iamEndpoint", DefaultMinioIAMEndpoint) Key: "minio.useIAM",
DefaultValue: DefaultMinioUseIAM,
Version: "2.0.0",
}
p.UseIAM.Init(base.mgr)
p.CloudProvider = ParamItem{
Key: "minio.cloudProvider",
DefaultValue: DefaultMinioCloudProvider,
Version: "2.2.0",
}
p.CloudProvider.Init(base.mgr)
p.IAMEndpoint = ParamItem{
Key: "minio.iamEndpoint",
DefaultValue: DefaultMinioIAMEndpoint,
Version: "2.0.0",
}
p.IAMEndpoint.Init(base.mgr)
} }

View File

@ -23,137 +23,123 @@ func TestServiceParam(t *testing.T) {
SParams.Init() SParams.Init()
t.Run("test etcdConfig", func(t *testing.T) { t.Run("test etcdConfig", func(t *testing.T) {
Params := SParams.EtcdCfg Params := &SParams.EtcdCfg
assert.NotZero(t, len(Params.Endpoints)) assert.NotZero(t, len(Params.Endpoints.GetAsStrings()))
t.Logf("etcd endpoints = %s", Params.Endpoints) t.Logf("etcd endpoints = %s", Params.Endpoints.GetAsStrings())
assert.NotEqual(t, Params.MetaRootPath, "") assert.NotEqual(t, Params.MetaRootPath, "")
t.Logf("meta root path = %s", Params.MetaRootPath) t.Logf("meta root path = %s", Params.MetaRootPath.GetValue())
assert.NotEqual(t, Params.KvRootPath, "") assert.NotEqual(t, Params.KvRootPath, "")
t.Logf("kv root path = %s", Params.KvRootPath) t.Logf("kv root path = %s", Params.KvRootPath.GetValue())
assert.NotNil(t, Params.EtcdUseSSL) assert.NotNil(t, Params.EtcdUseSSL.GetAsBool())
t.Logf("use ssl = %t", Params.EtcdUseSSL) t.Logf("use ssl = %t", Params.EtcdUseSSL.GetAsBool())
assert.NotEmpty(t, Params.EtcdTLSKey) assert.NotEmpty(t, Params.EtcdTLSKey.GetValue())
t.Logf("tls key = %s", Params.EtcdTLSKey) t.Logf("tls key = %s", Params.EtcdTLSKey.GetValue())
assert.NotEmpty(t, Params.EtcdTLSCACert) assert.NotEmpty(t, Params.EtcdTLSCACert.GetValue())
t.Logf("tls CACert = %s", Params.EtcdTLSCACert) t.Logf("tls CACert = %s", Params.EtcdTLSCACert.GetValue())
assert.NotEmpty(t, Params.EtcdTLSCert) assert.NotEmpty(t, Params.EtcdTLSCert.GetValue())
t.Logf("tls cert = %s", Params.EtcdTLSCert) t.Logf("tls cert = %s", Params.EtcdTLSCert.GetValue())
assert.NotEmpty(t, Params.EtcdTLSMinVersion) assert.NotEmpty(t, Params.EtcdTLSMinVersion.GetValue())
t.Logf("tls minVersion = %s", Params.EtcdTLSMinVersion) t.Logf("tls minVersion = %s", Params.EtcdTLSMinVersion.GetValue())
// test UseEmbedEtcd // test UseEmbedEtcd
Params.Base.Save("etcd.use.embed", "true") t.Setenv("etcd.use.embed", "true")
t.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.ClusterDeployMode) t.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.ClusterDeployMode)
assert.Panics(t, func() { Params.initUseEmbedEtcd() }) assert.Panics(t, func() { SParams.Init() })
t.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode) t.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode)
Params.LoadCfgToMemory() t.Setenv("etcd.use.embed", "false")
SParams.Init()
}) })
t.Run("test pulsarConfig", func(t *testing.T) { t.Run("test pulsarConfig", func(t *testing.T) {
{ {
Params := SParams.PulsarCfg assert.NotEqual(t, SParams.PulsarCfg.Address.GetValue(), "")
assert.NotEqual(t, Params.Address, "") t.Logf("pulsar address = %s", SParams.PulsarCfg.Address.GetValue())
t.Logf("pulsar address = %s", Params.Address) assert.Equal(t, SParams.PulsarCfg.MaxMessageSize.GetAsInt(), SuggestPulsarMaxMessageSize)
assert.Equal(t, Params.MaxMessageSize, SuggestPulsarMaxMessageSize)
} }
address := "pulsar://localhost:6650" address := "pulsar://localhost:6650"
{ {
Params := SParams.PulsarCfg
SParams.BaseTable.Save("pulsar.address", address) SParams.BaseTable.Save("pulsar.address", address)
Params.initAddress() assert.Equal(t, SParams.PulsarCfg.Address.GetValue(), address)
assert.Equal(t, Params.Address, address)
} }
{ {
Params := SParams.PulsarCfg
SParams.BaseTable.Save("pulsar.address", "localhost") SParams.BaseTable.Save("pulsar.address", "localhost")
SParams.BaseTable.Save("pulsar.port", "6650") SParams.BaseTable.Save("pulsar.port", "6650")
Params.initAddress() assert.Equal(t, SParams.PulsarCfg.Address.GetValue(), address)
assert.Equal(t, Params.Address, address)
} }
}) })
t.Run("test pulsar web config", func(t *testing.T) { t.Run("test pulsar web config", func(t *testing.T) {
Params := SParams.PulsarCfg assert.NotEqual(t, SParams.PulsarCfg.Address.GetValue(), "")
assert.NotEqual(t, Params.Address, "")
{ {
Params.initWebAddress() assert.NotEqual(t, SParams.PulsarCfg.WebAddress.GetValue(), "")
assert.NotEqual(t, Params.WebAddress, "")
} }
{ {
Params.Address = Params.Address + "invalid" SParams.BaseTable.Save(SParams.PulsarCfg.Address.Key, "u\\invalid")
Params.initWebAddress() assert.Equal(t, SParams.PulsarCfg.WebAddress.GetValue(), "")
assert.Equal(t, Params.WebAddress, "")
} }
{ {
Params.Address = "" SParams.BaseTable.Save(SParams.PulsarCfg.Address.Key, "")
Params.initWebAddress() assert.Equal(t, SParams.PulsarCfg.WebAddress.GetValue(), "")
assert.Equal(t, Params.WebAddress, "")
} }
}) })
t.Run("test pulsar auth config", func(t *testing.T) { t.Run("test pulsar auth config", func(t *testing.T) {
Params := SParams.PulsarCfg Params := SParams.PulsarCfg
Params.initAuthPlugin() assert.Equal(t, "", Params.AuthPlugin.GetValue())
assert.Equal(t, "", Params.AuthPlugin) assert.Equal(t, "", Params.AuthParams.GetValue())
Params.initAuthParams()
assert.Equal(t, "", Params.AuthParams)
}) })
t.Run("test pulsar tenant/namespace config", func(t *testing.T) { t.Run("test pulsar tenant/namespace config", func(t *testing.T) {
Params := SParams.PulsarCfg Params := SParams.PulsarCfg
Params.initTenant() assert.Equal(t, "public", Params.Tenant.GetValue())
assert.Equal(t, "public", Params.Tenant) assert.Equal(t, "default", Params.Namespace.GetValue())
Params.initNamespace()
assert.Equal(t, "default", Params.Namespace)
}) })
t.Run("test rocksmqConfig", func(t *testing.T) { t.Run("test rocksmqConfig", func(t *testing.T) {
Params := SParams.RocksmqCfg Params := &SParams.RocksmqCfg
assert.NotEqual(t, Params.Path, "") assert.NotEqual(t, Params.Path.GetValue(), "")
t.Logf("rocksmq path = %s", Params.Path) t.Logf("rocksmq path = %s", Params.Path.GetValue())
}) })
t.Run("test minioConfig", func(t *testing.T) { t.Run("test minioConfig", func(t *testing.T) {
Params := SParams.MinioCfg Params := &SParams.MinioCfg
addr := Params.Address addr := Params.Address.GetValue()
equal := addr == "localhost:9000" || addr == "minio:9000" equal := addr == "localhost:9000" || addr == "minio:9000"
assert.Equal(t, equal, true) assert.Equal(t, equal, true)
t.Logf("minio address = %s", Params.Address) t.Logf("minio address = %s", Params.Address.GetValue())
assert.Equal(t, Params.AccessKeyID, "minioadmin") assert.Equal(t, Params.AccessKeyID.GetValue(), "minioadmin")
assert.Equal(t, Params.SecretAccessKey, "minioadmin") assert.Equal(t, Params.SecretAccessKey.GetValue(), "minioadmin")
assert.Equal(t, Params.UseSSL, false) assert.Equal(t, Params.UseSSL.GetAsBool(), false)
assert.Equal(t, Params.UseIAM, false) assert.Equal(t, Params.UseIAM.GetAsBool(), false)
assert.Equal(t, Params.CloudProvider, "aws") assert.Equal(t, Params.CloudProvider.GetValue(), "aws")
assert.Equal(t, Params.IAMEndpoint, "") assert.Equal(t, Params.IAMEndpoint.GetValue(), "")
t.Logf("Minio BucketName = %s", Params.BucketName) t.Logf("Minio BucketName = %s", Params.BucketName.GetValue())
t.Logf("Minio rootpath = %s", Params.RootPath) t.Logf("Minio rootpath = %s", Params.RootPath.GetValue())
}) })
} }