From c05b9ad539c323162f4a520262c3bd5422534a00 Mon Sep 17 00:00:00 2001 From: Enwei Jiao Date: Thu, 17 Nov 2022 18:59:09 +0800 Subject: [PATCH] Add event dispatcher for config (#20393) Signed-off-by: Enwei Jiao Signed-off-by: Enwei Jiao --- cmd/milvus/mck.go | 2 +- cmd/roles/roles.go | 2 +- cmd/tools/migration/backend/etcd.go | 2 +- cmd/tools/migration/backend/etcd210.go | 4 +- cmd/tools/migration/configs/config.go | 3 +- cmd/tools/migration/migration/runner.go | 2 +- internal/config/config.go | 7 +- internal/config/config_test.go | 3 +- internal/config/etcd_source.go | 87 +-- internal/config/event_dispatcher.go | 66 ++ internal/config/file_source.go | 44 +- internal/config/manager.go | 51 +- internal/config/manager_test.go | 4 +- internal/config/refresher.go | 87 +++ internal/config/source.go | 14 +- internal/config/source_test.go | 4 +- internal/datacoord/garbage_collector_test.go | 10 +- internal/datacoord/server.go | 6 +- internal/datacoord/server_test.go | 28 +- internal/datanode/data_node.go | 4 +- internal/datanode/data_node_test.go | 4 +- .../datanode/flow_graph_delete_node_test.go | 13 +- .../flow_graph_insert_buffer_node_test.go | 10 +- .../datacoord/client/client_test.go | 2 +- internal/distributed/datacoord/service.go | 2 +- internal/distributed/datanode/service.go | 4 +- .../indexcoord/client/client_test.go | 6 +- internal/distributed/indexcoord/service.go | 4 +- internal/distributed/proxy/service.go | 8 +- .../querycoord/client/client_test.go | 2 +- internal/distributed/querycoord/service.go | 6 +- .../rootcoord/client/client_test.go | 2 +- internal/distributed/rootcoord/service.go | 6 +- .../distributed/rootcoord/service_test.go | 5 +- internal/indexcoord/index_builder.go | 18 +- internal/indexcoord/index_coord.go | 4 +- internal/indexcoord/index_coord_test.go | 10 +- internal/indexcoord/metrics_info.go | 2 +- internal/indexnode/indexnode.go | 6 +- internal/indexnode/indexnode_mock.go | 2 +- internal/indexnode/indexnode_service_test.go | 16 +- internal/indexnode/metrics_info.go | 2 +- internal/kv/etcd/embed_etcd_restart_test.go | 4 +- internal/kv/etcd/metakv_factory.go | 8 +- internal/mq/msgstream/mq_factory.go | 12 +- internal/mq/msgstream/mq_factory_test.go | 18 +- .../msgstream/mqwrapper/kafka/kafka_client.go | 18 +- .../mqwrapper/kafka/kafka_client_test.go | 30 +- internal/proxy/accesslog/log_writer_test.go | 12 +- internal/proxy/accesslog/minio_handler.go | 14 +- .../proxy/accesslog/minio_handler_test.go | 16 +- internal/proxy/impl.go | 1 - internal/proxy/proxy.go | 4 +- internal/proxy/proxy_test.go | 10 +- internal/proxy/task_insert.go | 3 +- .../balance/rowcount_based_balancer_test.go | 4 +- .../checkers/channel_checker_test.go | 4 +- .../checkers/segment_checker_test.go | 4 +- .../querycoordv2/dist/dist_controller_test.go | 4 +- internal/querycoordv2/job/job_test.go | 4 +- .../meta/collection_manager_test.go | 4 +- .../querycoordv2/meta/replica_manager_test.go | 4 +- .../querycoordv2/meta/target_manager_test.go | 4 +- internal/querycoordv2/mocks/querynode.go | 2 +- .../observers/collection_observer_test.go | 4 +- .../observers/leader_observer_test.go | 4 +- .../observers/target_observer_test.go | 4 +- internal/querycoordv2/params/params.go | 9 +- internal/querycoordv2/server.go | 8 +- internal/querycoordv2/server_test.go | 2 +- internal/querycoordv2/services_test.go | 6 +- internal/querycoordv2/task/task_test.go | 4 +- internal/querynode/impl_test.go | 6 +- internal/querynode/load_index_info.go | 16 +- internal/querynode/metrics_info_test.go | 4 +- internal/querynode/mock_test.go | 40 +- internal/querynode/query_node.go | 10 +- internal/querynode/query_node_test.go | 5 +- internal/querynode/query_shard_service.go | 2 +- internal/querynode/shard_cluster_service.go | 4 +- internal/rootcoord/proxy_client_manager.go | 4 +- internal/rootcoord/proxy_manager.go | 4 +- internal/rootcoord/proxy_manager_test.go | 6 +- internal/rootcoord/quota_center.go | 2 +- internal/rootcoord/root_coord.go | 12 +- internal/rootcoord/root_coord_test.go | 7 +- internal/storage/factory.go | 20 +- internal/storage/minio_chunk_manager.go | 8 +- internal/util/etcd/etcd_server.go | 12 +- internal/util/etcd/etcd_util.go | 8 +- internal/util/etcd/etcd_util_test.go | 32 +- .../util/indexcgowrapper/codec_index_test.go | 16 +- internal/util/paramtable/base_table.go | 24 +- internal/util/paramtable/base_table_test.go | 2 +- internal/util/paramtable/component_param.go | 6 +- .../util/paramtable/component_param_test.go | 4 +- internal/util/paramtable/param_item.go | 95 ++- internal/util/paramtable/service_param.go | 687 +++++++++--------- .../util/paramtable/service_param_test.go | 110 ++- 99 files changed, 1111 insertions(+), 824 deletions(-) create mode 100644 internal/config/event_dispatcher.go create mode 100644 internal/config/refresher.go diff --git a/cmd/milvus/mck.go b/cmd/milvus/mck.go index d4c3c626df..a5507565e0 100644 --- a/cmd/milvus/mck.go +++ b/cmd/milvus/mck.go @@ -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)) } diff --git a/cmd/roles/roles.go b/cmd/roles/roles.go index 075f7e4532..827b83c715 100644 --- a/cmd/roles/roles.go +++ b/cmd/roles/roles.go @@ -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(¶ms.EtcdCfg) defer etcd.StopEtcdServer() diff --git a/cmd/tools/migration/backend/etcd.go b/cmd/tools/migration/backend/etcd.go index 8f6aeaeed0..d2c50d4296 100644 --- a/cmd/tools/migration/backend/etcd.go +++ b/cmd/tools/migration/backend/etcd.go @@ -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 } diff --git a/cmd/tools/migration/backend/etcd210.go b/cmd/tools/migration/backend/etcd210.go index 7867bd2bd6..994f4d6ec4 100644 --- a/cmd/tools/migration/backend/etcd210.go +++ b/cmd/tools/migration/backend/etcd210.go @@ -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] diff --git a/cmd/tools/migration/configs/config.go b/cmd/tools/migration/configs/config.go index 4fe60c54a2..e16ac9ab23 100644 --- a/cmd/tools/migration/configs/config.go +++ b/cmd/tools/migration/configs/config.go @@ -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 { diff --git a/cmd/tools/migration/migration/runner.go b/cmd/tools/migration/migration/runner.go index 3156230f72..669659c242 100644 --- a/cmd/tools/migration/migration/runner.go +++ b/cmd/tools/migration/migration/runner.go @@ -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() diff --git a/internal/config/config.go b/internal/config/config.go index 21475bbcc1..3dad026af6 100644 --- a/internal/config/config.go +++ b/internal/config/config.go @@ -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 diff --git a/internal/config/config_test.go b/internal/config/config_test.go index 5bebe6c836..af5051f33e 100644 --- a/internal/config/config_test.go +++ b/internal/config/config_test.go @@ -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() diff --git a/internal/config/etcd_source.go b/internal/config/etcd_source.go index c9b8331845..0325a047ff 100644 --- a/internal/config/etcd_source.go +++ b/internal/config/etcd_source.go @@ -23,27 +23,18 @@ 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 { sync.RWMutex - etcdCli *clientv3.Client - ctx context.Context - currentConfig map[string]string - keyPrefix string - refreshMode int - refreshInterval time.Duration - intervalDone chan bool - intervalInitOnce sync.Once - eh EventHandler + etcdCli *clientv3.Client + ctx context.Context + currentConfig map[string]string + keyPrefix string + + configRefresher refresher + eh EventHandler } func NewEtcdSource(remoteInfo *EtcdInfo) (*EtcdSource, error) { @@ -54,15 +45,14 @@ func NewEtcdSource(remoteInfo *EtcdInfo) (*EtcdSource, error) { if err != nil { return nil, err } - return &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 := &EtcdSource{ + etcdCli: etcdCli, + ctx: context.Background(), + currentConfig: make(map[string]string), + keyPrefix: remoteInfo.KeyPrefix, + } + 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 } diff --git a/internal/config/event_dispatcher.go b/internal/config/event_dispatcher.go new file mode 100644 index 0000000000..53e5533d22 --- /dev/null +++ b/internal/config/event_dispatcher.go @@ -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 +} diff --git a/internal/config/file_source.go b/internal/config/file_source.go index ff393231e8..372e6a091c 100644 --- a/internal/config/file_source.go +++ b/internal/config/file_source.go @@ -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 } diff --git a/internal/config/manager.go b/internal/config/manager.go index f2f75d4bf7..3d5516a81b 100644 --- a/internal/config/manager.go +++ b/internal/config/manager.go @@ -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 { diff --git a/internal/config/manager_test.go b/internal/config/manager_test.go index c8cfa6ecf9..fbd0f29977 100644 --- a/internal/config/manager_test.go +++ b/internal/config/manager_test.go @@ -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)) } diff --git a/internal/config/refresher.go b/internal/config/refresher.go new file mode 100644 index 0000000000..da5f72cc8d --- /dev/null +++ b/internal/config/refresher.go @@ -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 +} diff --git a/internal/config/source.go b/internal/config/source.go index f7c491346e..85fa920b70 100644 --- a/internal/config/source.go +++ b/internal/config/source.go @@ -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 } } diff --git a/internal/config/source_test.go b/internal/config/source_test.go index 68ec514ab5..cbe85d8123 100644 --- a/internal/config/source_test.go +++ b/internal/config/source_test.go @@ -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) }) diff --git a/internal/datacoord/garbage_collector_test.go b/internal/datacoord/garbage_collector_test.go index 8b0c343104..cff6574db8 100644 --- a/internal/datacoord/garbage_collector_test.go +++ b/internal/datacoord/garbage_collector_test.go @@ -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 diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 48b28b0891..562fccf940 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -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 { diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index f0a6142c77..7fe96b9058 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -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) diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 808f3c0708..daf3ed5f78 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -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 } diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index e3293a4a4e..811472a65b 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -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})) diff --git a/internal/datanode/flow_graph_delete_node_test.go b/internal/datanode/flow_graph_delete_node_test.go index 87fdfb3192..e7cf7c465e 100644 --- a/internal/datanode/flow_graph_delete_node_test.go +++ b/internal/datanode/flow_graph_delete_node_test.go @@ -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), diff --git a/internal/datanode/flow_graph_insert_buffer_node_test.go b/internal/datanode/flow_graph_insert_buffer_node_test.go index 34f6cc489b..242da89055 100644 --- a/internal/datanode/flow_graph_insert_buffer_node_test.go +++ b/internal/datanode/flow_graph_insert_buffer_node_test.go @@ -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 { diff --git a/internal/distributed/datacoord/client/client_test.go b/internal/distributed/datacoord/client/client_test.go index 25a997ace5..1e4b420cac 100644 --- a/internal/distributed/datacoord/client/client_test.go +++ b/internal/distributed/datacoord/client/client_test.go @@ -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) diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index 7a9d2156f2..96a2f721f0 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -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 diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index 11f8f6a91d..b1a797da1f 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -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) diff --git a/internal/distributed/indexcoord/client/client_test.go b/internal/distributed/indexcoord/client/client_test.go index 2341214f7d..dbf25b0678 100644 --- a/internal/distributed/indexcoord/client/client_test.go +++ b/internal/distributed/indexcoord/client/client_test.go @@ -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) diff --git a/internal/distributed/indexcoord/service.go b/internal/distributed/indexcoord/service.go index bd681cfdf6..024c339c46 100644 --- a/internal/distributed/indexcoord/service.go +++ b/internal/distributed/indexcoord/service.go @@ -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) diff --git a/internal/distributed/proxy/service.go b/internal/distributed/proxy/service.go index 9f555150c8..70d5fefec2 100644 --- a/internal/distributed/proxy/service.go +++ b/internal/distributed/proxy/service.go @@ -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 diff --git a/internal/distributed/querycoord/client/client_test.go b/internal/distributed/querycoord/client/client_test.go index d212098522..4677073bb5 100644 --- a/internal/distributed/querycoord/client/client_test.go +++ b/internal/distributed/querycoord/client/client_test.go @@ -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) diff --git a/internal/distributed/querycoord/service.go b/internal/distributed/querycoord/service.go index 3fb296b9e6..ef88d34b04 100644 --- a/internal/distributed/querycoord/service.go +++ b/internal/distributed/querycoord/service.go @@ -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) diff --git a/internal/distributed/rootcoord/client/client_test.go b/internal/distributed/rootcoord/client/client_test.go index 1240d36519..914203783c 100644 --- a/internal/distributed/rootcoord/client/client_test.go +++ b/internal/distributed/rootcoord/client/client_test.go @@ -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) diff --git a/internal/distributed/rootcoord/service.go b/internal/distributed/rootcoord/service.go index e2fba67e25..99b995c17b 100644 --- a/internal/distributed/rootcoord/service.go +++ b/internal/distributed/rootcoord/service.go @@ -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) } diff --git a/internal/distributed/rootcoord/service_test.go b/internal/distributed/rootcoord/service_test.go index 7c0f5cb18e..0c532e142f 100644 --- a/internal/distributed/rootcoord/service_test.go +++ b/internal/distributed/rootcoord/service_test.go @@ -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() diff --git a/internal/indexcoord/index_builder.go b/internal/indexcoord/index_builder.go index 583d144305..100658e15f 100644 --- a/internal/indexcoord/index_builder.go +++ b/internal/indexcoord/index_builder.go @@ -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, } } diff --git a/internal/indexcoord/index_coord.go b/internal/indexcoord/index_coord.go index 3fac0521ef..4d60771006 100644 --- a/internal/indexcoord/index_coord.go +++ b/internal/indexcoord/index_coord.go @@ -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 } diff --git a/internal/indexcoord/index_coord_test.go b/internal/indexcoord/index_coord_test.go index 5e17db1e9f..e3db8324a7 100644 --- a/internal/indexcoord/index_coord_test.go +++ b/internal/indexcoord/index_coord_test.go @@ -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) } diff --git a/internal/indexcoord/metrics_info.go b/internal/indexcoord/metrics_info.go index f15f6cb5ad..9d9373a4d6 100644 --- a/internal/indexcoord/metrics_info.go +++ b/internal/indexcoord/metrics_info.go @@ -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), diff --git a/internal/indexnode/indexnode.go b/internal/indexnode/indexnode.go index c6b35e510e..b0e0262f9c 100644 --- a/internal/indexnode/indexnode.go +++ b/internal/indexnode/indexnode.go @@ -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") } diff --git a/internal/indexnode/indexnode_mock.go b/internal/indexnode/indexnode_mock.go index 321c61df62..45aef50094 100644 --- a/internal/indexnode/indexnode_mock.go +++ b/internal/indexnode/indexnode_mock.go @@ -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, }, } diff --git a/internal/indexnode/indexnode_service_test.go b/internal/indexnode/indexnode_service_test.go index 781e597ee9..55eb4fa9e2 100644 --- a/internal/indexnode/indexnode_service_test.go +++ b/internal/indexnode/indexnode_service_test.go @@ -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(), } } diff --git a/internal/indexnode/metrics_info.go b/internal/indexnode/metrics_info.go index 1ac9f6258d..4d1894e84c 100644 --- a/internal/indexnode/metrics_info.go +++ b/internal/indexnode/metrics_info.go @@ -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, }, } diff --git a/internal/kv/etcd/embed_etcd_restart_test.go b/internal/kv/etcd/embed_etcd_restart_test.go index d2218ae455..4fa035f397 100644 --- a/internal/kv/etcd/embed_etcd_restart_test.go +++ b/internal/kv/etcd/embed_etcd_restart_test.go @@ -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, ¶m.EtcdCfg) diff --git a/internal/kv/etcd/metakv_factory.go b/internal/kv/etcd/metakv_factory.go index 670eda34ca..424e7e5f4f 100644 --- a/internal/kv/etcd/metakv_factory.go +++ b/internal/kv/etcd/metakv_factory.go @@ -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 diff --git a/internal/mq/msgstream/mq_factory.go b/internal/mq/msgstream/mq_factory.go index bbcf93ad27..530b5a4753 100644 --- a/internal/mq/msgstream/mq_factory.go +++ b/internal/mq/msgstream/mq_factory.go @@ -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(), } } diff --git a/internal/mq/msgstream/mq_factory_test.go b/internal/mq/msgstream/mq_factory_test.go index 8dfa7f3b95..3840dc84cd 100644 --- a/internal/mq/msgstream/mq_factory_test.go +++ b/internal/mq/msgstream/mq_factory_test.go @@ -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 := ¶mtable.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() diff --git a/internal/mq/msgstream/mqwrapper/kafka/kafka_client.go b/internal/mq/msgstream/mqwrapper/kafka/kafka_client.go index d8f8bbce8c..163bbb6317 100644 --- a/internal/mq/msgstream/mqwrapper/kafka/kafka_client.go +++ b/internal/mq/msgstream/mqwrapper/kafka/kafka_client.go @@ -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())) } diff --git a/internal/mq/msgstream/mqwrapper/kafka/kafka_client_test.go b/internal/mq/msgstream/mqwrapper/kafka/kafka_client_test.go index 89b4909c8c..79c7d74057 100644 --- a/internal/mq/msgstream/mqwrapper/kafka/kafka_client_test.go +++ b/internal/mq/msgstream/mqwrapper/kafka/kafka_client_test.go @@ -298,20 +298,35 @@ func TestKafkaClient_MsgSerializAndDeserialize(t *testing.T) { } func TestKafkaClient_NewKafkaClientInstanceWithConfig(t *testing.T) { - config1 := ¶mtable.KafkaConfig{Address: "addr", SaslPassword: "password"} + config1 := ¶mtable.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 := ¶mtable.KafkaConfig{Address: "addr", SaslUsername: "username"} + config2 := ¶mtable.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 := ¶mtable.KafkaConfig{Address: "addr", SaslUsername: "username", SaslPassword: "password"} + config3 := ¶mtable.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 := ¶mtable.KafkaConfig{Address: "addr", SaslUsername: "username", SaslPassword: "password", ConsumerExtraConfig: consumerConfig} + config4 := ¶mtable.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 := ¶mtable.KafkaConfig{Address: "addr", SaslUsername: "username", SaslPassword: "password", ProducerExtraConfig: producerConfig} + config5 := ¶mtable.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"]) diff --git a/internal/proxy/accesslog/log_writer_test.go b/internal/proxy/accesslog/log_writer_test.go index c4b1d02809..afb8a173a9 100644 --- a/internal/proxy/accesslog/log_writer_test.go +++ b/internal/proxy/accesslog/log_writer_test.go @@ -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(¶ms.ProxyCfg.AccessLog, ¶ms.MinioCfg) assert.Error(t, err) } diff --git a/internal/proxy/accesslog/minio_handler.go b/internal/proxy/accesslog/minio_handler.go index e94e49bfe5..663058baf7 100644 --- a/internal/proxy/accesslog/minio_handler.go +++ b/internal/proxy/accesslog/minio_handler.go @@ -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) diff --git a/internal/proxy/accesslog/minio_handler_test.go b/internal/proxy/accesslog/minio_handler_test.go index bb5679a7e8..75615a9ed5 100644 --- a/internal/proxy/accesslog/minio_handler_test.go +++ b/internal/proxy/accesslog/minio_handler_test.go @@ -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, + ¶ms.MinioCfg, + params.ProxyCfg.AccessLog.RemotePath, + params.ProxyCfg.AccessLog.MaxBackups, ) assert.Error(t, err) } diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index 6dd2b0cee8..0f2e3ea791 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -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() diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index 7f3bf46af3..bb579867b6 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -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") diff --git a/internal/proxy/proxy_test.go b/internal/proxy/proxy_test.go index d835b3fa48..826d18b951 100644 --- a/internal/proxy/proxy_test.go +++ b/internal/proxy/proxy_test.go @@ -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) diff --git a/internal/proxy/task_insert.go b/internal/proxy/task_insert.go index 42aefdf3b4..cced718d6b 100644 --- a/internal/proxy/task_insert.go +++ b/internal/proxy/task_insert.go @@ -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(), diff --git a/internal/querycoordv2/balance/rowcount_based_balancer_test.go b/internal/querycoordv2/balance/rowcount_based_balancer_test.go index 6d62b89212..b3e6842ada 100644 --- a/internal/querycoordv2/balance/rowcount_based_balancer_test.go +++ b/internal/querycoordv2/balance/rowcount_based_balancer_test.go @@ -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() diff --git a/internal/querycoordv2/checkers/channel_checker_test.go b/internal/querycoordv2/checkers/channel_checker_test.go index 3d452b2041..2c27888111 100644 --- a/internal/querycoordv2/checkers/channel_checker_test.go +++ b/internal/querycoordv2/checkers/channel_checker_test.go @@ -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) diff --git a/internal/querycoordv2/checkers/segment_checker_test.go b/internal/querycoordv2/checkers/segment_checker_test.go index 9cfb67edb0..e16422122b 100644 --- a/internal/querycoordv2/checkers/segment_checker_test.go +++ b/internal/querycoordv2/checkers/segment_checker_test.go @@ -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) diff --git a/internal/querycoordv2/dist/dist_controller_test.go b/internal/querycoordv2/dist/dist_controller_test.go index 3fbbed2bfb..bd576ca0e2 100644 --- a/internal/querycoordv2/dist/dist_controller_test.go +++ b/internal/querycoordv2/dist/dist_controller_test.go @@ -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) diff --git a/internal/querycoordv2/job/job_test.go b/internal/querycoordv2/job/job_test.go index 3a7c2e295c..e78453e2fa 100644 --- a/internal/querycoordv2/job/job_test.go +++ b/internal/querycoordv2/job/job_test.go @@ -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() diff --git a/internal/querycoordv2/meta/collection_manager_test.go b/internal/querycoordv2/meta/collection_manager_test.go index ea767ce313..9a7a11f260 100644 --- a/internal/querycoordv2/meta/collection_manager_test.go +++ b/internal/querycoordv2/meta/collection_manager_test.go @@ -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) diff --git a/internal/querycoordv2/meta/replica_manager_test.go b/internal/querycoordv2/meta/replica_manager_test.go index 397e81203f..06f7ddbc27 100644 --- a/internal/querycoordv2/meta/replica_manager_test.go +++ b/internal/querycoordv2/meta/replica_manager_test.go @@ -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() diff --git a/internal/querycoordv2/meta/target_manager_test.go b/internal/querycoordv2/meta/target_manager_test.go index 342c920c4f..467c4cc679 100644 --- a/internal/querycoordv2/meta/target_manager_test.go +++ b/internal/querycoordv2/meta/target_manager_test.go @@ -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) diff --git a/internal/querycoordv2/mocks/querynode.go b/internal/querycoordv2/mocks/querynode.go index 6615517250..3c078faf59 100644 --- a/internal/querycoordv2/mocks/querynode.go +++ b/internal/querycoordv2/mocks/querynode.go @@ -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, diff --git a/internal/querycoordv2/observers/collection_observer_test.go b/internal/querycoordv2/observers/collection_observer_test.go index 02731185df..6cb5cf2935 100644 --- a/internal/querycoordv2/observers/collection_observer_test.go +++ b/internal/querycoordv2/observers/collection_observer_test.go @@ -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) diff --git a/internal/querycoordv2/observers/leader_observer_test.go b/internal/querycoordv2/observers/leader_observer_test.go index d792907dcb..9985cb3998 100644 --- a/internal/querycoordv2/observers/leader_observer_test.go +++ b/internal/querycoordv2/observers/leader_observer_test.go @@ -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) diff --git a/internal/querycoordv2/observers/target_observer_test.go b/internal/querycoordv2/observers/target_observer_test.go index cd3179f77b..b2922b15e9 100644 --- a/internal/querycoordv2/observers/target_observer_test.go +++ b/internal/querycoordv2/observers/target_observer_test.go @@ -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) diff --git a/internal/querycoordv2/params/params.go b/internal/querycoordv2/params/params.go index 02acf6246e..dc15984722 100644 --- a/internal/querycoordv2/params/params.go +++ b/internal/querycoordv2/params/params.go @@ -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 { diff --git a/internal/querycoordv2/server.go b/internal/querycoordv2/server.go index 70106bda23..217f54e790 100644 --- a/internal/querycoordv2/server.go +++ b/internal/querycoordv2/server.go @@ -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 { diff --git a/internal/querycoordv2/server_test.go b/internal/querycoordv2/server_test.go index 6baa83643c..ad237376c2 100644 --- a/internal/querycoordv2/server_test.go +++ b/internal/querycoordv2/server_test.go @@ -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{ diff --git a/internal/querycoordv2/services_test.go b/internal/querycoordv2/services_test.go index ad2bd756d4..5061d9f7c3 100644 --- a/internal/querycoordv2/services_test.go +++ b/internal/querycoordv2/services_test.go @@ -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, diff --git a/internal/querycoordv2/task/task_test.go b/internal/querycoordv2/task/task_test.go index 94629e6285..b7a1900522 100644 --- a/internal/querycoordv2/task/task_test.go +++ b/internal/querycoordv2/task/task_test.go @@ -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() diff --git a/internal/querynode/impl_test.go b/internal/querynode/impl_test.go index ed83985e50..357c6aa5f3 100644 --- a/internal/querynode/impl_test.go +++ b/internal/querynode/impl_test.go @@ -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" diff --git a/internal/querynode/load_index_info.go b/internal/querynode/load_index_info.go index ac55bedbca..a63b096b88 100644 --- a/internal/querynode/load_index_info.go +++ b/internal/querynode/load_index_info.go @@ -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) diff --git a/internal/querynode/metrics_info_test.go b/internal/querynode/metrics_info_test.go index 2aab185056..2fdff492b5 100644 --- a/internal/querynode/metrics_info_test.go +++ b/internal/querynode/metrics_info_test.go @@ -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), diff --git a/internal/querynode/mock_test.go b/internal/querynode/mock_test.go index d454541fbc..6b81c40e0a 100644 --- a/internal/querynode/mock_test.go +++ b/internal/querynode/mock_test.go @@ -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() diff --git a/internal/querynode/query_node.go b/internal/querynode/query_node.go index d9e5271a3c..6dffcc2dfc 100644 --- a/internal/querynode/query_node.go +++ b/internal/querynode/query_node.go @@ -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) diff --git a/internal/querynode/query_node_test.go b/internal/querynode/query_node_test.go index 2b483d17d7..063ef1af93 100644 --- a/internal/querynode/query_node_test.go +++ b/internal/querynode/query_node_test.go @@ -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) diff --git a/internal/querynode/query_shard_service.go b/internal/querynode/query_shard_service.go index dd06f388fb..6594abbc11 100644 --- a/internal/querynode/query_shard_service.go +++ b/internal/querynode/query_shard_service.go @@ -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)) diff --git a/internal/querynode/shard_cluster_service.go b/internal/querynode/shard_cluster_service.go index 0b354ca4d4..3c8f73454d 100644 --- a/internal/querynode/shard_cluster_service.go +++ b/internal/querynode/shard_cluster_service.go @@ -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 { diff --git a/internal/rootcoord/proxy_client_manager.go b/internal/rootcoord/proxy_client_manager.go index 75524bd773..251914f43e 100644 --- a/internal/rootcoord/proxy_client_manager.go +++ b/internal/rootcoord/proxy_client_manager.go @@ -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() diff --git a/internal/rootcoord/proxy_manager.go b/internal/rootcoord/proxy_manager.go index 0cd92aaeab..6bb88d67cb 100644 --- a/internal/rootcoord/proxy_manager.go +++ b/internal/rootcoord/proxy_manager.go @@ -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), ) diff --git a/internal/rootcoord/proxy_manager_test.go b/internal/rootcoord/proxy_manager_test.go index 03533ad5aa..ff31a44c81 100644 --- a/internal/rootcoord/proxy_manager_test.go +++ b/internal/rootcoord/proxy_manager_test.go @@ -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(), diff --git a/internal/rootcoord/quota_center.go b/internal/rootcoord/quota_center.go index eb00ecec0e..9c2fcb2c4e 100644 --- a/internal/rootcoord/quota_center.go +++ b/internal/rootcoord/quota_center.go @@ -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 } diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index 0e0431fe23..5a1545d16b 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -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 } diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 871e452005..906b573ae9 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -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) diff --git a/internal/storage/factory.go b/internal/storage/factory.go index ceedeed10c..8b4d2656cb 100644 --- a/internal/storage/factory.go +++ b/internal/storage/factory.go @@ -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)) } diff --git a/internal/storage/minio_chunk_manager.go b/internal/storage/minio_chunk_manager.go index a85890f8c7..58ca4d394d 100644 --- a/internal/storage/minio_chunk_manager.go +++ b/internal/storage/minio_chunk_manager.go @@ -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, "") diff --git a/internal/util/etcd/etcd_server.go b/internal/util/etcd/etcd_server.go index c38be3f80a..487b00cf7f 100644 --- a/internal/util/etcd/etcd_server.go +++ b/internal/util/etcd/etcd_server.go @@ -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 } diff --git a/internal/util/etcd/etcd_util.go b/internal/util/etcd/etcd_util.go index f63085c04b..1ed6b98054 100644 --- a/internal/util/etcd/etcd_util.go +++ b/internal/util/etcd/etcd_util.go @@ -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 diff --git a/internal/util/etcd/etcd_util_test.go b/internal/util/etcd/etcd_util_test.go index 3d5cc7f9d7..136bfc23af 100644 --- a/internal/util/etcd/etcd_util_test.go +++ b/internal/util/etcd/etcd_util_test.go @@ -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) } diff --git a/internal/util/indexcgowrapper/codec_index_test.go b/internal/util/indexcgowrapper/codec_index_test.go index e9bcea7832..d722382670 100644 --- a/internal/util/indexcgowrapper/codec_index_test.go +++ b/internal/util/indexcgowrapper/codec_index_test.go @@ -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(), } } diff --git a/internal/util/paramtable/base_table.go b/internal/util/paramtable/base_table.go index 140939d4c4..a76a79374b 100644 --- a/internal/util/paramtable/base_table.go +++ b/internal/util/paramtable/base_table.go @@ -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) diff --git a/internal/util/paramtable/base_table_test.go b/internal/util/paramtable/base_table_test.go index 0ebc07b654..f2a94e609d 100644 --- a/internal/util/paramtable/base_table_test.go +++ b/internal/util/paramtable/base_table_test.go @@ -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 { diff --git a/internal/util/paramtable/component_param.go b/internal/util/paramtable/component_param.go index 05ffeeed2a..85fd6b1577 100644 --- a/internal/util/paramtable/component_param.go +++ b/internal/util/paramtable/component_param.go @@ -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() != "" } // ///////////////////////////////////////////////////////////////////////////// diff --git a/internal/util/paramtable/component_param_test.go b/internal/util/paramtable/component_param_test.go index b07210f688..e4e8d02b8e 100644 --- a/internal/util/paramtable/component_param_test.go +++ b/internal/util/paramtable/component_param_test.go @@ -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"]) }) diff --git a/internal/util/paramtable/param_item.go b/internal/util/paramtable/param_item.go index 19964b65fb..c6d53826c3 100644 --- a/internal/util/paramtable/param_item.go +++ b/internal/util/paramtable/param_item.go @@ -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 } diff --git a/internal/util/paramtable/service_param.go b/internal/util/paramtable/service_param.go index b3c212ce55..b3604a3207 100644 --- a/internal/util/paramtable/service_param.go +++ b/internal/util/paramtable/service_param.go @@ -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) LoadCfgToMemory() { - p.initUseEmbedEtcd() - if p.UseEmbedEtcd { - p.initConfigPath() - p.initDataDir() - } else { - p.initEndpoints() +func (p *EtcdConfig) Init(base *BaseTable) { + p.Endpoints = ParamItem{ + Key: "etcd.endpoints", + Version: "2.0.0", + PanicIfEmpty: true, } - p.initMetaRootPath() - p.initKvRootPath() - p.initEtcdLogLevel() - p.initEtcdLogPath() - p.initEtcdUseSSL() - p.initEtcdTLSCert() - p.initEtcdTLSKey() - p.initEtcdTLSCACert() - p.initEtcdTLSMinVersion() -} + p.Endpoints.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) { + p.UseEmbedEtcd = ParamItem{ + Key: "etcd.use.embed", + DefaultValue: "false", + Version: "2.1.0", + } + p.UseEmbedEtcd.Init(base.mgr) + + if p.UseEmbedEtcd.GetAsBool() && (os.Getenv(metricsinfo.DeployModeEnvKey) != metricsinfo.StandaloneDeployMode) { panic("embedded etcd can not be used under distributed mode") } -} -func (p *EtcdConfig) initConfigPath() { - addr := p.Base.LoadWithDefault("etcd.config.path", "") - p.ConfigPath = addr -} + if p.UseEmbedEtcd.GetAsBool() { + p.ConfigPath = ParamItem{ + Key: "etcd.config.path", + DefaultValue: "", + Version: "2.1.0", + } + p.ConfigPath.Init(base.mgr) -func (p *EtcdConfig) initDataDir() { - addr := p.Base.LoadWithDefault("etcd.data.dir", "default.etcd") - p.DataDir = addr -} - -func (p *EtcdConfig) initEndpoints() { - endpoints, err := p.Base.Load("etcd.endpoints") - if err != nil { - panic(err) + p.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) } - p.Endpoints = strings.Split(endpoints, ",") -} -func (p *EtcdConfig) initMetaRootPath() { - rootPath, err := p.Base.Load("etcd.rootPath") - if err != nil { - panic(err) + p.RootPath = ParamItem{ + Key: "etcd.rootPath", + Version: "2.0.0", + PanicIfEmpty: true, } - subPath, err := p.Base.Load("etcd.metaSubPath") - if err != nil { - panic(err) + p.RootPath.Init(base.mgr) + + p.MetaSubPath = ParamItem{ + Key: "etcd.metaSubPath", + Version: "2.0.0", + PanicIfEmpty: true, } - p.MetaRootPath = path.Join(rootPath, subPath) -} + p.MetaSubPath.Init(base.mgr) -func (p *EtcdConfig) initKvRootPath() { - rootPath, err := p.Base.Load("etcd.rootPath") - if err != nil { - panic(err) + 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"]) + }, } - subPath, err := p.Base.Load("etcd.kvSubPath") - if err != nil { - panic(err) + + p.KvSubPath = ParamItem{ + Key: "etcd.kvSubPath", + Version: "2.0.0", + PanicIfEmpty: true, } - p.KvRootPath = path.Join(rootPath, subPath) -} + p.KvSubPath.Init(base.mgr) -func (p *EtcdConfig) initEtcdLogLevel() { - p.EtcdLogLevel = p.Base.LoadWithDefault("etcd.log.level", defaultEtcdLogLevel) -} + 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) initEtcdLogPath() { - p.EtcdLogPath = p.Base.LoadWithDefault("etcd.log.path", defaultEtcdLogPath) -} + p.EtcdLogLevel = ParamItem{ + Key: "etcd.log.level", + DefaultValue: defaultEtcdLogLevel, + Version: "2.0.0", + } + p.EtcdLogLevel.Init(base.mgr) -func (p *EtcdConfig) initEtcdUseSSL() { - p.EtcdUseSSL = p.Base.ParseBool("etcd.ssl.enabled", false) -} + p.EtcdLogPath = ParamItem{ + Key: "etcd.log.path", + DefaultValue: defaultEtcdLogPath, + Version: "2.0.0", + } + p.EtcdLogPath.Init(base.mgr) -func (p *EtcdConfig) initEtcdTLSCert() { - p.EtcdTLSCert = p.Base.LoadWithDefault("etcd.ssl.tlsCert", "") -} + p.EtcdUseSSL = ParamItem{ + Key: "etcd.ssl.enabled", + DefaultValue: "false", + Version: "2.0.0", + } + p.EtcdUseSSL.Init(base.mgr) -func (p *EtcdConfig) initEtcdTLSKey() { - p.EtcdTLSKey = p.Base.LoadWithDefault("etcd.ssl.tlsKey", "") -} + p.EtcdTLSCert = ParamItem{ + Key: "etcd.ssl.tlsCert", + Version: "2.0.0", + } + p.EtcdTLSCert.Init(base.mgr) -func (p *EtcdConfig) initEtcdTLSCACert() { - p.EtcdTLSCACert = p.Base.LoadWithDefault("etcd.ssl.tlsCACert", "") -} + p.EtcdTLSKey = ParamItem{ + Key: "etcd.ssl.tlsKey", + Version: "2.0.0", + } + p.EtcdTLSKey.Init(base.mgr) -func (p *EtcdConfig) initEtcdTLSMinVersion() { - p.EtcdTLSMinVersion = p.Base.LoadWithDefault("etcd.ssl.tlsMinVersion", "1.3") + p.EtcdTLSCACert = ParamItem{ + Key: "etcd.ssl.tlsCACert", + Version: "2.0.0", + } + p.EtcdTLSCACert.Init(base.mgr) + + p.EtcdTLSMinVersion = ParamItem{ + Key: "etcd.ssl.tlsMinVersion", + DefaultValue: "1.3", + Version: "2.0.0", + } + p.EtcdTLSMinVersion.Init(base.mgr) } type LocalStorageConfig struct { - Base *BaseTable - - Path string + Path ParamItem } -func (p *LocalStorageConfig) init(base *BaseTable) { - p.Base = base - p.initPath() -} - -func (p *LocalStorageConfig) initPath() { - p.Path = p.Base.LoadWithDefault("localStorage.path", "/var/lib/milvus/data") +func (p *LocalStorageConfig) Init(base *BaseTable) { + p.Path = ParamItem{ + Key: "localStorage.path", + Version: "2.0.0", + DefaultValue: "/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,286 +328,287 @@ 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) initAddress() { - pulsarHost := p.Base.LoadWithDefault("pulsar.address", "") - if strings.Contains(pulsarHost, ":") { - p.Address = pulsarHost - return +func (p *PulsarConfig) Init(base *BaseTable) { + p.Port = ParamItem{ + Key: "pulsar.port", + Version: "2.0.0", + DefaultValue: "6650", } + p.Port.Init(base.mgr) - port := p.Base.LoadWithDefault("pulsar.port", "") - if len(pulsarHost) != 0 && len(port) != 0 { - p.Address = "pulsar://" + pulsarHost + ":" + port + 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) -func (p *PulsarConfig) initWebAddress() { - if p.Address == "" { - return + p.WebPort = ParamItem{ + Key: "pulsar.webport", + Version: "2.0.0", + DefaultValue: "80", } + p.WebPort.Init(base.mgr) - pulsarURL, err := url.ParseRequestURI(p.Address) - 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 + 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 { + log.Info("failed to parse pulsar config, assume pulsar not used", zap.Error(err)) + 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", "") -} - -func (p *PulsarConfig) initAuthParams() { - paramString := p.Base.LoadWithDefault("pulsar.authParams", "") - - // 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 - jsonMap := make(map[string]string) - params := strings.Split(paramString, ",") - for _, param := range params { - kv := strings.Split(param, ":") - if len(kv) == 2 { - jsonMap[kv[0]] = kv[1] - } + p.Tenant = ParamItem{ + Key: "pulsar.tenant", + Version: "2.2.0", + DefaultValue: "public", } + p.Tenant.Init(base.mgr) - if len(jsonMap) == 0 { - p.AuthParams = "" - } else { - jsonData, _ := json.Marshal(&jsonMap) - p.AuthParams = string(jsonData) + p.Namespace = ParamItem{ + Key: "pulsar.namespace", + Version: "2.2.0", + DefaultValue: "default", } -} + p.Namespace.Init(base.mgr) -func (p *PulsarConfig) initTenant() { - p.Tenant = p.Base.LoadWithDefault("pulsar.tenant", "public") -} + 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(authParams, ",") + for _, param := range params { + kv := strings.Split(param, ":") + if len(kv) == 2 { + jsonMap[kv[0]] = kv[1] + } + } + + if len(jsonMap) == 0 { + return "" + } + jsonData, _ := json.Marshal(&jsonMap) + return string(jsonData) + }, + } + p.AuthParams.Init(base.mgr) -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 (p *RocksmqConfig) initPath() { - p.Path = p.Base.LoadWithDefault("rocksmq.path", "") +func (r *RocksmqConfig) Init(base *BaseTable) { + r.Path = ParamItem{ + Key: "rocksmq.path", + DefaultValue: "", + Version: "2.0.0", + } + 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 - - p.initAddress() - p.initAccessKeyID() - p.initSecretAccessKey() - p.initUseSSL() - p.initBucketName() - p.initRootPath() - p.initUseIAM() - p.initCloudProvider() - p.initIAMEndpoint() -} - -func (p *MinioConfig) initAddress() { - host, err := p.Base.Load("minio.Address") - if err != nil { - panic(err) +func (p *MinioConfig) Init(base *BaseTable) { + p.Port = ParamItem{ + Key: "minio.port", + DefaultValue: "9000", + Version: "2.0.0", } - // for compatible - if strings.Contains(host, ":") { - p.Address = host - } else { - port := p.Base.LoadWithDefault("minio.port", "9000") - p.Address = host + ":" + port + p.Port.Init(base.mgr) + + 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) initAccessKeyID() { - keyID, err := p.Base.Load("minio.accessKeyID") - if err != nil { - panic(err) + p.AccessKeyID = ParamItem{ + Key: "minio.accessKeyID", + Version: "2.0.0", + PanicIfEmpty: true, } - p.AccessKeyID = keyID -} + p.AccessKeyID.Init(base.mgr) -func (p *MinioConfig) initSecretAccessKey() { - key, err := p.Base.Load("minio.secretAccessKey") - if err != nil { - panic(err) + p.SecretAccessKey = ParamItem{ + Key: "minio.secretAccessKey", + Version: "2.0.0", + PanicIfEmpty: true, } - p.SecretAccessKey = key -} + p.SecretAccessKey.Init(base.mgr) -func (p *MinioConfig) initUseSSL() { - usessl, err := p.Base.Load("minio.useSSL") - if err != nil { - panic(err) + p.UseSSL = ParamItem{ + Key: "minio.useSSL", + Version: "2.0.0", + PanicIfEmpty: true, } - p.UseSSL, _ = strconv.ParseBool(usessl) -} + p.UseSSL.Init(base.mgr) -func (p *MinioConfig) initBucketName() { - bucketName, err := p.Base.Load("minio.bucketName") - if err != nil { - panic(err) + p.BucketName = ParamItem{ + Key: "minio.bucketName", + Version: "2.0.0", + PanicIfEmpty: true, } - p.BucketName = bucketName -} + p.BucketName.Init(base.mgr) -func (p *MinioConfig) initRootPath() { - rootPath, err := p.Base.Load("minio.rootPath") - if err != nil { - panic(err) + p.RootPath = ParamItem{ + Key: "minio.rootPath", + Version: "2.0.0", + PanicIfEmpty: true, } - p.RootPath = rootPath -} + p.RootPath.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.UseIAM = ParamItem{ + Key: "minio.useIAM", + DefaultValue: DefaultMinioUseIAM, + Version: "2.0.0", } -} + p.UseIAM.Init(base.mgr) -// CloudProvider supported -const ( - CloudProviderAWS = "aws" - CloudProviderGCP = "gcp" -) - -var supportedCloudProvider = map[string]bool{ - CloudProviderAWS: true, - CloudProviderGCP: true, -} - -func (p *MinioConfig) initCloudProvider() { - p.CloudProvider = p.Base.LoadWithDefault("minio.cloudProvider", DefaultMinioCloudProvider) - if !supportedCloudProvider[p.CloudProvider] { - panic("unsupported cloudProvider:" + p.CloudProvider) + p.CloudProvider = ParamItem{ + Key: "minio.cloudProvider", + DefaultValue: DefaultMinioCloudProvider, + Version: "2.2.0", } -} + p.CloudProvider.Init(base.mgr) -func (p *MinioConfig) initIAMEndpoint() { - p.IAMEndpoint = p.Base.LoadWithDefault("minio.iamEndpoint", DefaultMinioIAMEndpoint) + p.IAMEndpoint = ParamItem{ + Key: "minio.iamEndpoint", + DefaultValue: DefaultMinioIAMEndpoint, + Version: "2.0.0", + } + p.IAMEndpoint.Init(base.mgr) } diff --git a/internal/util/paramtable/service_param_test.go b/internal/util/paramtable/service_param_test.go index 78e0175ce2..26c159ad34 100644 --- a/internal/util/paramtable/service_param_test.go +++ b/internal/util/paramtable/service_param_test.go @@ -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()) }) }