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))
}
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)
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()
}
if params.EtcdCfg.UseEmbedEtcd {
if params.EtcdCfg.UseEmbedEtcd.GetAsBool() {
// Start etcd server.
etcd.InitEtcdServer(&params.EtcdCfg)
defer etcd.StopEtcdServer()

View File

@ -23,7 +23,7 @@ func newEtcdBasedBackend(cfg *configs.MilvusConfig) (*etcdBasedBackend, error) {
if err != nil {
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}
return b, nil
}

View File

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

View File

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

View File

@ -81,7 +81,7 @@ func (r *Runner) initEtcdCli() {
func (r *Runner) init() {
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))
// address not important here.
address := time.Now().String()

View File

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

View File

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

View File

@ -23,14 +23,7 @@ import (
"sync"
"time"
"github.com/milvus-io/milvus/internal/log"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
)
const (
ModeWatch = iota
ModeInterval
)
type EtcdSource struct {
@ -39,10 +32,8 @@ type EtcdSource struct {
ctx context.Context
currentConfig map[string]string
keyPrefix string
refreshMode int
refreshInterval time.Duration
intervalDone chan bool
intervalInitOnce sync.Once
configRefresher refresher
eh EventHandler
}
@ -54,15 +45,14 @@ func NewEtcdSource(remoteInfo *EtcdInfo) (*EtcdSource, error) {
if err != nil {
return nil, err
}
return &EtcdSource{
es := &EtcdSource{
etcdCli: etcdCli,
ctx: context.Background(),
currentConfig: make(map[string]string),
keyPrefix: remoteInfo.KeyPrefix,
refreshMode: remoteInfo.RefreshMode,
refreshInterval: remoteInfo.RefreshInterval,
intervalDone: make(chan bool, 1),
}, nil
}
es.configRefresher = newRefresher(remoteInfo.RefreshInterval, es.refreshConfigurations)
return es, nil
}
// GetConfigurationByKey implements ConfigSource
@ -83,12 +73,7 @@ func (es *EtcdSource) GetConfigurations() (map[string]string, error) {
if err != nil {
return nil, err
}
if es.refreshMode == ModeInterval {
es.intervalInitOnce.Do(func() {
go es.refreshConfigurationsPeriodically()
})
}
es.configRefresher.start()
es.RLock()
for key, value := range es.currentConfig {
configMap[key] = value
@ -109,7 +94,11 @@ func (es *EtcdSource) GetSourceName() string {
}
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 {
@ -125,42 +114,12 @@ func (es *EtcdSource) refreshConfigurations() error {
newConfig[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()
defer es.Unlock()
//Populate the events based on the changed value between current config and newly received Config
events, err := PopulateEvents(es.GetSourceName(), es.currentConfig, config)
err = es.configRefresher.fireEvents(es.GetSourceName(), es.currentConfig, newConfig)
if err != nil {
log.Warn("generating event error", zap.Error(err))
return err
}
es.currentConfig = config
//Generate OnEvent Callback based on the events created
if es.eh != nil {
for _, e := range events {
es.eh.OnEvent(e)
}
}
es.currentConfig = newConfig
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
file string
configs map[string]string
configRefresher refresher
}
func NewFileSource(file string) *FileSource {
fs := &FileSource{file: file, configs: make(map[string]string)}
fs.loadFromFile()
func NewFileSource(fileInfo *FileInfo) *FileSource {
fs := &FileSource{
file: fileInfo.Filepath,
configs: make(map[string]string),
}
fs.configRefresher = newRefresher(fileInfo.RefreshInterval, fs.loadFromFile)
return fs
}
// GetConfigurationByKey implements ConfigSource
func (fs *FileSource) GetConfigurationByKey(key string) (string, error) {
fs.RLock()
v, ok := fs.configs[key]
fs.RUnlock()
if !ok {
return "", fmt.Errorf("key not found: %s", key)
}
@ -52,11 +59,19 @@ func (fs *FileSource) GetConfigurationByKey(key string) (string, error) {
// GetConfigurations implements ConfigSource
func (fs *FileSource) GetConfigurations() (map[string]string, error) {
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 {
configMap[k] = v
}
fs.RUnlock()
return configMap, nil
}
@ -71,8 +86,12 @@ func (fs *FileSource) GetSourceName() string {
}
func (fs *FileSource) Close() {
fs.configRefresher.stop()
}
func (fs *FileSource) SetEventHandler(eh EventHandler) {
fs.configRefresher.eh = eh
}
func (fs *FileSource) loadFromFile() error {
yamlReader := viper.New()
configFile := fs.file
@ -86,6 +105,7 @@ func (fs *FileSource) loadFromFile() error {
return err
}
newConfig := make(map[string]string)
for _, key := range yamlReader.AllKeys() {
val := yamlReader.Get(key)
str, err := cast.ToStringE(val)
@ -110,11 +130,17 @@ func (fs *FileSource) loadFromFile() error {
continue
}
}
fs.Lock()
fs.configs[key] = str
fs.configs[formatKey(key)] = str
fs.Unlock()
newConfig[key] = str
newConfig[formatKey(key)] = str
}
fs.Lock()
defer fs.Unlock()
err := fs.configRefresher.fireEvents(fs.GetSourceName(), fs.configs, newConfig)
if err != nil {
return err
}
fs.configs = newConfig
return nil
}

View File

@ -26,19 +26,20 @@ import (
)
const (
TombValue = "TOMB_VALUE"
CustomSourceName = "CustomSource"
TombValue = "TOMB_VAULE"
)
type Manager struct {
sync.RWMutex
Dispatcher *EventDispatcher
sources map[string]Source
keySourceMap map[string]string
overlayConfigs map[string]string // store the configs setted or deleted by user
overlayConfigs map[string]string
}
func NewManager() *Manager {
return &Manager{
Dispatcher: NewEventDispatcher(),
sources: make(map[string]Source),
keySourceMap: 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]
if ok {
if v == TombValue {
return "", fmt.Errorf("key not found: %s", key)
return "", fmt.Errorf("key not found %s", key)
}
return v, nil
}
@ -94,8 +95,8 @@ func (m *Manager) GetConfigsByPattern(pattern string, withPrefix bool) map[strin
return matchedConfig
}
// Configs returns all the key values
func (m *Manager) Configs() map[string]string {
// GetConfigs returns all the key values
func (m *Manager) GetConfigs() map[string]string {
m.RLock()
defer m.RUnlock()
config := make(map[string]string)
@ -111,24 +112,6 @@ func (m *Manager) Configs() map[string]string {
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() {
for _, s := range m.sources {
s.Close()
@ -156,6 +139,20 @@ func (m *Manager) AddSource(source Source) error {
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.
func (m *Manager) pullSourceConfigs(source string) error {
configSource, ok := m.sources[source]
@ -260,7 +257,11 @@ func (m *Manager) OnEvent(event *Event) {
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 {

View File

@ -25,11 +25,11 @@ import (
func TestAllConfigFromManager(t *testing.T) {
mgr, _ := Init()
all := mgr.Configs()
all := mgr.GetConfigs()
assert.Equal(t, 0, len(all))
mgr, _ = Init(WithEnvSource(formatKey))
all = mgr.Configs()
all = mgr.GetConfigs()
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
type EventHandler interface {
OnEvent(event *Event)
GetIdentifier() string
}
// EtcdInfo has attribute for config center source initialization
@ -41,14 +42,19 @@ type EtcdInfo struct {
Endpoints []string
KeyPrefix string
RefreshMode int
//Pull Configuration interval, unit is second
RefreshInterval time.Duration
}
// FileInfo has attribute for file source
type FileInfo struct {
Filepath string
RefreshInterval time.Duration
}
//Options hold options
type Options struct {
File *string
File *FileInfo
EtcdInfo *EtcdInfo
EnvKeyFormatter func(string) string
}
@ -57,9 +63,9 @@ type Options struct {
type Option func(options *Options)
//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) {
options.File = &f
options.File = fi
}
}

View File

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

View File

@ -50,7 +50,7 @@ func Test_garbageCollector_basic(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
assert.NotNil(t, segRefer)
@ -112,7 +112,7 @@ func Test_garbageCollector_scan(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
assert.NotNil(t, segRefer)
@ -276,9 +276,9 @@ func Test_garbageCollector_scan(t *testing.T) {
// 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) {
Params.Init()
cli, err := minio.New(Params.MinioCfg.Address, &minio.Options{
Creds: credentials.NewStaticV4(Params.MinioCfg.AccessKeyID, Params.MinioCfg.SecretAccessKey, ""),
Secure: Params.MinioCfg.UseSSL,
cli, err := minio.New(Params.MinioCfg.Address.GetValue(), &minio.Options{
Creds: credentials.NewStaticV4(Params.MinioCfg.AccessKeyID.GetValue(), Params.MinioCfg.SecretAccessKey.GetValue(), ""),
Secure: Params.MinioCfg.UseSSL.GetAsBool(),
})
if err != nil {
return nil, nil, nil, nil, nil, err

View File

@ -238,7 +238,7 @@ func (s *Server) Register() 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 {
return errors.New("failed to initialize session")
}
@ -455,7 +455,7 @@ func (s *Server) initSegmentManager() {
}
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
reloadEtcdFn := func() error {
var err error
@ -803,7 +803,7 @@ func (s *Server) handleFlushingSegments(ctx context.Context) {
func (s *Server) initRootCoordClient() error {
var err error
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.EtcdCfg.MetaRootPath, s.etcdCli); err != nil {
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli); err != nil {
return err
}
if err = s.rootCoordClient.Init(); err != nil {

View File

@ -917,7 +917,7 @@ func TestServer_watchQueryCoord(t *testing.T) {
Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
assert.NotNil(t, etcdKV)
factory := dependency.NewDefaultFactory(true)
svr := CreateServer(context.TODO(), factory)
@ -3082,7 +3082,7 @@ func TestDataCoord_Import(t *testing.T) {
t.Run("no datanode available", func(t *testing.T) {
svr := newTestServer(t, nil)
Params.MinioCfg.Address = "minio:9000"
Params.BaseTable.Save("minio.address", "minio:9000")
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
ImportTask: &datapb.ImportTask{
CollectionId: 100,
@ -3363,7 +3363,7 @@ func newTestServer(t *testing.T, receiveCh chan any, opts ...Option) *Server {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
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())
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)
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())
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)
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())
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.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.Register()
@ -3567,7 +3567,7 @@ func Test_initServiceDiscovery(t *testing.T) {
})
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.Register()
req := &datapb.AcquireSegmentLockRequest{
@ -3578,7 +3578,7 @@ func Test_initServiceDiscovery(t *testing.T) {
assert.Nil(t, err)
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())
assert.Nil(t, err)
@ -3596,7 +3596,9 @@ func Test_newChunkManagerFactory(t *testing.T) {
Params.DataCoordCfg.EnableGarbageCollection = true
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()
assert.Nil(t, storageCli)
assert.Error(t, err)
@ -3629,7 +3631,9 @@ func Test_initGarbageCollection(t *testing.T) {
})
t.Run("err_minio_bad_address", func(t *testing.T) {
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()
assert.Nil(t, storageCli)
assert.Error(t, err)
@ -3644,7 +3648,7 @@ func testDataCoordBase(t *testing.T, opts ...Option) *Server {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
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())
assert.Nil(t, err)

View File

@ -211,7 +211,7 @@ func (node *DataNode) Register() 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 {
return errors.New("failed to initialize session")
}
@ -493,7 +493,7 @@ func (node *DataNode) Start() error {
}
connectEtcdFn := func() error {
etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath)
etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
node.watchKv = etcdKV
return nil
}

View File

@ -832,7 +832,7 @@ func TestWatchChannel(t *testing.T) {
t.Run("test watch channel", func(t *testing.T) {
// 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"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh)
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) {
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"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh)
err = kv.Save(path, string([]byte{23}))

View File

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

View File

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

View File

@ -35,7 +35,7 @@ func Test_NewClient(t *testing.T) {
ctx := context.Background()
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg)
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.NotNil(t, client)

View File

@ -100,7 +100,7 @@ func (s *Server) init() error {
if s.indexCoord == nil {
var err error
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 {
log.Warn("failed to create IndexCoord client for DataCoord", zap.Error(err))
return err

View File

@ -250,7 +250,7 @@ func (s *Server) init() error {
// --- RootCoord Client ---
if s.newRootCoordClient != nil {
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 {
log.Error("failed to create new RootCoord client", zap.Error(err))
panic(err)
@ -276,7 +276,7 @@ func (s *Server) init() error {
// --- DataCoord Client ---
if s.newDataCoordClient != nil {
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 {
log.Error("failed to create new DataCoord client", zap.Error(err))
panic(err)

View File

@ -52,13 +52,13 @@ func TestIndexCoordClient(t *testing.T) {
address = addr
}
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.Register()
return err
}
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)
return err
}
@ -76,7 +76,7 @@ func TestIndexCoordClient(t *testing.T) {
//
//etcdCli, err := etcd.GetEtcdClient(&indexcoord.Params.EtcdCfg)
//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.NotNil(t, icc)

View File

@ -120,7 +120,7 @@ func (s *Server) init() error {
// --- RootCoord ---
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 {
log.Debug("IndexCoord try to new RootCoord client failed", zap.Error(err))
panic(err)
@ -148,7 +148,7 @@ func (s *Server) init() error {
// --- DataCoord ---
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 {
log.Debug("IndexCoord try to new DataCoord client failed", zap.Error(err))
panic(err)

View File

@ -358,7 +358,7 @@ func (s *Server) init() error {
if s.rootCoordClient == nil {
var err error
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 {
log.Warn("failed to create RootCoord client for Proxy", zap.Error(err))
return err
@ -387,7 +387,7 @@ func (s *Server) init() error {
if s.dataCoordClient == nil {
var err error
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 {
log.Warn("failed to create DataCoord client for Proxy", zap.Error(err))
return err
@ -416,7 +416,7 @@ func (s *Server) init() error {
if s.indexCoordClient == nil {
var err error
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 {
log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err))
return err
@ -445,7 +445,7 @@ func (s *Server) init() error {
if s.queryCoordClient == nil {
var err error
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 {
log.Warn("failed to create QueryCoord client for Proxy", zap.Error(err))
return err

View File

@ -37,7 +37,7 @@ func Test_NewClient(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg)
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.NotNil(t, client)

View File

@ -134,7 +134,7 @@ func (s *Server) init() error {
// --- Master Server Client ---
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 {
log.Debug("QueryCoord try to new RootCoord client failed", zap.Error(err))
panic(err)
@ -165,7 +165,7 @@ func (s *Server) init() error {
// --- Data service client ---
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 {
log.Debug("QueryCoord try to new DataCoord client failed", zap.Error(err))
panic(err)
@ -193,7 +193,7 @@ func (s *Server) init() error {
// --- IndexCoord ---
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 {
log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err))
panic(err)

View File

@ -37,7 +37,7 @@ func Test_NewClient(t *testing.T) {
ctx := context.Background()
etcdCli, err := etcd.GetEtcdClient(&proxy.Params.EtcdCfg)
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.NotNil(t, client)

View File

@ -180,7 +180,7 @@ func (s *Server) init() error {
if s.newDataCoordClient != nil {
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 {
panic(err)
}
@ -188,7 +188,7 @@ func (s *Server) init() error {
}
if s.newIndexCoordClient != nil {
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 {
panic(err)
}
@ -196,7 +196,7 @@ func (s *Server) init() error {
}
if s.newQueryCoordClient != nil {
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 {
panic(err)
}

View File

@ -180,12 +180,13 @@ func TestRun(t *testing.T) {
rand.Seed(time.Now().UnixNano())
randVal := rand.Int()
rootPath := fmt.Sprintf("/%d/test", randVal)
rootcoord.Params.BaseTable.Save("etcd.rootPath", rootPath)
rootcoord.Params.Init()
rootcoord.Params.EtcdCfg.MetaRootPath = fmt.Sprintf("/%d/test/meta", randVal)
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
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())
assert.Nil(t, err)
err = svr.Run()

View File

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

View File

@ -157,7 +157,7 @@ func (i *IndexCoord) Register() 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 {
return errors.New("failed to initialize session")
}
@ -183,7 +183,7 @@ func (i *IndexCoord) Init() 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)
return err
}

View File

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

View File

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

View File

@ -164,14 +164,14 @@ func (i *IndexNode) initKnowhere() {
cThreadCoreCoefficient := C.int64_t(Params.CommonCfg.ThreadCoreCoefficient)
C.InitThreadCoreCoefficient(cThreadCoreCoefficient)
cCpuNum := C.int(hardware.GetCPUNum())
C.InitCpuNum(cCpuNum)
cCPUNum := C.int(hardware.GetCPUNum())
C.InitCpuNum(cCPUNum)
initcore.InitLocalStorageConfig(Params)
}
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 {
return errors.New("failed to initialize session")
}

View File

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

View File

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

View File

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

View File

@ -31,10 +31,9 @@ import (
func TestEtcdRestartLoad(te *testing.T) {
etcdDataDir := "/tmp/_etcd_data"
te.Setenv(metricsinfo.DeployModeEnvKey, metricsinfo.StandaloneDeployMode)
te.Setenv("ETCD_USE_EMBED", "true")
param := new(paramtable.ServiceParam)
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.data.dir", etcdDataDir)
//clean up data
@ -42,7 +41,6 @@ func TestEtcdRestartLoad(te *testing.T) {
err := os.RemoveAll(etcdDataDir)
assert.NoError(te, err)
}()
param.EtcdCfg.LoadCfgToMemory()
te.Run("EtcdKV SaveRestartAndLoad", func(t *testing.T) {
rootPath := "/etcd/test/root/saveRestartAndLoad"
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) {
log.Info("start etcd with rootPath",
zap.String("rootpath", rootPath),
zap.Bool("isEmbed", etcdCfg.UseEmbedEtcd))
if etcdCfg.UseEmbedEtcd {
path := etcdCfg.ConfigPath
zap.Bool("isEmbed", etcdCfg.UseEmbedEtcd.GetAsBool()))
if etcdCfg.UseEmbedEtcd.GetAsBool() {
path := etcdCfg.ConfigPath.GetValue()
var cfg *embed.Config
if len(path) > 0 {
cfgFromFile, err := embed.ConfigFromFile(path)
@ -43,7 +43,7 @@ func NewMetaKvFactory(rootPath string, etcdCfg *paramtable.EtcdConfig) (kv.MetaK
} else {
cfg = embed.NewConfig()
}
cfg.Dir = etcdCfg.DataDir
cfg.Dir = etcdCfg.DataDir.GetValue()
metaKv, err := NewEmbededEtcdKV(cfg, rootPath)
if err != nil {
return nil, err

View File

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

View File

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

View File

@ -47,18 +47,18 @@ func NewKafkaClientInstanceWithConfigMap(config kafka.ConfigMap, extraConsumerCo
}
func NewKafkaClientInstanceWithConfig(config *paramtable.KafkaConfig) *kafkaClient {
kafkaConfig := getBasicConfig(config.Address)
kafkaConfig := getBasicConfig(config.Address.GetValue())
if (config.SaslUsername == "" && config.SaslPassword != "") ||
(config.SaslUsername != "" && config.SaslPassword == "") {
if (config.SaslUsername.GetValue() == "" && config.SaslPassword.GetValue() != "") ||
(config.SaslUsername.GetValue() != "" && config.SaslPassword.GetValue() == "") {
panic("enable security mode need config username and password at the same time!")
}
if config.SaslUsername != "" && config.SaslPassword != "" {
kafkaConfig.SetKey("sasl.mechanisms", config.SaslMechanisms)
kafkaConfig.SetKey("security.protocol", config.SecurityProtocol)
kafkaConfig.SetKey("sasl.username", config.SaslUsername)
kafkaConfig.SetKey("sasl.password", config.SaslPassword)
if config.SaslUsername.GetValue() != "" && config.SaslPassword.GetValue() != "" {
kafkaConfig.SetKey("sasl.mechanisms", config.SaslMechanisms.GetValue())
kafkaConfig.SetKey("security.protocol", config.SecurityProtocol.GetValue())
kafkaConfig.SetKey("sasl.username", config.SaslUsername.GetValue())
kafkaConfig.SetKey("sasl.password", config.SaslPassword.GetValue())
}
specExtraConfig := func(config map[string]string) kafka.ConfigMap {
@ -69,7 +69,7 @@ func NewKafkaClientInstanceWithConfig(config *paramtable.KafkaConfig) *kafkaClie
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) {
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) })
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) })
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)
assert.NotNil(t, client)
assert.NotNil(t, client.basicConfig)
consumerConfig := make(map[string]string)
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)
assert.Equal(t, "dc", client4.consumerConfig["client.id"])
@ -322,7 +337,12 @@ func TestKafkaClient_NewKafkaClientInstanceWithConfig(t *testing.T) {
producerConfig := make(map[string]string)
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)
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) {
var Params paramtable.ComponentParam
Params.Init()
var params paramtable.ComponentParam
params.Init()
testPath := ""
Params.ProxyCfg.AccessLog.LocalPath = testPath
Params.ProxyCfg.AccessLog.MinioEnable = true
Params.MinioCfg.Address = ""
params.ProxyCfg.AccessLog.LocalPath = testPath
params.ProxyCfg.AccessLog.MinioEnable = true
params.Save(params.MinioCfg.Address.Key, "")
//init err with invalid minio address
_, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
_, err := NewRotateLogger(&params.ProxyCfg.AccessLog, &params.MinioCfg)
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) {
handlerCfg := config{
address: cfg.Address,
bucketName: cfg.BucketName,
accessKeyID: cfg.AccessKeyID,
secretAccessKeyID: cfg.SecretAccessKey,
useSSL: cfg.UseSSL,
address: cfg.Address.GetValue(),
bucketName: cfg.BucketName.GetValue(),
accessKeyID: cfg.AccessKeyID.GetValue(),
secretAccessKeyID: cfg.SecretAccessKey.GetValue(),
useSSL: cfg.UseSSL.GetAsBool(),
createBucket: true,
useIAM: cfg.UseIAM,
iamEndpoint: cfg.IAMEndpoint,
useIAM: cfg.UseIAM.GetAsBool(),
iamEndpoint: cfg.IAMEndpoint.GetValue(),
}
client, err := newMinioClient(ctx, handlerCfg)

View File

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

View File

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

View File

@ -150,7 +150,7 @@ func (node *Proxy) Register() error {
// initSession initialize the session of Proxy.
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 {
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")
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)
log.Debug("init access log for Proxy done")

View File

@ -458,7 +458,7 @@ func TestProxy(t *testing.T) {
}
qn := runQueryNode(ctx, localMsg, alias)
log.Info("running query node ...")
log.Info("running QueryNode ...")
if qn != nil {
defer func() {
@ -506,7 +506,7 @@ func TestProxy(t *testing.T) {
go testServer.startGrpc(ctx, &wg)
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)
err = rootCoordClient.Init()
assert.NoError(t, err)
@ -515,7 +515,7 @@ func TestProxy(t *testing.T) {
proxy.SetRootCoordClient(rootCoordClient)
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)
err = dataCoordClient.Init()
assert.NoError(t, err)
@ -524,7 +524,7 @@ func TestProxy(t *testing.T) {
proxy.SetDataCoordClient(dataCoordClient)
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)
err = queryCoordClient.Init()
assert.NoError(t, err)
@ -533,7 +533,7 @@ func TestProxy(t *testing.T) {
proxy.SetQueryCoordClient(queryCoordClient)
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)
err = indexCoordClient.Init()
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) {
threshold := Params.PulsarCfg.MaxMessageSize
threshold := Params.PulsarCfg.MaxMessageSize.GetAsInt()
log.Debug("assign segmentid", zap.Int("threshold", threshold))
result := &msgstream.MsgPack{
BeginTs: it.BeginTs(),

View File

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

View File

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

View File

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

View File

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

View File

@ -116,9 +116,9 @@ func (suite *JobSuite) SetupSuite() {
func (suite *JobSuite) SetupTest() {
config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config)
cli, err := etcd.GetEtcdClient(config)
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.dist = meta.NewDistributionManager()

View File

@ -68,9 +68,9 @@ func (suite *CollectionManagerSuite) SetupSuite() {
func (suite *CollectionManagerSuite) SetupTest() {
var err error
config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config)
cli, err := etcd.GetEtcdClient(config)
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.mgr = NewCollectionManager(suite.store)

View File

@ -51,9 +51,9 @@ func (suite *ReplicaManagerSuite) SetupSuite() {
func (suite *ReplicaManagerSuite) SetupTest() {
var err error
config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config)
cli, err := etcd.GetEtcdClient(config)
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.idAllocator = RandomIncrementIDAllocator()

View File

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

View File

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

View File

@ -145,9 +145,9 @@ func (suite *CollectionObserverSuite) SetupTest() {
suite.idAllocator = RandomIncrementIDAllocator()
log.Debug("create embedded etcd KV...")
config := GenerateEtcdConfig()
client, err := etcd.GetEtcdClient(&config)
client, err := etcd.GetEtcdClient(config)
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)
log.Debug("create meta store...")
suite.store = meta.NewMetaStore(suite.kv)

View File

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

View File

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

View File

@ -34,12 +34,13 @@ var (
// GenerateEtcdConfig returns a etcd config with a random root path,
// NOTE: for test only
func GenerateEtcdConfig() paramtable.EtcdConfig {
config := Params.EtcdCfg
func GenerateEtcdConfig() *paramtable.EtcdConfig {
config := &Params.EtcdCfg
rand.Seed(time.Now().UnixNano())
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 {

View File

@ -146,11 +146,11 @@ func (s *Server) Register() error {
func (s *Server) Init() error {
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))
// 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 {
return fmt.Errorf("failed to create session")
}
@ -160,12 +160,12 @@ func (s *Server) Init() error {
s.factory.Init(Params)
// Init KV
etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath)
etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
s.kv = etcdKV
log.Info("query coordinator try to connect etcd success")
// 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)
err := idAllocator.Initialize()
if err != nil {

View File

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

View File

@ -108,9 +108,9 @@ func (suite *ServiceSuite) SetupSuite() {
func (suite *ServiceSuite) SetupTest() {
config := params.GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config)
cli, err := etcd.GetEtcdClient(config)
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.dist = meta.NewDistributionManager()
@ -135,7 +135,7 @@ func (suite *ServiceSuite) SetupTest() {
suite.server = &Server{
kv: suite.kv,
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(),
dist: suite.dist,
meta: suite.meta,

View File

@ -118,10 +118,10 @@ func (suite *TaskSuite) SetupSuite() {
func (suite *TaskSuite) SetupTest() {
config := GenerateEtcdConfig()
cli, err := etcd.GetEtcdClient(&config)
cli, err := etcd.GetEtcdClient(config)
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.meta = meta.NewMeta(RandomIncrementIDAllocator(), suite.store)
suite.dist = meta.NewDistributionManager()

View File

@ -451,7 +451,7 @@ func TestImpl_ShowConfigurations(t *testing.T) {
t.Run("test ShowConfigurations", func(t *testing.T) {
node, err := genSimpleQueryNode(ctx)
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"
req := &internalpb.ShowConfigurationsRequest{
@ -467,7 +467,7 @@ func TestImpl_ShowConfigurations(t *testing.T) {
t.Run("test ShowConfigurations node failed", func(t *testing.T) {
node, err := genSimpleQueryNode(ctx)
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)
pattern := "Cache"
@ -496,7 +496,7 @@ func TestImpl_GetMetrics(t *testing.T) {
defer wg.Done()
node, err := genSimpleQueryNode(ctx)
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[metricsinfo.MetricTypeKey] = "system_info"

View File

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

View File

@ -42,7 +42,7 @@ func TestGetSystemInfoMetrics(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.NoError(t, err)
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{
Base: genCommonMsgBase(commonpb.MsgType_WatchQueryChannels, node.session.ServerID),
@ -70,7 +70,7 @@ func TestGetComponentConfigurationsFailed(t *testing.T) {
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.NoError(t, err)
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{
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 {
return &indexpb.StorageConfig{
Address: Params.MinioCfg.Address,
AccessKeyID: Params.MinioCfg.AccessKeyID,
SecretAccessKey: Params.MinioCfg.SecretAccessKey,
BucketName: Params.MinioCfg.BucketName,
RootPath: Params.MinioCfg.RootPath,
IAMEndpoint: Params.MinioCfg.IAMEndpoint,
UseSSL: Params.MinioCfg.UseSSL,
UseIAM: Params.MinioCfg.UseIAM,
Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: Params.MinioCfg.SecretAccessKey.GetValue(),
BucketName: Params.MinioCfg.BucketName.GetValue(),
RootPath: Params.MinioCfg.RootPath.GetValue(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
UseSSL: Params.MinioCfg.UseSSL.GetAsBool(),
UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
}
}
@ -533,7 +533,7 @@ func genEtcdKV() (*etcdkv.EtcdKV, error) {
if err != nil {
return nil, err
}
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
return etcdKV, nil
}
@ -560,11 +560,11 @@ func genLocalChunkManager() (storage.ChunkManager, error) {
func genRemoteChunkManager(ctx context.Context) (storage.ChunkManager, error) {
return storage.NewMinioChunkManager(
ctx,
storage.Address(Params.MinioCfg.Address),
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
storage.UseSSL(Params.MinioCfg.UseSSL),
storage.BucketName(Params.MinioCfg.BucketName),
storage.Address(Params.MinioCfg.Address.GetValue()),
storage.AccessKeyID(Params.MinioCfg.AccessKeyID.GetValue()),
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey.GetValue()),
storage.UseSSL(Params.MinioCfg.UseSSL.GetAsBool()),
storage.BucketName(Params.MinioCfg.BucketName.GetValue()),
storage.CreateBucket(true))
}
@ -574,11 +574,11 @@ func genVectorChunkManager(ctx context.Context, col *Collection) (*storage.Vecto
rcm, err := storage.NewMinioChunkManager(
ctx,
storage.Address(Params.MinioCfg.Address),
storage.AccessKeyID(Params.MinioCfg.AccessKeyID),
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey),
storage.UseSSL(Params.MinioCfg.UseSSL),
storage.BucketName(Params.MinioCfg.BucketName),
storage.Address(Params.MinioCfg.Address.GetValue()),
storage.AccessKeyID(Params.MinioCfg.AccessKeyID.GetValue()),
storage.SecretAccessKeyID(Params.MinioCfg.SecretAccessKey.GetValue()),
storage.UseSSL(Params.MinioCfg.UseSSL.GetAsBool()),
storage.BucketName(Params.MinioCfg.BucketName.GetValue()),
storage.CreateBucket(true))
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))
return nil, err
}
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
node.etcdKV = etcdKV
node.tSafeReplica = newTSafeReplica()

View File

@ -144,7 +144,7 @@ func NewQueryNode(ctx context.Context, factory dependency.Factory) *QueryNode {
}
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 {
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)
C.InitThreadCoreCoefficient(cThreadCoreCoefficient)
cCpuNum := C.int(hardware.GetCPUNum())
C.InitCpuNum(cCpuNum)
cCPUNum := C.int(hardware.GetCPUNum())
C.InitCpuNum(cCPUNum)
initcore.InitLocalStorageConfig(Params)
}
@ -227,7 +227,7 @@ func (node *QueryNode) Init() error {
var initError error
node.initOnce.Do(func() {
//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()
if err != nil {
log.Error("QueryNode init session failed", zap.Error(err))
@ -252,7 +252,7 @@ func (node *QueryNode) Init() error {
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))
cpuNum := runtime.GOMAXPROCS(0)

View File

@ -48,7 +48,8 @@ type queryCoordMock struct {
func setup() {
os.Setenv("QUERY_NODE_ID", "1")
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) {
@ -87,7 +88,7 @@ func newQueryNodeMock() *QueryNode {
if err != nil {
panic(err)
}
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
factory := newMessageStreamFactory()
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) {
// 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)
if err != nil {
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.
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) {
result := make(map[int64]string)
sessions, _, err := s.session.GetSessions(typeutil.QueryNodeRole)
@ -64,7 +64,7 @@ func (s *ShardClusterService) addShardCluster(collectionID, replicaID int64, vch
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,
func(nodeID int64, addr string) shardQueryNode {

View File

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

View File

@ -92,7 +92,7 @@ func (p *proxyManager) WatchProxy() error {
eventCh := p.etcdCli.Watch(
p.ctx,
path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
clientv3.WithPrefix(),
clientv3.WithCreatedNotify(),
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) {
resp, err := p.etcdCli.Get(
ctx,
path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
clientv3.WithPrefix(),
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)
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())
defer etcdCli.Delete(ctx, sessKey, clientv3.WithPrefix())
s1 := sessionutil.Session{
@ -109,7 +109,7 @@ func TestProxyManager_ErrCompacted(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot)
f1 := func(sess []*sessionutil.Session) {
t.Log("get sessions num", len(sess))
}
@ -117,7 +117,7 @@ func TestProxyManager_ErrCompacted(t *testing.T) {
eventCh := pm.etcdCli.Watch(
pm.ctx,
path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.ProxyRole),
clientv3.WithPrefix(),
clientv3.WithCreatedNotify(),
clientv3.WithPrevKV(),

View File

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

View File

@ -331,7 +331,7 @@ func (c *Core) SetEtcdClient(etcdClient *clientv3.Client) {
}
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 {
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 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
}
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
}
@ -389,7 +389,7 @@ func (c *Core) initMetaTable() 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)
if err := idAllocator.Initialize(); err != nil {
return err
@ -399,7 +399,7 @@ func (c *Core) initIDAllocator() 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)
if err := tsoAllocator.Initialize(); err != nil {
return err
@ -410,7 +410,7 @@ func (c *Core) initTSOAllocator() error {
}
func (c *Core) initImportManager() error {
impTaskKv, err := c.metaKVCreator(Params.EtcdCfg.KvRootPath)
impTaskKv, err := c.metaKVCreator(Params.EtcdCfg.KvRootPath.GetValue())
if err != nil {
return err
}

View File

@ -1324,13 +1324,12 @@ func TestCore_startTimeTickLoop(t *testing.T) {
// make sure the main functions work well when EnableActiveStandby=true
func TestRootcoord_EnableActiveStandby(t *testing.T) {
Params.Init()
Params.RootCoordCfg.EnableActiveStandby = true
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.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.RootCoordDml = fmt.Sprintf("rootcoord-dml-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 {
if params.CommonCfg.StorageType == "local" {
return NewChunkManagerFactory("local", RootPath(params.LocalStorageCfg.Path))
return NewChunkManagerFactory("local", RootPath(params.LocalStorageCfg.Path.GetValue()))
}
return NewChunkManagerFactory("minio",
RootPath(params.MinioCfg.RootPath),
Address(params.MinioCfg.Address),
AccessKeyID(params.MinioCfg.AccessKeyID),
SecretAccessKeyID(params.MinioCfg.SecretAccessKey),
UseSSL(params.MinioCfg.UseSSL),
BucketName(params.MinioCfg.BucketName),
UseIAM(params.MinioCfg.UseIAM),
CloudProvider(params.MinioCfg.CloudProvider),
IAMEndpoint(params.MinioCfg.IAMEndpoint),
RootPath(params.MinioCfg.RootPath.GetValue()),
Address(params.MinioCfg.Address.GetValue()),
AccessKeyID(params.MinioCfg.AccessKeyID.GetValue()),
SecretAccessKeyID(params.MinioCfg.SecretAccessKey.GetValue()),
UseSSL(params.MinioCfg.UseSSL.GetAsBool()),
BucketName(params.MinioCfg.BucketName.GetValue()),
UseIAM(params.MinioCfg.UseIAM.GetAsBool()),
CloudProvider(params.MinioCfg.CloudProvider.GetValue()),
IAMEndpoint(params.MinioCfg.IAMEndpoint.GetValue()),
CreateBucket(true))
}

View File

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

View File

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

View File

@ -35,13 +35,13 @@ var (
// GetEtcdClient returns etcd client
func GetEtcdClient(cfg *paramtable.EtcdConfig) (*clientv3.Client, error) {
if cfg.UseEmbedEtcd {
if cfg.UseEmbedEtcd.GetAsBool() {
return GetEmbedEtcdClient()
}
if cfg.EtcdUseSSL {
return GetRemoteEtcdSSLClient(cfg.Endpoints, cfg.EtcdTLSCert, cfg.EtcdTLSKey, cfg.EtcdTLSCACert, cfg.EtcdTLSMinVersion)
if cfg.EtcdUseSSL.GetAsBool() {
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

View File

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

View File

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

View File

@ -83,7 +83,10 @@ type BaseTable struct {
// NewBaseTableFromYamlOnly only used in migration tool.
// Maybe we shouldn't limit the configDir internally.
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}
return gp
}
@ -126,7 +129,12 @@ func (gp *BaseTable) initConfigsFromLocal(formatter func(key string) string) {
gp.configDir = gp.initConfPath()
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 {
log.Warn("init baseTable with file failed", zap.String("configFile", configFilePath), zap.Error(err))
return
@ -147,11 +155,13 @@ func (gp *BaseTable) initConfigsFromRemote(formatter func(key string) string) {
configFilePath := gp.configDir + "/" + gp.YamlFile
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{
Endpoints: strings.Split(endpoints, ","),
KeyPrefix: rootPath,
RefreshMode: config.ModeInterval,
RefreshInterval: 10 * time.Second,
}))
if err != nil {
@ -183,7 +193,7 @@ func (gp *BaseTable) initConfPath() string {
}
func (gp *BaseTable) Configs() map[string]string {
return gp.mgr.Configs()
return gp.mgr.GetConfigs()
}
// 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)
}
func (gp *BaseTable) GetAll() map[string]string {
return gp.mgr.GetConfigs()
}
// For compatible reason, only visiable for Test
func (gp *BaseTable) Remove(key string) error {
gp.mgr.DeleteConfig(key)

View File

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

View File

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

View File

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