mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
Add event dispatcher for config (#20393)
Signed-off-by: Enwei Jiao <enwei.jiao@zilliz.com> Signed-off-by: Enwei Jiao <enwei.jiao@zilliz.com>
This commit is contained in:
parent
cc371d6801
commit
c05b9ad539
@ -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))
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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]
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -23,14 +23,7 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
const (
|
||||
ModeWatch = iota
|
||||
ModeInterval
|
||||
)
|
||||
|
||||
type EtcdSource struct {
|
||||
@ -39,10 +32,8 @@ type EtcdSource struct {
|
||||
ctx context.Context
|
||||
currentConfig map[string]string
|
||||
keyPrefix string
|
||||
refreshMode int
|
||||
refreshInterval time.Duration
|
||||
intervalDone chan bool
|
||||
intervalInitOnce sync.Once
|
||||
|
||||
configRefresher refresher
|
||||
eh EventHandler
|
||||
}
|
||||
|
||||
@ -54,15 +45,14 @@ func NewEtcdSource(remoteInfo *EtcdInfo) (*EtcdSource, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &EtcdSource{
|
||||
es := &EtcdSource{
|
||||
etcdCli: etcdCli,
|
||||
ctx: context.Background(),
|
||||
currentConfig: make(map[string]string),
|
||||
keyPrefix: remoteInfo.KeyPrefix,
|
||||
refreshMode: remoteInfo.RefreshMode,
|
||||
refreshInterval: remoteInfo.RefreshInterval,
|
||||
intervalDone: make(chan bool, 1),
|
||||
}, nil
|
||||
}
|
||||
es.configRefresher = newRefresher(remoteInfo.RefreshInterval, es.refreshConfigurations)
|
||||
return es, nil
|
||||
}
|
||||
|
||||
// GetConfigurationByKey implements ConfigSource
|
||||
@ -83,12 +73,7 @@ func (es *EtcdSource) GetConfigurations() (map[string]string, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if es.refreshMode == ModeInterval {
|
||||
es.intervalInitOnce.Do(func() {
|
||||
go es.refreshConfigurationsPeriodically()
|
||||
})
|
||||
}
|
||||
|
||||
es.configRefresher.start()
|
||||
es.RLock()
|
||||
for key, value := range es.currentConfig {
|
||||
configMap[key] = value
|
||||
@ -109,7 +94,11 @@ func (es *EtcdSource) GetSourceName() string {
|
||||
}
|
||||
|
||||
func (es *EtcdSource) Close() {
|
||||
es.intervalDone <- true
|
||||
es.configRefresher.stop()
|
||||
}
|
||||
|
||||
func (es *EtcdSource) SetEventHandler(eh EventHandler) {
|
||||
es.configRefresher.eh = eh
|
||||
}
|
||||
|
||||
func (es *EtcdSource) refreshConfigurations() error {
|
||||
@ -125,42 +114,12 @@ func (es *EtcdSource) refreshConfigurations() error {
|
||||
newConfig[key] = string(kv.Value)
|
||||
newConfig[formatKey(key)] = string(kv.Value)
|
||||
}
|
||||
return es.updateConfigurationAndFireEvent(newConfig)
|
||||
}
|
||||
|
||||
func (es *EtcdSource) refreshConfigurationsPeriodically() {
|
||||
ticker := time.NewTicker(es.refreshInterval)
|
||||
log.Info("start refreshing configurations")
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
err := es.refreshConfigurations()
|
||||
if err != nil {
|
||||
log.Error("can not pull configs", zap.Error(err))
|
||||
es.intervalDone <- true
|
||||
}
|
||||
case <-es.intervalDone:
|
||||
log.Info("stop refreshing configurations")
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (es *EtcdSource) updateConfigurationAndFireEvent(config map[string]string) error {
|
||||
es.Lock()
|
||||
defer es.Unlock()
|
||||
//Populate the events based on the changed value between current config and newly received Config
|
||||
events, err := PopulateEvents(es.GetSourceName(), es.currentConfig, config)
|
||||
err = es.configRefresher.fireEvents(es.GetSourceName(), es.currentConfig, newConfig)
|
||||
if err != nil {
|
||||
log.Warn("generating event error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
es.currentConfig = config
|
||||
//Generate OnEvent Callback based on the events created
|
||||
if es.eh != nil {
|
||||
for _, e := range events {
|
||||
es.eh.OnEvent(e)
|
||||
}
|
||||
}
|
||||
es.currentConfig = newConfig
|
||||
return nil
|
||||
}
|
||||
|
||||
66
internal/config/event_dispatcher.go
Normal file
66
internal/config/event_dispatcher.go
Normal file
@ -0,0 +1,66 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
package config
|
||||
|
||||
type EventDispatcher struct {
|
||||
registry map[string][]EventHandler
|
||||
}
|
||||
|
||||
func NewEventDispatcher() *EventDispatcher {
|
||||
return &EventDispatcher{
|
||||
registry: make(map[string][]EventHandler),
|
||||
}
|
||||
}
|
||||
|
||||
func (ed *EventDispatcher) Get(key string) []EventHandler {
|
||||
return ed.registry[formatKey(key)]
|
||||
}
|
||||
|
||||
func (ed *EventDispatcher) Dispatch(event *Event) {
|
||||
hs, ok := ed.registry[event.Key]
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
for _, h := range hs {
|
||||
h.OnEvent(event)
|
||||
}
|
||||
}
|
||||
|
||||
// register a handler to watch specific config changed
|
||||
func (ed *EventDispatcher) Register(key string, handler EventHandler) {
|
||||
key = formatKey(key)
|
||||
v, ok := ed.registry[key]
|
||||
if ok {
|
||||
ed.registry[key] = append(v, handler)
|
||||
} else {
|
||||
ed.registry[key] = []EventHandler{handler}
|
||||
}
|
||||
}
|
||||
|
||||
func (ed *EventDispatcher) Unregister(key string, handler EventHandler) {
|
||||
v, ok := ed.registry[key]
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
newGroup := make([]EventHandler, 0)
|
||||
for _, eh := range v {
|
||||
if eh.GetIdentifier() == handler.GetIdentifier() {
|
||||
continue
|
||||
}
|
||||
newGroup = append(newGroup, eh)
|
||||
}
|
||||
ed.registry[key] = newGroup
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
|
||||
87
internal/config/refresher.go
Normal file
87
internal/config/refresher.go
Normal file
@ -0,0 +1,87 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
package config
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type refresher struct {
|
||||
refreshInterval time.Duration
|
||||
intervalDone chan bool
|
||||
intervalInitOnce sync.Once
|
||||
eh EventHandler
|
||||
|
||||
fetchFunc func() error
|
||||
}
|
||||
|
||||
func newRefresher(interval time.Duration, fetchFunc func() error) refresher {
|
||||
return refresher{
|
||||
refreshInterval: interval,
|
||||
intervalDone: make(chan bool, 1),
|
||||
fetchFunc: fetchFunc,
|
||||
}
|
||||
}
|
||||
|
||||
func (r refresher) start() {
|
||||
if r.refreshInterval > 0 {
|
||||
r.intervalInitOnce.Do(func() {
|
||||
go r.refreshPeriodically()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func (r refresher) stop() {
|
||||
r.intervalDone <- true
|
||||
}
|
||||
|
||||
func (r refresher) refreshPeriodically() {
|
||||
ticker := time.NewTicker(r.refreshInterval)
|
||||
log.Info("start refreshing configurations")
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
err := r.fetchFunc()
|
||||
if err != nil {
|
||||
log.Error("can not pull configs", zap.Error(err))
|
||||
r.intervalDone <- true
|
||||
}
|
||||
case <-r.intervalDone:
|
||||
log.Info("stop refreshing configurations")
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func (r refresher) fireEvents(name string, source, target map[string]string) error {
|
||||
events, err := PopulateEvents(name, source, target)
|
||||
if err != nil {
|
||||
log.Warn("generating event error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
//Generate OnEvent Callback based on the events created
|
||||
if r.eh != nil {
|
||||
for _, e := range events {
|
||||
r.eh.OnEvent(e)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -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)
|
||||
})
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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}))
|
||||
|
||||
@ -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),
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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,
|
||||
}
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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),
|
||||
|
||||
@ -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")
|
||||
}
|
||||
|
||||
@ -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,
|
||||
},
|
||||
}
|
||||
|
||||
@ -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(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -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,
|
||||
},
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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()))
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -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"])
|
||||
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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")
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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(),
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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{
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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"
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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),
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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()
|
||||
|
||||
|
||||
@ -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),
|
||||
)
|
||||
|
||||
@ -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(),
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
|
||||
@ -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, "")
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
}
|
||||
|
||||
@ -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(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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() != ""
|
||||
}
|
||||
|
||||
// /////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
@ -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"])
|
||||
})
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
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)
|
||||
if p.UseEmbedEtcd.GetAsBool() {
|
||||
p.ConfigPath = ParamItem{
|
||||
Key: "etcd.config.path",
|
||||
DefaultValue: "",
|
||||
Version: "2.1.0",
|
||||
}
|
||||
p.Endpoints = strings.Split(endpoints, ",")
|
||||
}
|
||||
p.ConfigPath.Init(base.mgr)
|
||||
|
||||
func (p *EtcdConfig) initMetaRootPath() {
|
||||
rootPath, err := p.Base.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
p.DataDir = ParamItem{
|
||||
Key: "etcd.data.dir",
|
||||
DefaultValue: "default.etcd",
|
||||
Version: "2.1.0",
|
||||
}
|
||||
subPath, err := p.Base.Load("etcd.metaSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
p.DataDir.Init(base.mgr)
|
||||
} else {
|
||||
p.Endpoints = ParamItem{
|
||||
Key: "etcd.endpoints",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
p.MetaRootPath = path.Join(rootPath, subPath)
|
||||
}
|
||||
|
||||
func (p *EtcdConfig) initKvRootPath() {
|
||||
rootPath, err := p.Base.Load("etcd.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
p.Endpoints.Init(base.mgr)
|
||||
}
|
||||
subPath, err := p.Base.Load("etcd.kvSubPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
||||
p.RootPath = ParamItem{
|
||||
Key: "etcd.rootPath",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
p.KvRootPath = path.Join(rootPath, subPath)
|
||||
}
|
||||
p.RootPath.Init(base.mgr)
|
||||
|
||||
func (p *EtcdConfig) initEtcdLogLevel() {
|
||||
p.EtcdLogLevel = p.Base.LoadWithDefault("etcd.log.level", defaultEtcdLogLevel)
|
||||
}
|
||||
p.MetaSubPath = ParamItem{
|
||||
Key: "etcd.metaSubPath",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
p.MetaSubPath.Init(base.mgr)
|
||||
|
||||
func (p *EtcdConfig) initEtcdLogPath() {
|
||||
p.EtcdLogPath = p.Base.LoadWithDefault("etcd.log.path", defaultEtcdLogPath)
|
||||
}
|
||||
p.MetaRootPath = CompositeParamItem{
|
||||
Items: []*ParamItem{&p.RootPath, &p.MetaSubPath},
|
||||
Format: func(kvs map[string]string) string {
|
||||
return path.Join(kvs["etcd.rootPath"], kvs["etcd.metaSubPath"])
|
||||
},
|
||||
}
|
||||
|
||||
func (p *EtcdConfig) initEtcdUseSSL() {
|
||||
p.EtcdUseSSL = p.Base.ParseBool("etcd.ssl.enabled", false)
|
||||
}
|
||||
p.KvSubPath = ParamItem{
|
||||
Key: "etcd.kvSubPath",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
p.KvSubPath.Init(base.mgr)
|
||||
|
||||
func (p *EtcdConfig) initEtcdTLSCert() {
|
||||
p.EtcdTLSCert = p.Base.LoadWithDefault("etcd.ssl.tlsCert", "")
|
||||
}
|
||||
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) initEtcdTLSKey() {
|
||||
p.EtcdTLSKey = p.Base.LoadWithDefault("etcd.ssl.tlsKey", "")
|
||||
}
|
||||
p.EtcdLogLevel = ParamItem{
|
||||
Key: "etcd.log.level",
|
||||
DefaultValue: defaultEtcdLogLevel,
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.EtcdLogLevel.Init(base.mgr)
|
||||
|
||||
func (p *EtcdConfig) initEtcdTLSCACert() {
|
||||
p.EtcdTLSCACert = p.Base.LoadWithDefault("etcd.ssl.tlsCACert", "")
|
||||
}
|
||||
p.EtcdLogPath = ParamItem{
|
||||
Key: "etcd.log.path",
|
||||
DefaultValue: defaultEtcdLogPath,
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.EtcdLogPath.Init(base.mgr)
|
||||
|
||||
func (p *EtcdConfig) initEtcdTLSMinVersion() {
|
||||
p.EtcdTLSMinVersion = p.Base.LoadWithDefault("etcd.ssl.tlsMinVersion", "1.3")
|
||||
p.EtcdUseSSL = ParamItem{
|
||||
Key: "etcd.ssl.enabled",
|
||||
DefaultValue: "false",
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.EtcdUseSSL.Init(base.mgr)
|
||||
|
||||
p.EtcdTLSCert = ParamItem{
|
||||
Key: "etcd.ssl.tlsCert",
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.EtcdTLSCert.Init(base.mgr)
|
||||
|
||||
p.EtcdTLSKey = ParamItem{
|
||||
Key: "etcd.ssl.tlsKey",
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.EtcdTLSKey.Init(base.mgr)
|
||||
|
||||
p.EtcdTLSCACert = ParamItem{
|
||||
Key: "etcd.ssl.tlsCACert",
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.EtcdTLSCACert.Init(base.mgr)
|
||||
|
||||
p.EtcdTLSMinVersion = ParamItem{
|
||||
Key: "etcd.ssl.tlsMinVersion",
|
||||
DefaultValue: "1.3",
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.EtcdTLSMinVersion.Init(base.mgr)
|
||||
}
|
||||
|
||||
type LocalStorageConfig struct {
|
||||
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,87 +328,101 @@ func (p *MetaDBConfig) initMaxIdleConns() {
|
||||
// /////////////////////////////////////////////////////////////////////////////
|
||||
// --- pulsar ---
|
||||
type PulsarConfig struct {
|
||||
Base *BaseTable
|
||||
|
||||
Address string
|
||||
WebAddress string
|
||||
MaxMessageSize int
|
||||
Address ParamItem
|
||||
Port ParamItem
|
||||
WebAddress ParamItem
|
||||
WebPort ParamItem
|
||||
MaxMessageSize ParamItem
|
||||
|
||||
// support auth
|
||||
AuthPlugin string
|
||||
AuthParams string
|
||||
AuthPlugin ParamItem
|
||||
AuthParams ParamItem
|
||||
|
||||
// support tenant
|
||||
Tenant string
|
||||
Namespace string
|
||||
Tenant ParamItem
|
||||
Namespace ParamItem
|
||||
}
|
||||
|
||||
func (p *PulsarConfig) init(base *BaseTable) {
|
||||
p.Base = base
|
||||
|
||||
p.initAddress()
|
||||
p.initWebAddress()
|
||||
p.initMaxMessageSize()
|
||||
p.initAuthPlugin()
|
||||
p.initAuthParams()
|
||||
p.initTenant()
|
||||
p.initNamespace()
|
||||
}
|
||||
|
||||
func (p *PulsarConfig) 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 ""
|
||||
}
|
||||
}
|
||||
|
||||
func (p *PulsarConfig) initWebAddress() {
|
||||
if p.Address == "" {
|
||||
return
|
||||
if strings.Contains(addr, ":") {
|
||||
return addr
|
||||
}
|
||||
port, _ := p.Port.get()
|
||||
return "pulsar://" + addr + ":" + port
|
||||
},
|
||||
}
|
||||
p.Address.Init(base.mgr)
|
||||
|
||||
pulsarURL, err := url.ParseRequestURI(p.Address)
|
||||
p.WebPort = ParamItem{
|
||||
Key: "pulsar.webport",
|
||||
Version: "2.0.0",
|
||||
DefaultValue: "80",
|
||||
}
|
||||
p.WebPort.Init(base.mgr)
|
||||
|
||||
p.WebAddress = ParamItem{
|
||||
Key: "pulsar.webaddress",
|
||||
Version: "2.0.0",
|
||||
DefaultValue: "",
|
||||
Formatter: func(add string) string {
|
||||
pulsarURL, err := url.ParseRequestURI(p.Address.GetValue())
|
||||
if err != nil {
|
||||
p.WebAddress = ""
|
||||
log.Info("failed to parse pulsar config, assume pulsar not used", zap.Error(err))
|
||||
} else {
|
||||
webport := p.Base.LoadWithDefault("pulsar.webport", "80")
|
||||
p.WebAddress = "http://" + pulsarURL.Hostname() + ":" + webport
|
||||
return ""
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
return "http://" + pulsarURL.Hostname() + ":" + p.WebPort.GetValue()
|
||||
},
|
||||
}
|
||||
p.WebAddress.Init(base.mgr)
|
||||
|
||||
p.MaxMessageSize = ParamItem{
|
||||
Key: "pulsar.maxMessageSize",
|
||||
Version: "2.0.0",
|
||||
DefaultValue: strconv.Itoa(SuggestPulsarMaxMessageSize),
|
||||
}
|
||||
}
|
||||
p.MaxMessageSize.Init(base.mgr)
|
||||
|
||||
func (p *PulsarConfig) initAuthPlugin() {
|
||||
p.AuthPlugin = p.Base.LoadWithDefault("pulsar.authPlugin", "")
|
||||
}
|
||||
p.Tenant = ParamItem{
|
||||
Key: "pulsar.tenant",
|
||||
Version: "2.2.0",
|
||||
DefaultValue: "public",
|
||||
}
|
||||
p.Tenant.Init(base.mgr)
|
||||
|
||||
func (p *PulsarConfig) initAuthParams() {
|
||||
paramString := p.Base.LoadWithDefault("pulsar.authParams", "")
|
||||
p.Namespace = ParamItem{
|
||||
Key: "pulsar.namespace",
|
||||
Version: "2.2.0",
|
||||
DefaultValue: "default",
|
||||
}
|
||||
p.Namespace.Init(base.mgr)
|
||||
|
||||
// need to parse params to json due to .yaml config file doesn't support json format config item
|
||||
// official pulsar client JWT config : {"token","fake_token_string"}
|
||||
// milvus config: token:fake_token_string
|
||||
p.AuthPlugin = ParamItem{
|
||||
Key: "pulsar.authPlugin",
|
||||
Version: "2.2.0",
|
||||
}
|
||||
p.AuthPlugin.Init(base.mgr)
|
||||
|
||||
p.AuthParams = ParamItem{
|
||||
Key: "pulsar.authParams",
|
||||
Version: "2.2.0",
|
||||
Formatter: func(authParams string) string {
|
||||
jsonMap := make(map[string]string)
|
||||
params := strings.Split(paramString, ",")
|
||||
params := strings.Split(authParams, ",")
|
||||
for _, param := range params {
|
||||
kv := strings.Split(param, ":")
|
||||
if len(kv) == 2 {
|
||||
@ -395,197 +431,184 @@ func (p *PulsarConfig) initAuthParams() {
|
||||
}
|
||||
|
||||
if len(jsonMap) == 0 {
|
||||
p.AuthParams = ""
|
||||
} else {
|
||||
jsonData, _ := json.Marshal(&jsonMap)
|
||||
p.AuthParams = string(jsonData)
|
||||
return ""
|
||||
}
|
||||
}
|
||||
jsonData, _ := json.Marshal(&jsonMap)
|
||||
return string(jsonData)
|
||||
},
|
||||
}
|
||||
p.AuthParams.Init(base.mgr)
|
||||
|
||||
func (p *PulsarConfig) initTenant() {
|
||||
p.Tenant = p.Base.LoadWithDefault("pulsar.tenant", "public")
|
||||
}
|
||||
|
||||
func (p *PulsarConfig) initNamespace() {
|
||||
p.Namespace = p.Base.LoadWithDefault("pulsar.namespace", "default")
|
||||
}
|
||||
|
||||
// --- kafka ---
|
||||
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 ""
|
||||
}
|
||||
}
|
||||
|
||||
func (p *MinioConfig) initAccessKeyID() {
|
||||
keyID, err := p.Base.Load("minio.accessKeyID")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
if strings.Contains(addr, ":") {
|
||||
return addr
|
||||
}
|
||||
p.AccessKeyID = keyID
|
||||
}
|
||||
|
||||
func (p *MinioConfig) initSecretAccessKey() {
|
||||
key, err := p.Base.Load("minio.secretAccessKey")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
port, _ := p.Port.get()
|
||||
return addr + ":" + port
|
||||
},
|
||||
}
|
||||
p.SecretAccessKey = key
|
||||
}
|
||||
p.Address.Init(base.mgr)
|
||||
|
||||
func (p *MinioConfig) initUseSSL() {
|
||||
usessl, err := p.Base.Load("minio.useSSL")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
p.AccessKeyID = ParamItem{
|
||||
Key: "minio.accessKeyID",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
p.UseSSL, _ = strconv.ParseBool(usessl)
|
||||
}
|
||||
p.AccessKeyID.Init(base.mgr)
|
||||
|
||||
func (p *MinioConfig) initBucketName() {
|
||||
bucketName, err := p.Base.Load("minio.bucketName")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
p.SecretAccessKey = ParamItem{
|
||||
Key: "minio.secretAccessKey",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
p.BucketName = bucketName
|
||||
}
|
||||
p.SecretAccessKey.Init(base.mgr)
|
||||
|
||||
func (p *MinioConfig) initRootPath() {
|
||||
rootPath, err := p.Base.Load("minio.rootPath")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
p.UseSSL = ParamItem{
|
||||
Key: "minio.useSSL",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
p.RootPath = rootPath
|
||||
}
|
||||
p.UseSSL.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.BucketName = ParamItem{
|
||||
Key: "minio.bucketName",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
}
|
||||
p.BucketName.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.RootPath = ParamItem{
|
||||
Key: "minio.rootPath",
|
||||
Version: "2.0.0",
|
||||
PanicIfEmpty: true,
|
||||
}
|
||||
}
|
||||
p.RootPath.Init(base.mgr)
|
||||
|
||||
func (p *MinioConfig) initIAMEndpoint() {
|
||||
p.IAMEndpoint = p.Base.LoadWithDefault("minio.iamEndpoint", DefaultMinioIAMEndpoint)
|
||||
p.UseIAM = ParamItem{
|
||||
Key: "minio.useIAM",
|
||||
DefaultValue: DefaultMinioUseIAM,
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.UseIAM.Init(base.mgr)
|
||||
|
||||
p.CloudProvider = ParamItem{
|
||||
Key: "minio.cloudProvider",
|
||||
DefaultValue: DefaultMinioCloudProvider,
|
||||
Version: "2.2.0",
|
||||
}
|
||||
p.CloudProvider.Init(base.mgr)
|
||||
|
||||
p.IAMEndpoint = ParamItem{
|
||||
Key: "minio.iamEndpoint",
|
||||
DefaultValue: DefaultMinioIAMEndpoint,
|
||||
Version: "2.0.0",
|
||||
}
|
||||
p.IAMEndpoint.Init(base.mgr)
|
||||
}
|
||||
|
||||
@ -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())
|
||||
})
|
||||
}
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user