Refactor all params into ParamItem (#20987)

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

Signed-off-by: Enwei Jiao <enwei.jiao@zilliz.com>
This commit is contained in:
Enwei Jiao 2022-12-07 18:01:19 +08:00 committed by GitHub
parent 80a2a49681
commit 89b810a4db
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
171 changed files with 3174 additions and 2957 deletions

View File

@ -24,6 +24,7 @@ import (
"strings" "strings"
"sync" "sync"
"syscall" "syscall"
"time"
"github.com/milvus-io/milvus/internal/management" "github.com/milvus-io/milvus/internal/management"
rocksmqimpl "github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server" rocksmqimpl "github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server"
@ -31,13 +32,9 @@ import (
"go.uber.org/zap" "go.uber.org/zap"
"github.com/milvus-io/milvus/cmd/components" "github.com/milvus-io/milvus/cmd/components"
"github.com/milvus-io/milvus/internal/datanode"
"github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/management/healthz" "github.com/milvus-io/milvus/internal/management/healthz"
"github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proxy"
"github.com/milvus-io/milvus/internal/querynode"
"github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/logutil" "github.com/milvus-io/milvus/internal/util/logutil"
@ -70,7 +67,6 @@ type component interface {
func runComponent[T component](ctx context.Context, func runComponent[T component](ctx context.Context,
localMsg bool, localMsg bool,
extraInit func(),
creator func(context.Context, dependency.Factory) (T, error), creator func(context.Context, dependency.Factory) (T, error),
metricRegister func(*prometheus.Registry)) T { metricRegister func(*prometheus.Registry)) T {
var role T var role T
@ -78,9 +74,6 @@ func runComponent[T component](ctx context.Context,
wg.Add(1) wg.Add(1)
go func() { go func() {
if extraInit != nil {
extraInit()
}
factory := dependency.NewFactory(localMsg) factory := dependency.NewFactory(localMsg)
var err error var err error
role, err = creator(ctx, factory) role, err = creator(ctx, factory)
@ -130,55 +123,35 @@ func (mr *MilvusRoles) printLDPreLoad() {
} }
func (mr *MilvusRoles) runRootCoord(ctx context.Context, localMsg bool) *components.RootCoord { func (mr *MilvusRoles) runRootCoord(ctx context.Context, localMsg bool) *components.RootCoord {
return runComponent(ctx, localMsg, nil, components.NewRootCoord, metrics.RegisterRootCoord) return runComponent(ctx, localMsg, components.NewRootCoord, metrics.RegisterRootCoord)
} }
func (mr *MilvusRoles) runProxy(ctx context.Context, localMsg bool, alias string) *components.Proxy { func (mr *MilvusRoles) runProxy(ctx context.Context, localMsg bool) *components.Proxy {
return runComponent(ctx, localMsg, return runComponent(ctx, localMsg, components.NewProxy, metrics.RegisterProxy)
func() {
proxy.Params.ProxyCfg.InitAlias(alias)
},
components.NewProxy,
metrics.RegisterProxy)
} }
func (mr *MilvusRoles) runQueryCoord(ctx context.Context, localMsg bool) *components.QueryCoord { func (mr *MilvusRoles) runQueryCoord(ctx context.Context, localMsg bool) *components.QueryCoord {
return runComponent(ctx, localMsg, nil, components.NewQueryCoord, metrics.RegisterQueryCoord) return runComponent(ctx, localMsg, components.NewQueryCoord, metrics.RegisterQueryCoord)
} }
func (mr *MilvusRoles) runQueryNode(ctx context.Context, localMsg bool, alias string) *components.QueryNode { func (mr *MilvusRoles) runQueryNode(ctx context.Context, localMsg bool) *components.QueryNode {
return runComponent(ctx, localMsg, return runComponent(ctx, localMsg, components.NewQueryNode, metrics.RegisterQueryNode)
func() {
querynode.Params.QueryNodeCfg.InitAlias(alias)
},
components.NewQueryNode,
metrics.RegisterQueryNode)
} }
func (mr *MilvusRoles) runDataCoord(ctx context.Context, localMsg bool) *components.DataCoord { func (mr *MilvusRoles) runDataCoord(ctx context.Context, localMsg bool) *components.DataCoord {
return runComponent(ctx, localMsg, nil, components.NewDataCoord, metrics.RegisterDataCoord) return runComponent(ctx, localMsg, components.NewDataCoord, metrics.RegisterDataCoord)
} }
func (mr *MilvusRoles) runDataNode(ctx context.Context, localMsg bool, alias string) *components.DataNode { func (mr *MilvusRoles) runDataNode(ctx context.Context, localMsg bool) *components.DataNode {
return runComponent(ctx, localMsg, return runComponent(ctx, localMsg, components.NewDataNode, metrics.RegisterDataNode)
func() {
datanode.Params.DataNodeCfg.InitAlias(alias)
},
components.NewDataNode,
metrics.RegisterDataNode)
} }
func (mr *MilvusRoles) runIndexCoord(ctx context.Context, localMsg bool) *components.IndexCoord { func (mr *MilvusRoles) runIndexCoord(ctx context.Context, localMsg bool) *components.IndexCoord {
return runComponent(ctx, localMsg, nil, components.NewIndexCoord, metrics.RegisterIndexCoord) return runComponent(ctx, localMsg, components.NewIndexCoord, metrics.RegisterIndexCoord)
} }
func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool, alias string) *components.IndexNode { func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool) *components.IndexNode {
return runComponent(ctx, localMsg, return runComponent(ctx, localMsg, components.NewIndexNode, metrics.RegisterIndexNode)
func() {
indexnode.Params.IndexNodeCfg.InitAlias(alias)
},
components.NewIndexNode,
metrics.RegisterIndexNode)
} }
func (mr *MilvusRoles) setupLogger() { func (mr *MilvusRoles) setupLogger() {
@ -260,7 +233,7 @@ func (mr *MilvusRoles) Run(local bool, alias string) {
var pn *components.Proxy var pn *components.Proxy
if mr.EnableProxy { if mr.EnableProxy {
pctx := log.WithModule(ctx, "Proxy") pctx := log.WithModule(ctx, "Proxy")
pn = mr.runProxy(pctx, local, alias) pn = mr.runProxy(pctx, local)
if pn != nil { if pn != nil {
defer pn.Stop() defer pn.Stop()
} }
@ -276,7 +249,7 @@ func (mr *MilvusRoles) Run(local bool, alias string) {
var qn *components.QueryNode var qn *components.QueryNode
if mr.EnableQueryNode { if mr.EnableQueryNode {
qn = mr.runQueryNode(ctx, local, alias) qn = mr.runQueryNode(ctx, local)
if qn != nil { if qn != nil {
defer qn.Stop() defer qn.Stop()
} }
@ -292,7 +265,7 @@ func (mr *MilvusRoles) Run(local bool, alias string) {
var dn *components.DataNode var dn *components.DataNode
if mr.EnableDataNode { if mr.EnableDataNode {
dn = mr.runDataNode(ctx, local, alias) dn = mr.runDataNode(ctx, local)
if dn != nil { if dn != nil {
defer dn.Stop() defer dn.Stop()
} }
@ -308,7 +281,7 @@ func (mr *MilvusRoles) Run(local bool, alias string) {
var in *components.IndexNode var in *components.IndexNode
if mr.EnableIndexNode { if mr.EnableIndexNode {
in = mr.runIndexNode(ctx, local, alias) in = mr.runIndexNode(ctx, local)
if in != nil { if in != nil {
defer in.Stop() defer in.Stop()
} }
@ -318,6 +291,10 @@ func (mr *MilvusRoles) Run(local bool, alias string) {
metrics.Register(Registry) metrics.Register(Registry)
management.ServeHTTP() management.ServeHTTP()
paramtable.SetCreateTime(time.Now())
paramtable.SetUpdateTime(time.Now())
sc := make(chan os.Signal, 1) sc := make(chan os.Signal, 1)
signal.Notify(sc, signal.Notify(sc,
syscall.SIGHUP, syscall.SIGHUP,

View File

@ -23,8 +23,8 @@ type Backend interface {
} }
func NewBackend(cfg *configs.MilvusConfig, version string) (Backend, error) { func NewBackend(cfg *configs.MilvusConfig, version string) (Backend, error) {
if cfg.MetaStoreCfg.MetaStoreType != util.MetaStoreTypeEtcd { if cfg.MetaStoreCfg.MetaStoreType.GetValue() != util.MetaStoreTypeEtcd {
return nil, fmt.Errorf("%s is not supported now", cfg.MetaStoreCfg.MetaStoreType) return nil, fmt.Errorf("%s is not supported now", cfg.MetaStoreCfg.MetaStoreType.GetValue())
} }
v, err := semver.Parse(version) v, err := semver.Parse(version)
if err != nil { if err != nil {

View File

@ -79,13 +79,11 @@ func (c *MilvusConfig) init(base *paramtable.BaseTable) {
c.EtcdCfg = &paramtable.EtcdConfig{} c.EtcdCfg = &paramtable.EtcdConfig{}
c.MysqlCfg = &paramtable.MetaDBConfig{} c.MysqlCfg = &paramtable.MetaDBConfig{}
c.MetaStoreCfg.Base = base c.MetaStoreCfg.Init(base)
c.MetaStoreCfg.LoadCfgToMemory()
switch c.MetaStoreCfg.MetaStoreType { switch c.MetaStoreCfg.MetaStoreType.GetValue() {
case util.MetaStoreTypeMysql: case util.MetaStoreTypeMysql:
c.MysqlCfg.Base = base c.MysqlCfg.Init(base)
c.MysqlCfg.LoadCfgToMemory()
default: default:
} }
@ -96,11 +94,11 @@ func (c *MilvusConfig) String() string {
if c == nil { if c == nil {
return "" return ""
} }
switch c.MetaStoreCfg.MetaStoreType { switch c.MetaStoreCfg.MetaStoreType.GetValue() {
case util.MetaStoreTypeEtcd: case util.MetaStoreTypeEtcd:
return fmt.Sprintf("Type: %s, EndPoints: %v, MetaRootPath: %s", c.MetaStoreCfg.MetaStoreType, c.EtcdCfg.Endpoints, c.EtcdCfg.MetaRootPath) return fmt.Sprintf("Type: %s, EndPoints: %v, MetaRootPath: %s", c.MetaStoreCfg.MetaStoreType.GetValue(), c.EtcdCfg.Endpoints.GetValue(), c.EtcdCfg.MetaRootPath.GetValue())
default: default:
return fmt.Sprintf("unsupported meta store: %s", c.MetaStoreCfg.MetaStoreType) return fmt.Sprintf("unsupported meta store: %s", c.MetaStoreCfg.MetaStoreType.GetValue())
} }
} }

View File

@ -162,7 +162,7 @@ func combineToCollectionIndexesMeta220(fieldIndexes FieldIndexes210, collectionI
newIndexParamsMap := make(map[string]string) newIndexParamsMap := make(map[string]string)
for _, kv := range indexInfo.IndexParams { for _, kv := range indexInfo.IndexParams {
if kv.Key == common.IndexParamsKey { if kv.Key == common.IndexParamsKey {
params, err := funcutil.ParseIndexParamsMap(kv.Value) params, err := funcutil.JSONToMap(kv.Value)
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -77,7 +77,7 @@ func (es *EtcdSource) GetConfigurations() (map[string]string, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
es.configRefresher.start() es.configRefresher.start(es.GetSourceName())
es.RLock() es.RLock()
for key, value := range es.currentConfig { for key, value := range es.currentConfig {
configMap[key] = value configMap[key] = value
@ -98,6 +98,7 @@ func (es *EtcdSource) GetSourceName() string {
} }
func (es *EtcdSource) Close() { func (es *EtcdSource) Close() {
es.etcdCli.Close()
es.configRefresher.stop() es.configRefresher.stop()
} }

View File

@ -65,7 +65,7 @@ func (fs *FileSource) GetConfigurations() (map[string]string, error) {
return nil, err return nil, err
} }
fs.configRefresher.start() fs.configRefresher.start(fs.GetSourceName())
fs.RLock() fs.RLock()
for k, v := range fs.configs { for k, v := range fs.configs {

View File

@ -153,6 +153,12 @@ func (m *Manager) DeleteConfig(key string) {
m.overlayConfigs[formatKey(key)] = TombValue m.overlayConfigs[formatKey(key)] = TombValue
} }
func (m *Manager) ResetConfig(key string) {
m.Lock()
defer m.Unlock()
delete(m.overlayConfigs, formatKey(key))
}
// Do not use it directly, only used when add source and unittests. // Do not use it directly, only used when add source and unittests.
func (m *Manager) pullSourceConfigs(source string) error { func (m *Manager) pullSourceConfigs(source string) error {
configSource, ok := m.sources[source] configSource, ok := m.sources[source]

View File

@ -40,10 +40,10 @@ func newRefresher(interval time.Duration, fetchFunc func() error) refresher {
} }
} }
func (r refresher) start() { func (r refresher) start(name string) {
if r.refreshInterval > 0 { if r.refreshInterval > 0 {
r.intervalInitOnce.Do(func() { r.intervalInitOnce.Do(func() {
go r.refreshPeriodically() go r.refreshPeriodically(name)
}) })
} }
} }
@ -52,9 +52,9 @@ func (r refresher) stop() {
r.intervalDone <- true r.intervalDone <- true
} }
func (r refresher) refreshPeriodically() { func (r refresher) refreshPeriodically(name string) {
ticker := time.NewTicker(r.refreshInterval) ticker := time.NewTicker(r.refreshInterval)
log.Info("start refreshing configurations") log.Info("start refreshing configurations", zap.String("source", name))
for { for {
select { select {
case <-ticker.C: case <-ticker.C:

View File

@ -55,7 +55,7 @@ func (c *channelStateTimer) getWatchers(prefix string) (clientv3.WatchChan, chan
} }
func (c *channelStateTimer) loadAllChannels(nodeID UniqueID) ([]*datapb.ChannelWatchInfo, error) { func (c *channelStateTimer) loadAllChannels(nodeID UniqueID) ([]*datapb.ChannelWatchInfo, error) {
prefix := path.Join(Params.DataCoordCfg.ChannelWatchSubPath, strconv.FormatInt(nodeID, 10)) prefix := path.Join(Params.CommonCfg.DataCoordWatchSubPath.GetValue(), strconv.FormatInt(nodeID, 10))
// TODO: change to LoadWithPrefixBytes // TODO: change to LoadWithPrefixBytes
keys, values, err := c.watchkv.LoadWithPrefix(prefix) keys, values, err := c.watchkv.LoadWithPrefix(prefix)

View File

@ -32,7 +32,7 @@ func TestChannelStateTimer(t *testing.T) {
kv := getMetaKv(t) kv := getMetaKv(t)
defer kv.Close() defer kv.Close()
prefix := Params.DataCoordCfg.ChannelWatchSubPath prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
t.Run("test getWatcher", func(t *testing.T) { t.Run("test getWatcher", func(t *testing.T) {
timer := newChannelStateTimer(kv) timer := newChannelStateTimer(kv)
@ -60,7 +60,7 @@ func TestChannelStateTimer(t *testing.T) {
validData, err := proto.Marshal(&validWatchInfo) validData, err := proto.Marshal(&validWatchInfo)
require.NoError(t, err) require.NoError(t, err)
prefix = Params.DataCoordCfg.ChannelWatchSubPath prefix = Params.CommonCfg.DataCoordWatchSubPath.GetValue()
prepareKvs := map[string]string{ prepareKvs := map[string]string{
path.Join(prefix, "1/channel-1"): "invalidWatchInfo", path.Join(prefix, "1/channel-1"): "invalidWatchInfo",
path.Join(prefix, "1/channel-2"): string(validData), path.Join(prefix, "1/channel-2"): string(validData),

View File

@ -414,7 +414,7 @@ func (c *ChannelManager) unsubAttempt(ncInfo *NodeChannelInfo) {
nodeID := ncInfo.NodeID nodeID := ncInfo.NodeID
for _, ch := range ncInfo.Channels { for _, ch := range ncInfo.Channels {
// align to datanode subname, using vchannel name // align to datanode subname, using vchannel name
subName := fmt.Sprintf("%s-%d-%s", Params.CommonCfg.DataNodeSubName, nodeID, ch.Name) subName := fmt.Sprintf("%s-%d-%s", Params.CommonCfg.DataNodeSubName.GetValue(), nodeID, ch.Name)
pchannelName := funcutil.ToPhysicalChannel(ch.Name) pchannelName := funcutil.ToPhysicalChannel(ch.Name)
msgstream.UnsubscribeChannels(c.ctx, c.msgstreamFactory, subName, []string{pchannelName}) msgstream.UnsubscribeChannels(c.ctx, c.msgstreamFactory, subName, []string{pchannelName})
} }
@ -449,7 +449,7 @@ func (c *ChannelManager) fillChannelWatchInfo(op *ChannelOp) {
Vchan: vcInfo, Vchan: vcInfo,
StartTs: time.Now().Unix(), StartTs: time.Now().Unix(),
State: datapb.ChannelWatchState_Uncomplete, State: datapb.ChannelWatchState_Uncomplete,
TimeoutTs: time.Now().Add(Params.DataCoordCfg.MaxWatchDuration).UnixNano(), TimeoutTs: time.Now().Add(Params.DataCoordCfg.MaxWatchDuration.GetAsDuration(time.Second)).UnixNano(),
Schema: ch.Schema, Schema: ch.Schema,
} }
op.ChannelWatchInfos = append(op.ChannelWatchInfos, info) op.ChannelWatchInfos = append(op.ChannelWatchInfos, info)
@ -460,7 +460,7 @@ func (c *ChannelManager) fillChannelWatchInfo(op *ChannelOp) {
func (c *ChannelManager) fillChannelWatchInfoWithState(op *ChannelOp, state datapb.ChannelWatchState) []string { func (c *ChannelManager) fillChannelWatchInfoWithState(op *ChannelOp, state datapb.ChannelWatchState) []string {
var channelsWithTimer = []string{} var channelsWithTimer = []string{}
startTs := time.Now().Unix() startTs := time.Now().Unix()
timeoutTs := time.Now().Add(Params.DataCoordCfg.MaxWatchDuration).UnixNano() timeoutTs := time.Now().Add(Params.DataCoordCfg.MaxWatchDuration.GetAsDuration(time.Second)).UnixNano()
for _, ch := range op.Channels { for _, ch := range op.Channels {
vcInfo := c.h.GetDataVChanPositions(ch, allPartitionID) vcInfo := c.h.GetDataVChanPositions(ch, allPartitionID)
info := &datapb.ChannelWatchInfo{ info := &datapb.ChannelWatchInfo{
@ -653,7 +653,7 @@ func (c *ChannelManager) watchChannelStatesLoop(ctx context.Context) {
defer logutil.LogPanic() defer logutil.LogPanic()
// REF MEP#7 watchInfo paths are orgnized as: [prefix]/channel/{node_id}/{channel_name} // REF MEP#7 watchInfo paths are orgnized as: [prefix]/channel/{node_id}/{channel_name}
watchPrefix := Params.DataCoordCfg.ChannelWatchSubPath watchPrefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
// TODO, this is risky, we'd better watch etcd with revision rather simply a path // TODO, this is risky, we'd better watch etcd with revision rather simply a path
etcdWatcher, timeoutWatcher := c.stateTimer.getWatchers(watchPrefix) etcdWatcher, timeoutWatcher := c.stateTimer.getWatchers(watchPrefix)
@ -788,7 +788,7 @@ func (c *ChannelManager) CleanupAndReassign(nodeID UniqueID, channelName string)
if c.msgstreamFactory == nil { if c.msgstreamFactory == nil {
log.Warn("msgstream factory is not set, unable to clean up topics") log.Warn("msgstream factory is not set, unable to clean up topics")
} else { } else {
subName := fmt.Sprintf("%s-%d-%d", Params.CommonCfg.DataNodeSubName, nodeID, chToCleanUp.CollectionID) subName := fmt.Sprintf("%s-%d-%d", Params.CommonCfg.DataNodeSubName.GetValue(), nodeID, chToCleanUp.CollectionID)
pchannelName := funcutil.ToPhysicalChannel(channelName) pchannelName := funcutil.ToPhysicalChannel(channelName)
msgstream.UnsubscribeChannels(c.ctx, c.msgstreamFactory, subName, []string{pchannelName}) msgstream.UnsubscribeChannels(c.ctx, c.msgstreamFactory, subName, []string{pchannelName})
} }

View File

@ -57,7 +57,7 @@ func waitAndStore(t *testing.T, metakv kv.MetaKv, key string, waitState, storeSt
// waitAndCheckState checks if the DataCoord writes expected state into Etcd // waitAndCheckState checks if the DataCoord writes expected state into Etcd
func waitAndCheckState(t *testing.T, kv kv.MetaKv, expectedState datapb.ChannelWatchState, nodeID UniqueID, channelName string, collectionID UniqueID) { func waitAndCheckState(t *testing.T, kv kv.MetaKv, expectedState datapb.ChannelWatchState, nodeID UniqueID, channelName string, collectionID UniqueID) {
for { for {
prefix := Params.DataCoordCfg.ChannelWatchSubPath prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
v, err := kv.Load(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName)) v, err := kv.Load(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName))
if err == nil && len(v) > 0 { if err == nil && len(v) > 0 {
watchInfo, err := parseWatchInfo("fake", []byte(v)) watchInfo, err := parseWatchInfo("fake", []byte(v))
@ -93,7 +93,7 @@ func TestChannelManager_StateTransfer(t *testing.T) {
p := "/tmp/milvus_ut/rdb_data" p := "/tmp/milvus_ut/rdb_data"
t.Setenv("ROCKSMQ_PATH", p) t.Setenv("ROCKSMQ_PATH", p)
prefix := Params.DataCoordCfg.ChannelWatchSubPath prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
var ( var (
collectionID = UniqueID(9) collectionID = UniqueID(9)
@ -376,7 +376,7 @@ func TestChannelManager(t *testing.T) {
metakv.Close() metakv.Close()
}() }()
prefix := Params.DataCoordCfg.ChannelWatchSubPath prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
t.Run("test AddNode with avalible node", func(t *testing.T) { t.Run("test AddNode with avalible node", func(t *testing.T) {
// Note: this test is based on the default registerPolicy // Note: this test is based on the default registerPolicy
defer metakv.RemoveWithPrefix("") defer metakv.RemoveWithPrefix("")
@ -574,7 +574,7 @@ func TestChannelManager(t *testing.T) {
bufferID: {bufferID, []*channel{}}, bufferID: {bufferID, []*channel{}},
}, },
} }
chManager.stateTimer.startOne(datapb.ChannelWatchState_ToRelease, "channel-1", 1, time.Now().Add(Params.DataCoordCfg.MaxWatchDuration).UnixNano()) chManager.stateTimer.startOne(datapb.ChannelWatchState_ToRelease, "channel-1", 1, time.Now().Add(Params.DataCoordCfg.MaxWatchDuration.GetAsDuration(time.Second)).UnixNano())
err = chManager.DeleteNode(1) err = chManager.DeleteNode(1)
assert.NoError(t, err) assert.NoError(t, err)
@ -736,7 +736,7 @@ func TestChannelManager_Reload(t *testing.T) {
collectionID = UniqueID(2) collectionID = UniqueID(2)
channelName = "channel-checkOldNodes" channelName = "channel-checkOldNodes"
) )
prefix := Params.DataCoordCfg.ChannelWatchSubPath prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
getWatchInfoWithState := func(state datapb.ChannelWatchState, collectionID UniqueID, channelName string) *datapb.ChannelWatchInfo { getWatchInfoWithState := func(state datapb.ChannelWatchState, collectionID UniqueID, channelName string) *datapb.ChannelWatchInfo {
return &datapb.ChannelWatchInfo{ return &datapb.ChannelWatchInfo{
@ -904,7 +904,7 @@ func TestChannelManager_BalanceBehaviour(t *testing.T) {
metakv.Close() metakv.Close()
}() }()
prefix := Params.DataCoordCfg.ChannelWatchSubPath prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
t.Run("one node with three channels add a new node", func(t *testing.T) { t.Run("one node with three channels add a new node", func(t *testing.T) {
defer metakv.RemoveWithPrefix("") defer metakv.RemoveWithPrefix("")

View File

@ -134,7 +134,7 @@ func NewChannelStore(kv kv.TxnKV) *ChannelStore {
// Reload restores the buffer channels and node-channels mapping from kv. // Reload restores the buffer channels and node-channels mapping from kv.
func (c *ChannelStore) Reload() error { func (c *ChannelStore) Reload() error {
record := timerecord.NewTimeRecorder("datacoord") record := timerecord.NewTimeRecorder("datacoord")
keys, values, err := c.store.LoadWithPrefix(Params.DataCoordCfg.ChannelWatchSubPath) keys, values, err := c.store.LoadWithPrefix(Params.CommonCfg.DataCoordWatchSubPath.GetValue())
if err != nil { if err != nil {
return err return err
} }
@ -368,12 +368,12 @@ func (c *ChannelStore) txn(opSet ChannelOpSet) error {
// buildNodeChannelKey generates a key for kv store, where the key is a concatenation of ChannelWatchSubPath, nodeID and channel name. // buildNodeChannelKey generates a key for kv store, where the key is a concatenation of ChannelWatchSubPath, nodeID and channel name.
func buildNodeChannelKey(nodeID int64, chName string) string { func buildNodeChannelKey(nodeID int64, chName string) string {
return fmt.Sprintf("%s%s%d%s%s", Params.DataCoordCfg.ChannelWatchSubPath, delimiter, nodeID, delimiter, chName) return fmt.Sprintf("%s%s%d%s%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), delimiter, nodeID, delimiter, chName)
} }
// buildKeyPrefix generates a key *prefix* for kv store, where the key prefix is a concatenation of ChannelWatchSubPath and nodeID. // buildKeyPrefix generates a key *prefix* for kv store, where the key prefix is a concatenation of ChannelWatchSubPath and nodeID.
func buildKeyPrefix(nodeID int64) string { func buildKeyPrefix(nodeID int64) string {
return fmt.Sprintf("%s%s%d", Params.DataCoordCfg.ChannelWatchSubPath, delimiter, nodeID) return fmt.Sprintf("%s%s%d", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), delimiter, nodeID)
} }
// parseNodeKey validates a given node key, then extracts and returns the corresponding node id on success. // parseNodeKey validates a given node key, then extracts and returns the corresponding node id on success.

View File

@ -85,7 +85,7 @@ func TestClusterCreate(t *testing.T) {
} }
info1Data, err := proto.Marshal(info1) info1Data, err := proto.Marshal(info1)
assert.Nil(t, err) assert.Nil(t, err)
err = kv.Save(Params.DataCoordCfg.ChannelWatchSubPath+"/1/channel1", string(info1Data)) err = kv.Save(Params.CommonCfg.DataCoordWatchSubPath.GetValue()+"/1/channel1", string(info1Data))
assert.Nil(t, err) assert.Nil(t, err)
sessionManager := NewSessionManager() sessionManager := NewSessionManager()

View File

@ -119,7 +119,7 @@ func newCompactionPlanHandler(sessions *SessionManager, cm *ChannelManager, meta
} }
func (c *compactionPlanHandler) start() { func (c *compactionPlanHandler) start() {
interval := time.Duration(Params.DataCoordCfg.CompactionCheckIntervalInSeconds) * time.Second interval := Params.DataCoordCfg.CompactionCheckIntervalInSeconds.GetAsDuration(time.Second)
ticker := time.NewTicker(interval) ticker := time.NewTicker(interval)
c.quit = make(chan struct{}) c.quit = make(chan struct{})
c.wg.Add(1) c.wg.Add(1)

View File

@ -23,6 +23,7 @@ import (
"time" "time"
"github.com/milvus-io/milvus/internal/util/metautil" "github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus-proto/go-api/commonpb" "github.com/milvus-io/milvus-proto/go-api/commonpb"
@ -124,7 +125,7 @@ func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
parallelCh: make(map[int64]chan struct{}), parallelCh: make(map[int64]chan struct{}),
allocator: newMockAllocator(), allocator: newMockAllocator(),
} }
Params.DataCoordCfg.CompactionCheckIntervalInSeconds = 1 Params.Save(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key, "1")
c.start() c.start()
err := c.execCompactionPlan(tt.args.signal, tt.args.plan) err := c.execCompactionPlan(tt.args.signal, tt.args.plan)
assert.Equal(t, tt.err, err) assert.Equal(t, tt.err, err)
@ -154,7 +155,8 @@ func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
func Test_compactionPlanHandler_execWithParallels(t *testing.T) { func Test_compactionPlanHandler_execWithParallels(t *testing.T) {
mockDataNode := &mocks.DataNode{} mockDataNode := &mocks.DataNode{}
Params.DataCoordCfg.CompactionCheckIntervalInSeconds = 1 paramtable.Get().Save(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key, "1")
defer paramtable.Get().Reset(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key)
c := &compactionPlanHandler{ c := &compactionPlanHandler{
plans: map[int64]*compactionTask{}, plans: map[int64]*compactionTask{},
sessions: &SessionManager{ sessions: &SessionManager{

View File

@ -102,7 +102,7 @@ func newCompactionTrigger(
func (t *compactionTrigger) start() { func (t *compactionTrigger) start() {
t.quit = make(chan struct{}) t.quit = make(chan struct{})
t.globalTrigger = time.NewTicker(Params.DataCoordCfg.GlobalCompactionInterval) t.globalTrigger = time.NewTicker(Params.DataCoordCfg.GlobalCompactionInterval.GetAsDuration(time.Second))
t.wg.Add(2) t.wg.Add(2)
go func() { go func() {
defer logutil.LogPanic() defer logutil.LogPanic()
@ -134,7 +134,7 @@ func (t *compactionTrigger) startGlobalCompactionLoop() {
defer t.wg.Done() defer t.wg.Done()
// If AutoCompaction disabled, global loop will not start // If AutoCompaction disabled, global loop will not start
if !Params.DataCoordCfg.GetEnableAutoCompaction() { if !Params.DataCoordCfg.EnableAutoCompaction.GetAsBool() {
return return
} }
@ -184,7 +184,7 @@ func (t *compactionTrigger) getCompactTime(ts Timestamp, collectionID UniqueID)
} }
pts, _ := tsoutil.ParseTS(ts) pts, _ := tsoutil.ParseTS(ts)
ttRetention := pts.Add(-time.Duration(Params.CommonCfg.RetentionDuration) * time.Second) ttRetention := pts.Add(Params.CommonCfg.RetentionDuration.GetAsDuration(time.Second) * -1)
ttRetentionLogic := tsoutil.ComposeTS(ttRetention.UnixNano()/int64(time.Millisecond), 0) ttRetentionLogic := tsoutil.ComposeTS(ttRetention.UnixNano()/int64(time.Millisecond), 0)
if collectionTTL > 0 { if collectionTTL > 0 {
@ -216,7 +216,7 @@ func (t *compactionTrigger) triggerCompaction() error {
// triggerSingleCompaction triger a compaction bundled with collection-partiiton-channel-segment // triggerSingleCompaction triger a compaction bundled with collection-partiiton-channel-segment
func (t *compactionTrigger) triggerSingleCompaction(collectionID, partitionID, segmentID int64, channel string) error { func (t *compactionTrigger) triggerSingleCompaction(collectionID, partitionID, segmentID int64, channel string) error {
// If AutoCompaction diabled, flush request will not trigger compaction // If AutoCompaction diabled, flush request will not trigger compaction
if !Params.DataCoordCfg.GetEnableAutoCompaction() { if !Params.DataCoordCfg.EnableAutoCompaction.GetAsBool() {
return nil return nil
} }
@ -515,7 +515,7 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, c
if segment.GetNumOfRows() < segment.GetMaxRowNum() { if segment.GetNumOfRows() < segment.GetMaxRowNum() {
var result []*SegmentInfo var result []*SegmentInfo
free := segment.GetMaxRowNum() - segment.GetNumOfRows() free := segment.GetMaxRowNum() - segment.GetNumOfRows()
maxNum := Params.DataCoordCfg.MaxSegmentToMerge - 1 maxNum := Params.DataCoordCfg.MaxSegmentToMerge.GetAsInt() - 1
prioritizedCandidates, result, free = greedySelect(prioritizedCandidates, free, maxNum) prioritizedCandidates, result, free = greedySelect(prioritizedCandidates, free, maxNum)
bucket = append(bucket, result...) bucket = append(bucket, result...)
maxNum -= len(result) maxNum -= len(result)
@ -550,7 +550,7 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, c
// for small segment merge, we pick one largest segment and merge as much as small segment together with it // for small segment merge, we pick one largest segment and merge as much as small segment together with it
// Why reverse? try to merge as many segments as expected. // Why reverse? try to merge as many segments as expected.
// for instance, if a 255M and 255M is the largest small candidates, they will never be merged because of the MinSegmentToMerge limit. // for instance, if a 255M and 255M is the largest small candidates, they will never be merged because of the MinSegmentToMerge limit.
smallCandidates, result, _ = reverseGreedySelect(smallCandidates, free, Params.DataCoordCfg.MaxSegmentToMerge-1) smallCandidates, result, _ = reverseGreedySelect(smallCandidates, free, Params.DataCoordCfg.MaxSegmentToMerge.GetAsInt()-1)
bucket = append(bucket, result...) bucket = append(bucket, result...)
var size int64 var size int64
@ -560,7 +560,7 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, c
targetRow += s.GetNumOfRows() targetRow += s.GetNumOfRows()
} }
// only merge if candidate number is large than MinSegmentToMerge or if target row is large enough // only merge if candidate number is large than MinSegmentToMerge or if target row is large enough
if len(bucket) >= Params.DataCoordCfg.MinSegmentToMerge || targetRow > int64(float64(segment.GetMaxRowNum())*Params.DataCoordCfg.SegmentCompactableProportion) { if len(bucket) >= Params.DataCoordCfg.MinSegmentToMerge.GetAsInt() || targetRow > int64(float64(segment.GetMaxRowNum())*Params.DataCoordCfg.SegmentCompactableProportion.GetAsFloat()) {
plan := segmentsToPlan(bucket, compactTime) plan := segmentsToPlan(bucket, compactTime)
log.Info("generate a plan for small candidates", zap.Any("plan", plan), log.Info("generate a plan for small candidates", zap.Any("plan", plan),
zap.Int64("target segment row", targetRow), zap.Int64("target segment size", size)) zap.Int64("target segment row", targetRow), zap.Int64("target segment size", size))
@ -643,7 +643,7 @@ func (t *compactionTrigger) getCandidateSegments(channel string, partitionID Uni
} }
func (t *compactionTrigger) isSmallSegment(segment *SegmentInfo) bool { func (t *compactionTrigger) isSmallSegment(segment *SegmentInfo) bool {
return segment.GetNumOfRows() < int64(float64(segment.GetMaxRowNum())*Params.DataCoordCfg.SegmentSmallProportion) return segment.GetNumOfRows() < int64(float64(segment.GetMaxRowNum())*Params.DataCoordCfg.SegmentSmallProportion.GetAsFloat())
} }
func (t *compactionTrigger) fillOriginPlan(plan *datapb.CompactionPlan) error { func (t *compactionTrigger) fillOriginPlan(plan *datapb.CompactionPlan) error {
@ -653,7 +653,7 @@ func (t *compactionTrigger) fillOriginPlan(plan *datapb.CompactionPlan) error {
return err return err
} }
plan.PlanID = id plan.PlanID = id
plan.TimeoutInSeconds = Params.DataCoordCfg.CompactionTimeoutInSeconds plan.TimeoutInSeconds = int32(Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt())
return nil return nil
} }
@ -676,7 +676,7 @@ func (t *compactionTrigger) ShouldDoSingleCompaction(segment *SegmentInfo, compa
totalLogNum += len(statsLogs.GetBinlogs()) totalLogNum += len(statsLogs.GetBinlogs())
} }
// avoid segment has too many bin logs and the etcd meta is too large, force trigger compaction // avoid segment has too many bin logs and the etcd meta is too large, force trigger compaction
if totalLogNum > int(Params.DataCoordCfg.SingleCompactionBinlogMaxNum) { if totalLogNum > Params.DataCoordCfg.SingleCompactionBinlogMaxNum.GetAsInt() {
log.Info("total binlog number is too much, trigger compaction", zap.Int64("segment", segment.ID), log.Info("total binlog number is too much, trigger compaction", zap.Int64("segment", segment.ID),
zap.Int("Delta logs", len(segment.GetDeltalogs())), zap.Int("Bin Logs", len(segment.GetBinlogs())), zap.Int("Stat logs", len(segment.GetStatslogs()))) zap.Int("Delta logs", len(segment.GetDeltalogs())), zap.Int("Bin Logs", len(segment.GetBinlogs())), zap.Int("Stat logs", len(segment.GetStatslogs())))
return true return true
@ -695,7 +695,7 @@ func (t *compactionTrigger) ShouldDoSingleCompaction(segment *SegmentInfo, compa
} }
} }
if float32(totalExpiredRows)/float32(segment.GetNumOfRows()) >= Params.DataCoordCfg.SingleCompactionRatioThreshold || totalExpiredSize > Params.DataCoordCfg.SingleCompactionExpiredLogMaxSize { if float64(totalExpiredRows)/float64(segment.GetNumOfRows()) >= Params.DataCoordCfg.SingleCompactionRatioThreshold.GetAsFloat() || totalExpiredSize > Params.DataCoordCfg.SingleCompactionExpiredLogMaxSize.GetAsInt64() {
log.Info("total expired entities is too much, trigger compation", zap.Int64("segment", segment.ID), log.Info("total expired entities is too much, trigger compation", zap.Int64("segment", segment.ID),
zap.Int("expired rows", totalExpiredRows), zap.Int64("expired log size", totalExpiredSize)) zap.Int("expired rows", totalExpiredRows), zap.Int64("expired log size", totalExpiredSize))
return true return true
@ -721,7 +721,7 @@ func (t *compactionTrigger) ShouldDoSingleCompaction(segment *SegmentInfo, compa
} }
// currently delta log size and delete ratio policy is applied // currently delta log size and delete ratio policy is applied
if float32(totalDeletedRows)/float32(segment.GetNumOfRows()) >= Params.DataCoordCfg.SingleCompactionRatioThreshold || totalDeleteLogSize > Params.DataCoordCfg.SingleCompactionDeltaLogMaxSize { if float64(totalDeletedRows)/float64(segment.GetNumOfRows()) >= Params.DataCoordCfg.SingleCompactionRatioThreshold.GetAsFloat() || totalDeleteLogSize > Params.DataCoordCfg.SingleCompactionDeltaLogMaxSize.GetAsInt64() {
log.Info("total delete entities is too much, trigger compation", zap.Int64("segment", segment.ID), log.Info("total delete entities is too much, trigger compation", zap.Int64("segment", segment.ID),
zap.Int("deleted rows", totalDeletedRows), zap.Int64("delete log size", totalDeleteLogSize)) zap.Int("deleted rows", totalDeletedRows), zap.Int64("delete log size", totalDeleteLogSize))
return true return true

View File

@ -24,6 +24,7 @@ import (
"github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus-proto/go-api/commonpb" "github.com/milvus-io/milvus-proto/go-api/commonpb"
@ -84,11 +85,12 @@ func Test_compactionTrigger_force(t *testing.T) {
segRefer *SegmentReferenceManager segRefer *SegmentReferenceManager
} }
Params.Init() paramtable.Init()
Params.CommonCfg.RetentionDuration = 200 paramtable.Get().Save(Params.CommonCfg.RetentionDuration.Key, "200")
defer paramtable.Get().Reset(Params.CommonCfg.RetentionDuration.Key)
pts, _ := tsoutil.ParseTS(0) pts, _ := tsoutil.ParseTS(0)
ttRetention := pts.Add(-time.Duration(Params.CommonCfg.RetentionDuration) * time.Second) ttRetention := pts.Add(-1 * Params.CommonCfg.RetentionDuration.GetAsDuration(time.Second))
timeTravel := tsoutil.ComposeTS(ttRetention.UnixNano()/int64(time.Millisecond), 0) timeTravel := tsoutil.ComposeTS(ttRetention.UnixNano()/int64(time.Millisecond), 0)
vecFieldID := int64(201) vecFieldID := int64(201)
@ -331,7 +333,7 @@ func Test_compactionTrigger_force(t *testing.T) {
}, },
}, },
StartTime: 0, StartTime: 0,
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds, TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(),
Type: datapb.CompactionType_MixCompaction, Type: datapb.CompactionType_MixCompaction,
Timetravel: timeTravel, Timetravel: timeTravel,
Channel: "ch1", Channel: "ch1",
@ -766,7 +768,7 @@ func Test_compactionTrigger_force_maxSegmentLimit(t *testing.T) {
}, },
}, },
StartTime: 3, StartTime: 3,
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds, TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(),
Type: datapb.CompactionType_MixCompaction, Type: datapb.CompactionType_MixCompaction,
Timetravel: 200, Timetravel: 200,
Channel: "ch1", Channel: "ch1",
@ -870,7 +872,7 @@ func Test_compactionTrigger_noplan(t *testing.T) {
Binlogs: []*datapb.FieldBinlog{ Binlogs: []*datapb.FieldBinlog{
{ {
Binlogs: []*datapb.Binlog{ Binlogs: []*datapb.Binlog{
{EntriesNum: 5, LogPath: "log2", LogSize: int64(Params.DataCoordCfg.SegmentMaxSize)*1024*1024 - 1}, {EntriesNum: 5, LogPath: "log2", LogSize: Params.DataCoordCfg.SegmentMaxSize.GetAsInt64()*1024*1024 - 1},
}, },
}, },
}, },

View File

@ -1022,7 +1022,7 @@ func (m *meta) alterMetaStoreAfterCompaction(modSegments []*SegmentInfo, newSegm
m.Lock() m.Lock()
defer m.Unlock() defer m.Unlock()
modInfos := lo.Map[*SegmentInfo, *datapb.SegmentInfo](modSegments, func(item *SegmentInfo, _ int) *datapb.SegmentInfo { modInfos := lo.Map(modSegments, func(item *SegmentInfo, _ int) *datapb.SegmentInfo {
return item.SegmentInfo return item.SegmentInfo
}) })
@ -1052,7 +1052,7 @@ func (m *meta) revertAlterMetaStoreAfterCompaction(oldSegments []*SegmentInfo, r
m.Lock() m.Lock()
defer m.Unlock() defer m.Unlock()
oldSegmentInfos := lo.Map[*SegmentInfo, *datapb.SegmentInfo](oldSegments, func(item *SegmentInfo, _ int) *datapb.SegmentInfo { oldSegmentInfos := lo.Map(oldSegments, func(item *SegmentInfo, _ int) *datapb.SegmentInfo {
return item.SegmentInfo return item.SegmentInfo
}) })

View File

@ -691,7 +691,7 @@ func TestMeta_alterMetaStore(t *testing.T) {
}}, }},
} }
toAlterInfo := lo.Map[*datapb.SegmentInfo, *SegmentInfo](toAlter, func(item *datapb.SegmentInfo, _ int) *SegmentInfo { toAlterInfo := lo.Map(toAlter, func(item *datapb.SegmentInfo, _ int) *SegmentInfo {
return &SegmentInfo{SegmentInfo: item} return &SegmentInfo{SegmentInfo: item}
}) })

View File

@ -43,7 +43,7 @@ func (s *Server) getQuotaMetrics() *metricsinfo.DataCoordQuotaMetrics {
//getComponentConfigurations returns the configurations of dataNode matching req.Pattern //getComponentConfigurations returns the configurations of dataNode matching req.Pattern
func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse { func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse {
prefix := "datacoord." prefix := "datacoord."
matchedConfig := Params.DataCoordCfg.Base.GetByPattern(prefix + req.Pattern) matchedConfig := Params.GetByPattern(prefix + req.Pattern)
configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig)) configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig))
for key, value := range matchedConfig { for key, value := range matchedConfig {
configList = append(configList, configList = append(configList,
@ -129,13 +129,13 @@ func (s *Server) getDataCoordMetrics() metricsinfo.DataCoordInfos {
DiskUsage: hardware.GetDiskUsage(), DiskUsage: hardware.GetDiskUsage(),
}, },
SystemInfo: metricsinfo.DeployMetrics{}, SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.DataCoordCfg.CreatedTime.String(), CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: Params.DataCoordCfg.UpdatedTime.String(), UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.DataCoordRole, Type: typeutil.DataCoordRole,
ID: s.session.ServerID, ID: s.session.ServerID,
}, },
SystemConfigurations: metricsinfo.DataCoordConfiguration{ SystemConfigurations: metricsinfo.DataCoordConfiguration{
SegmentMaxSize: Params.DataCoordCfg.SegmentMaxSize, SegmentMaxSize: Params.DataCoordCfg.SegmentMaxSize.GetAsFloat(),
}, },
QuotaMetrics: s.getQuotaMetrics(), QuotaMetrics: s.getQuotaMetrics(),
} }

View File

@ -461,7 +461,7 @@ func BgCheckWithMaxWatchDuration(kv kv.TxnKV) ChannelBGChecker {
} }
startTime := time.Unix(watchInfo.StartTs, 0) startTime := time.Unix(watchInfo.StartTs, 0)
d := ts.Sub(startTime) d := ts.Sub(startTime)
if d >= Params.DataCoordCfg.MaxWatchDuration { if d >= Params.DataCoordCfg.MaxWatchDuration.GetAsDuration(time.Second) {
cinfo.Channels = append(cinfo.Channels, c) cinfo.Channels = append(cinfo.Channels, c)
} }
} }

View File

@ -402,7 +402,7 @@ func TestBgCheckWithMaxWatchDuration(t *testing.T) {
getKv([]*watch{{1, "chan1", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Uncomplete}}, getKv([]*watch{{1, "chan1", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Uncomplete}},
{1, "chan2", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Complete}}}), {1, "chan2", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Complete}}}),
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}}}, []*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}}},
ts.Add(Params.DataCoordCfg.MaxWatchDuration), ts.Add(Params.DataCoordCfg.MaxWatchDuration.GetAsDuration(time.Second)),
}, },
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}}, []*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
nil, nil,
@ -412,7 +412,7 @@ func TestBgCheckWithMaxWatchDuration(t *testing.T) {
args{ args{
getKv([]*watch{{1, "chan1", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Uncomplete}}}), getKv([]*watch{{1, "chan1", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Uncomplete}}}),
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}}, []*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
ts.Add(Params.DataCoordCfg.MaxWatchDuration).Add(-time.Second), ts.Add(Params.DataCoordCfg.MaxWatchDuration.GetAsDuration(time.Second)).Add(-time.Second),
}, },
[]*NodeChannelInfo{}, []*NodeChannelInfo{},
nil, nil,

View File

@ -41,7 +41,7 @@ func calBySchemaPolicy(schema *schemapb.CollectionSchema) (int, error) {
if sizePerRecord == 0 { if sizePerRecord == 0 {
return -1, errors.New("zero size record schema found") return -1, errors.New("zero size record schema found")
} }
threshold := Params.DataCoordCfg.SegmentMaxSize * 1024 * 1024 threshold := Params.DataCoordCfg.SegmentMaxSize.GetAsFloat() * 1024 * 1024
return int(threshold / float64(sizePerRecord)), nil return int(threshold / float64(sizePerRecord)), nil
} }
@ -57,7 +57,7 @@ func calBySchemaPolicyWithDiskIndex(schema *schemapb.CollectionSchema) (int, err
if sizePerRecord == 0 { if sizePerRecord == 0 {
return -1, errors.New("zero size record schema found") return -1, errors.New("zero size record schema found")
} }
threshold := Params.DataCoordCfg.DiskSegmentMaxSize * 1024 * 1024 threshold := Params.DataCoordCfg.DiskSegmentMaxSize.GetAsFloat() * 1024 * 1024
return int(threshold / float64(sizePerRecord)), nil return int(threshold / float64(sizePerRecord)), nil
} }

View File

@ -78,7 +78,7 @@ func TestUpperLimitCalBySchema(t *testing.T) {
}, },
}, },
}, },
expected: int(Params.DataCoordCfg.SegmentMaxSize * 1024 * 1024 / float64(524)), expected: int(Params.DataCoordCfg.SegmentMaxSize.GetAsFloat() * 1024 * 1024 / float64(524)),
expectErr: false, expectErr: false,
}, },
} }

View File

@ -184,9 +184,9 @@ func defaultAllocatePolicy() AllocatePolicy {
func defaultSegmentSealPolicy() []segmentSealPolicy { func defaultSegmentSealPolicy() []segmentSealPolicy {
return []segmentSealPolicy{ return []segmentSealPolicy{
sealByLifetimePolicy(Params.DataCoordCfg.SegmentMaxLifetime), sealByLifetimePolicy(Params.DataCoordCfg.SegmentMaxLifetime.GetAsDuration(time.Second)),
getSegmentCapacityPolicy(Params.DataCoordCfg.SegmentSealProportion), getSegmentCapacityPolicy(Params.DataCoordCfg.SegmentSealProportion.GetAsFloat()),
sealLongTimeIdlePolicy(Params.DataCoordCfg.SegmentMaxIdleTime, Params.DataCoordCfg.SegmentMinSizeFromIdleToSealed, Params.DataCoordCfg.SegmentMaxSize), sealLongTimeIdlePolicy(Params.DataCoordCfg.SegmentMaxIdleTime.GetAsDuration(time.Second), Params.DataCoordCfg.SegmentMinSizeFromIdleToSealed.GetAsFloat(), Params.DataCoordCfg.SegmentMaxSize.GetAsFloat()),
} }
} }
@ -330,7 +330,7 @@ func (s *SegmentManager) genExpireTs(ctx context.Context) (Timestamp, error) {
return 0, err return 0, err
} }
physicalTs, logicalTs := tsoutil.ParseTS(ts) physicalTs, logicalTs := tsoutil.ParseTS(ts)
expirePhysicalTs := physicalTs.Add(time.Duration(Params.DataCoordCfg.SegAssignmentExpiration) * time.Millisecond) expirePhysicalTs := physicalTs.Add(time.Duration(Params.DataCoordCfg.SegAssignmentExpiration.GetAsFloat()) * time.Millisecond)
expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs)) expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs))
return expireTs, nil return expireTs, nil
} }

View File

@ -194,7 +194,7 @@ func CreateServer(ctx context.Context, factory dependency.Factory, opts ...Optio
rootCoordClientCreator: defaultRootCoordCreatorFunc, rootCoordClientCreator: defaultRootCoordCreatorFunc,
helper: defaultServerHelper(), helper: defaultServerHelper(),
metricsCacheManager: metricsinfo.NewMetricsCacheManager(), metricsCacheManager: metricsinfo.NewMetricsCacheManager(),
enableActiveStandBy: Params.DataCoordCfg.EnableActiveStandby, enableActiveStandBy: Params.DataCoordCfg.EnableActiveStandby.GetAsBool(),
} }
for _, opt := range opts { for _, opt := range opts {
@ -282,7 +282,7 @@ func (s *Server) Init() error {
return err return err
} }
if Params.DataCoordCfg.EnableCompaction { if Params.DataCoordCfg.EnableCompaction.GetAsBool() {
s.createCompactionHandler() s.createCompactionHandler()
s.createCompactionTrigger() s.createCompactionTrigger()
} }
@ -301,7 +301,7 @@ func (s *Server) Init() error {
// datanodes etcd watch, etcd alive check and flush completed status check // datanodes etcd watch, etcd alive check and flush completed status check
// 4. set server state to Healthy // 4. set server state to Healthy
func (s *Server) Start() error { func (s *Server) Start() error {
if Params.DataCoordCfg.EnableCompaction { if Params.DataCoordCfg.EnableCompaction.GetAsBool() {
s.compactionHandler.start() s.compactionHandler.start()
s.compactionTrigger.start() s.compactionTrigger.start()
} }
@ -322,9 +322,6 @@ func (s *Server) Start() error {
logutil.Logger(s.ctx).Info("DataCoord startup successfully") logutil.Logger(s.ctx).Info("DataCoord startup successfully")
} }
Params.DataCoordCfg.CreatedTime = time.Now()
Params.DataCoordCfg.UpdatedTime = time.Now()
// DataCoord (re)starts successfully and starts to collection segment stats // DataCoord (re)starts successfully and starts to collection segment stats
// data from all DataNode. // data from all DataNode.
// This will prevent DataCoord from missing out any important segment stats // This will prevent DataCoord from missing out any important segment stats
@ -392,10 +389,10 @@ func (s *Server) newChunkManagerFactory() (storage.ChunkManager, error) {
func (s *Server) initGarbageCollection(cli storage.ChunkManager) { func (s *Server) initGarbageCollection(cli storage.ChunkManager) {
s.garbageCollector = newGarbageCollector(s.meta, s.handler, s.segReferManager, s.indexCoord, GcOption{ s.garbageCollector = newGarbageCollector(s.meta, s.handler, s.segReferManager, s.indexCoord, GcOption{
cli: cli, cli: cli,
enabled: Params.DataCoordCfg.EnableGarbageCollection, enabled: Params.DataCoordCfg.EnableGarbageCollection.GetAsBool(),
checkInterval: Params.DataCoordCfg.GCInterval, checkInterval: Params.DataCoordCfg.GCInterval.GetAsDuration(time.Second),
missingTolerance: Params.DataCoordCfg.GCMissingTolerance, missingTolerance: Params.DataCoordCfg.GCMissingTolerance.GetAsDuration(time.Second),
dropTolerance: Params.DataCoordCfg.GCDropTolerance, dropTolerance: Params.DataCoordCfg.GCDropTolerance.GetAsDuration(time.Second),
}) })
} }
@ -486,11 +483,11 @@ func (s *Server) startDataNodeTtLoop(ctx context.Context) {
log.Error("DataCoord failed to create timetick channel", zap.Error(err)) log.Error("DataCoord failed to create timetick channel", zap.Error(err))
panic(err) panic(err)
} }
subName := fmt.Sprintf("%s-%d-datanodeTl", Params.CommonCfg.DataCoordSubName, paramtable.GetNodeID()) subName := fmt.Sprintf("%s-%d-datanodeTl", Params.CommonCfg.DataCoordSubName.GetValue(), paramtable.GetNodeID())
ttMsgStream.AsConsumer([]string{Params.CommonCfg.DataCoordTimeTick}, ttMsgStream.AsConsumer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()},
subName, mqwrapper.SubscriptionPositionLatest) subName, mqwrapper.SubscriptionPositionLatest)
log.Info("DataCoord creates the timetick channel consumer", log.Info("DataCoord creates the timetick channel consumer",
zap.String("timeTickChannel", Params.CommonCfg.DataCoordTimeTick), zap.String("timeTickChannel", Params.CommonCfg.DataCoordTimeTick.GetValue()),
zap.String("subscription", subName)) zap.String("subscription", subName))
go s.handleDataNodeTimetickMsgstream(ctx, ttMsgStream) go s.handleDataNodeTimetickMsgstream(ctx, ttMsgStream)
@ -827,7 +824,7 @@ func (s *Server) Stop() error {
s.stopServerLoop() s.stopServerLoop()
s.session.Revoke(time.Second) s.session.Revoke(time.Second)
if Params.DataCoordCfg.EnableCompaction { if Params.DataCoordCfg.EnableCompaction.GetAsBool() {
s.stopCompactionTrigger() s.stopCompactionTrigger()
s.stopCompactionHandler() s.stopCompactionHandler()
} }

View File

@ -75,7 +75,7 @@ func TestGetSegmentInfoChannel(t *testing.T) {
resp, err := svr.GetSegmentInfoChannel(context.TODO()) resp, err := svr.GetSegmentInfoChannel(context.TODO())
assert.Nil(t, err) assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, Params.CommonCfg.DataCoordSegmentInfo, resp.Value) assert.EqualValues(t, Params.CommonCfg.DataCoordSegmentInfo.GetValue(), resp.Value)
}) })
} }
@ -301,7 +301,7 @@ func TestGetTimeTickChannel(t *testing.T) {
resp, err := svr.GetTimeTickChannel(context.TODO()) resp, err := svr.GetTimeTickChannel(context.TODO())
assert.Nil(t, err) assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, Params.CommonCfg.DataCoordTimeTick, resp.Value) assert.EqualValues(t, Params.CommonCfg.DataCoordTimeTick.GetValue(), resp.Value)
} }
func TestGetSegmentStates(t *testing.T) { func TestGetSegmentStates(t *testing.T) {
@ -1486,7 +1486,7 @@ func TestDataNodeTtChannel(t *testing.T) {
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO()) ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.Nil(t, err) assert.Nil(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick}) ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close() defer ttMsgStream.Close()
info := &NodeInfo{ info := &NodeInfo{
Address: "localhost:7777", Address: "localhost:7777",
@ -1553,7 +1553,7 @@ func TestDataNodeTtChannel(t *testing.T) {
}) })
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO()) ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.Nil(t, err) assert.Nil(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick}) ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close() defer ttMsgStream.Close()
info := &NodeInfo{ info := &NodeInfo{
Address: "localhost:7777", Address: "localhost:7777",
@ -1634,7 +1634,7 @@ func TestDataNodeTtChannel(t *testing.T) {
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO()) ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.Nil(t, err) assert.Nil(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick}) ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close() defer ttMsgStream.Close()
node := &NodeInfo{ node := &NodeInfo{
NodeID: 0, NodeID: 0,
@ -2600,7 +2600,8 @@ func TestGetRecoveryInfo(t *testing.T) {
} }
func TestGetCompactionState(t *testing.T) { func TestGetCompactionState(t *testing.T) {
Params.DataCoordCfg.EnableCompaction = true paramtable.Get().Save(Params.DataCoordCfg.EnableCompaction.Key, "true")
defer paramtable.Get().Reset(Params.DataCoordCfg.EnableCompaction.Key)
t.Run("test get compaction state with new compactionhandler", func(t *testing.T) { t.Run("test get compaction state with new compactionhandler", func(t *testing.T) {
svr := &Server{} svr := &Server{}
svr.stateCode.Store(commonpb.StateCode_Healthy) svr.stateCode.Store(commonpb.StateCode_Healthy)
@ -2665,7 +2666,8 @@ func TestGetCompactionState(t *testing.T) {
} }
func TestManualCompaction(t *testing.T) { func TestManualCompaction(t *testing.T) {
Params.DataCoordCfg.EnableCompaction = true paramtable.Get().Save(Params.DataCoordCfg.EnableCompaction.Key, "true")
defer paramtable.Get().Reset(Params.DataCoordCfg.EnableCompaction.Key)
t.Run("test manual compaction successfully", func(t *testing.T) { t.Run("test manual compaction successfully", func(t *testing.T) {
svr := &Server{allocator: &MockAllocator{}} svr := &Server{allocator: &MockAllocator{}}
svr.stateCode.Store(commonpb.StateCode_Healthy) svr.stateCode.Store(commonpb.StateCode_Healthy)
@ -3457,8 +3459,7 @@ func (ms *MockClosePanicMsgstream) Chan() <-chan *msgstream.MsgPack {
func newTestServer(t *testing.T, receiveCh chan any, opts ...Option) *Server { func newTestServer(t *testing.T, receiveCh chan any, opts ...Option) *Server {
var err error var err error
Params.Init() paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))
Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int())
factory := dependency.NewDefaultFactory(true) factory := dependency.NewDefaultFactory(true)
etcdCli, err := etcd.GetEtcdClient( etcdCli, err := etcd.GetEtcdClient(
@ -3503,8 +3504,7 @@ func newTestServer(t *testing.T, receiveCh chan any, opts ...Option) *Server {
func newTestServerWithMeta(t *testing.T, receiveCh chan any, meta *meta, opts ...Option) *Server { func newTestServerWithMeta(t *testing.T, receiveCh chan any, meta *meta, opts ...Option) *Server {
var err error var err error
Params.Init() paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))
Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int())
factory := dependency.NewDefaultFactory(true) factory := dependency.NewDefaultFactory(true)
etcdCli, err := etcd.GetEtcdClient( etcdCli, err := etcd.GetEtcdClient(
@ -3558,8 +3558,8 @@ func closeTestServer(t *testing.T, svr *Server) {
func newTestServer2(t *testing.T, receiveCh chan any, opts ...Option) *Server { func newTestServer2(t *testing.T, receiveCh chan any, opts ...Option) *Server {
var err error var err error
Params.Init() paramtable.Init()
Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int()) paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))
factory := dependency.NewDefaultFactory(true) factory := dependency.NewDefaultFactory(true)
etcdCli, err := etcd.GetEtcdClient( etcdCli, err := etcd.GetEtcdClient(
@ -3714,12 +3714,11 @@ func Test_initServiceDiscovery(t *testing.T) {
func Test_newChunkManagerFactory(t *testing.T) { func Test_newChunkManagerFactory(t *testing.T) {
server := newTestServer2(t, nil) server := newTestServer2(t, nil)
Params.DataCoordCfg.EnableGarbageCollection = true paramtable.Get().Save(Params.DataCoordCfg.EnableGarbageCollection.Key, "true")
t.Run("err_minio_bad_address", func(t *testing.T) { t.Run("err_minio_bad_address", func(t *testing.T) {
os.Setenv("minio.address", "host:9000:bad") paramtable.Get().Save(Params.MinioCfg.Address.Key, "host:9000:bad")
defer os.Unsetenv("minio.address") defer paramtable.Get().Reset(Params.MinioCfg.Address.Key)
Params.Init()
storageCli, err := server.newChunkManagerFactory() storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli) assert.Nil(t, storageCli)
assert.Error(t, err) assert.Error(t, err)
@ -3727,22 +3726,19 @@ func Test_newChunkManagerFactory(t *testing.T) {
}) })
t.Run("local storage init", func(t *testing.T) { t.Run("local storage init", func(t *testing.T) {
Params.CommonCfg.StorageType = "local" paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "local")
defer paramtable.Get().Reset(Params.CommonCfg.StorageType.Key)
storageCli, err := server.newChunkManagerFactory() storageCli, err := server.newChunkManagerFactory()
assert.NotNil(t, storageCli) assert.NotNil(t, storageCli)
assert.NoError(t, err) assert.NoError(t, err)
}) })
t.Run("bad storage type", func(t *testing.T) {
Params.CommonCfg.StorageType = "bad"
storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli)
assert.Error(t, err)
})
} }
func Test_initGarbageCollection(t *testing.T) { func Test_initGarbageCollection(t *testing.T) {
paramtable.Get().Save(Params.DataCoordCfg.EnableGarbageCollection.Key, "true")
defer paramtable.Get().Reset(Params.DataCoordCfg.EnableGarbageCollection.Key)
server := newTestServer2(t, nil) server := newTestServer2(t, nil)
Params.DataCoordCfg.EnableGarbageCollection = true
t.Run("ok", func(t *testing.T) { t.Run("ok", func(t *testing.T) {
storageCli, err := server.newChunkManagerFactory() storageCli, err := server.newChunkManagerFactory()
@ -3751,10 +3747,9 @@ func Test_initGarbageCollection(t *testing.T) {
server.initGarbageCollection(storageCli) server.initGarbageCollection(storageCli)
}) })
t.Run("err_minio_bad_address", func(t *testing.T) { t.Run("err_minio_bad_address", func(t *testing.T) {
Params.CommonCfg.StorageType = "minio" paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "minio")
os.Setenv("minio.address", "host:9000:bad") paramtable.Get().Save(Params.MinioCfg.Address.Key, "host:9000:bad")
defer os.Unsetenv("minio.address") defer paramtable.Get().Reset(Params.MinioCfg.Address.Key)
Params.Init()
storageCli, err := server.newChunkManagerFactory() storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli) assert.Nil(t, storageCli)
assert.Error(t, err) assert.Error(t, err)
@ -3764,7 +3759,7 @@ func Test_initGarbageCollection(t *testing.T) {
func testDataCoordBase(t *testing.T, opts ...Option) *Server { func testDataCoordBase(t *testing.T, opts ...Option) *Server {
var err error var err error
Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int()) paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))
factory := dependency.NewDefaultFactory(true) factory := dependency.NewDefaultFactory(true)
etcdCli, err := etcd.GetEtcdClient( etcdCli, err := etcd.GetEtcdClient(
@ -3810,16 +3805,15 @@ func testDataCoordBase(t *testing.T, opts ...Option) *Server {
} }
func TestDataCoord_DisableActiveStandby(t *testing.T) { func TestDataCoord_DisableActiveStandby(t *testing.T) {
Params.Init() paramtable.Get().Save(Params.DataCoordCfg.EnableActiveStandby.Key, "false")
Params.DataCoordCfg.EnableActiveStandby = false
svr := testDataCoordBase(t) svr := testDataCoordBase(t)
defer closeTestServer(t, svr) defer closeTestServer(t, svr)
} }
// make sure the main functions work well when EnableActiveStandby=true // make sure the main functions work well when EnableActiveStandby=true
func TestDataCoord_EnableActiveStandby(t *testing.T) { func TestDataCoord_EnableActiveStandby(t *testing.T) {
Params.Init() paramtable.Get().Save(Params.DataCoordCfg.EnableActiveStandby.Key, "true")
Params.DataCoordCfg.EnableActiveStandby = true defer paramtable.Get().Reset(Params.DataCoordCfg.EnableActiveStandby.Key)
svr := testDataCoordBase(t) svr := testDataCoordBase(t)
defer closeTestServer(t, svr) defer closeTestServer(t, svr)
} }

View File

@ -56,7 +56,7 @@ func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringRespon
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success, ErrorCode: commonpb.ErrorCode_Success,
}, },
Value: Params.CommonCfg.DataCoordTimeTick, Value: Params.CommonCfg.DataCoordTimeTick.GetValue(),
}, nil }, nil
} }
@ -330,7 +330,7 @@ func (s *Server) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringRes
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success, ErrorCode: commonpb.ErrorCode_Success,
}, },
Value: Params.CommonCfg.DataCoordSegmentInfo, Value: Params.CommonCfg.DataCoordSegmentInfo.GetValue(),
}, nil }, nil
} }
@ -453,7 +453,7 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
s.segmentManager.DropSegment(ctx, req.SegmentID) s.segmentManager.DropSegment(ctx, req.SegmentID)
s.flushCh <- req.SegmentID s.flushCh <- req.SegmentID
if !req.Importing && Params.DataCoordCfg.EnableCompaction { if !req.Importing && Params.DataCoordCfg.EnableCompaction.GetAsBool() {
err = s.compactionTrigger.triggerSingleCompaction(segment.GetCollectionID(), segment.GetPartitionID(), err = s.compactionTrigger.triggerSingleCompaction(segment.GetCollectionID(), segment.GetPartitionID(),
segmentID, segment.GetInsertChannel()) segmentID, segment.GetInsertChannel())
if err != nil { if err != nil {
@ -923,7 +923,7 @@ func (s *Server) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompa
return resp, nil return resp, nil
} }
if !Params.DataCoordCfg.EnableCompaction { if !Params.DataCoordCfg.EnableCompaction.GetAsBool() {
resp.Status.Reason = "compaction disabled" resp.Status.Reason = "compaction disabled"
return resp, nil return resp, nil
} }
@ -957,7 +957,7 @@ func (s *Server) GetCompactionState(ctx context.Context, req *milvuspb.GetCompac
return resp, nil return resp, nil
} }
if !Params.DataCoordCfg.EnableCompaction { if !Params.DataCoordCfg.EnableCompaction.GetAsBool() {
resp.Status.Reason = "compaction disabled" resp.Status.Reason = "compaction disabled"
return resp, nil return resp, nil
} }
@ -996,7 +996,7 @@ func (s *Server) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.
return resp, nil return resp, nil
} }
if !Params.DataCoordCfg.EnableCompaction { if !Params.DataCoordCfg.EnableCompaction.GetAsBool() {
resp.Status.Reason = "compaction disabled" resp.Status.Reason = "compaction disabled"
return resp, nil return resp, nil
} }

View File

@ -88,14 +88,14 @@ func GetCompactTime(ctx context.Context, allocator allocator) (*compactTime, err
} }
pts, _ := tsoutil.ParseTS(ts) pts, _ := tsoutil.ParseTS(ts)
ttRetention := pts.Add(-time.Duration(Params.CommonCfg.RetentionDuration) * time.Second) ttRetention := pts.Add(-1 * Params.CommonCfg.RetentionDuration.GetAsDuration(time.Second))
ttRetentionLogic := tsoutil.ComposeTS(ttRetention.UnixNano()/int64(time.Millisecond), 0) ttRetentionLogic := tsoutil.ComposeTS(ttRetention.UnixNano()/int64(time.Millisecond), 0)
// TODO, change to collection level // TODO, change to collection level
if Params.CommonCfg.EntityExpirationTTL > 0 { if Params.CommonCfg.EntityExpirationTTL.GetAsInt() > 0 {
ttexpired := pts.Add(-Params.CommonCfg.EntityExpirationTTL) ttexpired := pts.Add(-1 * Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second))
ttexpiredLogic := tsoutil.ComposeTS(ttexpired.UnixNano()/int64(time.Millisecond), 0) ttexpiredLogic := tsoutil.ComposeTS(ttexpired.UnixNano()/int64(time.Millisecond), 0)
return &compactTime{ttRetentionLogic, ttexpiredLogic, Params.CommonCfg.EntityExpirationTTL}, nil return &compactTime{ttRetentionLogic, ttexpiredLogic, Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second)}, nil
} }
// no expiration time // no expiration time
return &compactTime{ttRetentionLogic, 0, 0}, nil return &compactTime{ttRetentionLogic, 0, 0}, nil
@ -207,5 +207,5 @@ func getCollectionTTL(properties map[string]string) (time.Duration, error) {
return time.Duration(ttl) * time.Second, nil return time.Duration(ttl) * time.Second, nil
} }
return Params.CommonCfg.EntityExpirationTTL, nil return Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second), nil
} }

View File

@ -25,6 +25,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb" "github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
) )
@ -117,11 +118,11 @@ func (suite *UtilSuite) TestVerifyResponse() {
} }
func (suite *UtilSuite) TestGetCompactTime() { func (suite *UtilSuite) TestGetCompactTime() {
Params.Init() paramtable.Get().Save(Params.CommonCfg.RetentionDuration.Key, "43200") // 5 days
Params.CommonCfg.RetentionDuration = 43200 // 5 days defer paramtable.Get().Reset(Params.CommonCfg.RetentionDuration.Key) // 5 days
tFixed := time.Date(2021, 11, 15, 0, 0, 0, 0, time.Local) tFixed := time.Date(2021, 11, 15, 0, 0, 0, 0, time.Local)
tBefore := tFixed.Add(-time.Duration(Params.CommonCfg.RetentionDuration) * time.Second) tBefore := tFixed.Add(-1 * Params.CommonCfg.RetentionDuration.GetAsDuration(time.Second))
type args struct { type args struct {
allocator allocator allocator allocator
@ -192,5 +193,5 @@ func (suite *UtilSuite) TestGetCollectionTTL() {
ttl, err = getCollectionTTL(map[string]string{}) ttl, err = getCollectionTTL(map[string]string{})
suite.NoError(err) suite.NoError(err)
suite.Equal(ttl, Params.CommonCfg.EntityExpirationTTL) suite.Equal(ttl, Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second))
} }

View File

@ -156,7 +156,7 @@ func (bm *DelBufferManager) CompactSegBuf(compactedToSegID UniqueID, compactedFr
func (bm *DelBufferManager) ShouldFlushSegments() []UniqueID { func (bm *DelBufferManager) ShouldFlushSegments() []UniqueID {
var shouldFlushSegments []UniqueID var shouldFlushSegments []UniqueID
if bm.delMemorySize < Params.DataNodeCfg.FlushDeleteBufferBytes { if bm.delMemorySize < Params.DataNodeCfg.FlushDeleteBufferBytes.GetAsInt64() {
return shouldFlushSegments return shouldFlushSegments
} }
mmUsage := bm.delMemorySize mmUsage := bm.delMemorySize
@ -167,7 +167,7 @@ func (bm *DelBufferManager) ShouldFlushSegments() []UniqueID {
shouldFlushSegments = append(shouldFlushSegments, segMem.segmentID) shouldFlushSegments = append(shouldFlushSegments, segMem.segmentID)
log.Debug("add segment for delete buf flush", zap.Int64("segmentID", segMem.segmentID)) log.Debug("add segment for delete buf flush", zap.Int64("segmentID", segMem.segmentID))
mmUsage -= segMem.memorySize mmUsage -= segMem.memorySize
if mmUsage < Params.DataNodeCfg.FlushDeleteBufferBytes { if mmUsage < Params.DataNodeCfg.FlushDeleteBufferBytes.GetAsInt64() {
break break
} }
} }
@ -357,7 +357,7 @@ func newBufferData(collSchema *schemapb.CollectionSchema) (*BufferData, error) {
return nil, errors.New("Invalid dimension") return nil, errors.New("Invalid dimension")
} }
limit := Params.DataNodeCfg.FlushInsertBufferSize / int64(vectorSize) limit := Params.DataNodeCfg.FlushInsertBufferSize.GetAsInt64() / int64(vectorSize)
//TODO::xige-16 eval vec and string field //TODO::xige-16 eval vec and string field
return &BufferData{ return &BufferData{

View File

@ -20,6 +20,7 @@ import (
"container/heap" "container/heap"
"fmt" "fmt"
"math" "math"
"strconv"
"testing" "testing"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -28,6 +29,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb" "github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/schemapb" "github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/paramtable"
) )
func genTestCollectionSchema(dim int64, vectorType schemapb.DataType) *schemapb.CollectionSchema { func genTestCollectionSchema(dim int64, vectorType schemapb.DataType) *schemapb.CollectionSchema {
@ -52,8 +54,7 @@ func genTestCollectionSchema(dim int64, vectorType schemapb.DataType) *schemapb.
} }
func TestBufferData(t *testing.T) { func TestBufferData(t *testing.T) {
Params.DataNodeCfg.FlushInsertBufferSize = 16 * (1 << 20) // 16 MB paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, strconv.FormatInt(16*(1<<20), 10)) // 16 MB
tests := []struct { tests := []struct {
isValid bool isValid bool
@ -92,7 +93,7 @@ func TestBufferData(t *testing.T) {
} }
func TestBufferData_updateTimeRange(t *testing.T) { func TestBufferData_updateTimeRange(t *testing.T) {
Params.DataNodeCfg.FlushInsertBufferSize = 16 * (1 << 20) // 16 MB paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, strconv.FormatInt(16*(1<<20), 10)) // 16 MB
type testCase struct { type testCase struct {
tag string tag string

View File

@ -172,7 +172,7 @@ func (c *ChannelMeta) maxRowCountPerSegment(ts Timestamp) (int64, error) {
log.Warn("failed to estimate size per record", zap.Error(err)) log.Warn("failed to estimate size per record", zap.Error(err))
return 0, err return 0, err
} }
threshold := Params.DataCoordCfg.SegmentMaxSize * 1024 * 1024 threshold := Params.DataCoordCfg.SegmentMaxSize.GetAsFloat() * 1024 * 1024
return int64(threshold / float64(sizePerRecord)), nil return int64(threshold / float64(sizePerRecord)), nil
} }

View File

@ -318,7 +318,7 @@ func (t *compactionTask) merge(
numRows = 0 numRows = 0
numBinlogs = 0 numBinlogs = 0
currentTs := t.GetCurrentTime() currentTs := t.GetCurrentTime()
maxRowsPerBinlog = int(Params.DataNodeCfg.FlushInsertBufferSize / (int64(dim) * 4)) maxRowsPerBinlog = int(Params.DataNodeCfg.FlushInsertBufferSize.GetAsInt64() / (int64(dim) * 4))
currentRows := 0 currentRows := 0
downloadTimeCost := time.Duration(0) downloadTimeCost := time.Duration(0)
uploadInsertTimeCost := time.Duration(0) uploadInsertTimeCost := time.Duration(0)

View File

@ -34,6 +34,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -273,7 +274,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
t.Run("Merge without expiration", func(t *testing.T) { t.Run("Merge without expiration", func(t *testing.T) {
alloc := NewAllocatorFactory(1) alloc := NewAllocatorFactory(1)
mockbIO := &binlogIO{cm, alloc} mockbIO := &binlogIO{cm, alloc}
Params.CommonCfg.EntityExpirationTTL = 0 paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0")
iData := genInsertDataWithExpiredTS() iData := genInsertDataWithExpiredTS()
var allPaths [][]string var allPaths [][]string
@ -305,12 +306,12 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
t.Run("Merge without expiration2", func(t *testing.T) { t.Run("Merge without expiration2", func(t *testing.T) {
alloc := NewAllocatorFactory(1) alloc := NewAllocatorFactory(1)
mockbIO := &binlogIO{cm, alloc} mockbIO := &binlogIO{cm, alloc}
Params.CommonCfg.EntityExpirationTTL = 0 paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0")
flushInsertBufferSize := Params.DataNodeCfg.FlushInsertBufferSize flushInsertBufferSize := Params.DataNodeCfg.FlushInsertBufferSize
defer func() { defer func() {
Params.DataNodeCfg.FlushInsertBufferSize = flushInsertBufferSize Params.DataNodeCfg.FlushInsertBufferSize = flushInsertBufferSize
}() }()
Params.DataNodeCfg.FlushInsertBufferSize = 128 paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "128")
iData := genInsertDataWithExpiredTS() iData := genInsertDataWithExpiredTS()
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
@ -385,7 +386,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
t.Run("Merge with meta error", func(t *testing.T) { t.Run("Merge with meta error", func(t *testing.T) {
alloc := NewAllocatorFactory(1) alloc := NewAllocatorFactory(1)
mockbIO := &binlogIO{cm, alloc} mockbIO := &binlogIO{cm, alloc}
Params.CommonCfg.EntityExpirationTTL = 0 paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0")
iData := genInsertDataWithExpiredTS() iData := genInsertDataWithExpiredTS()
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
@ -422,7 +423,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
t.Run("Merge with meta type param error", func(t *testing.T) { t.Run("Merge with meta type param error", func(t *testing.T) {
alloc := NewAllocatorFactory(1) alloc := NewAllocatorFactory(1)
mockbIO := &binlogIO{cm, alloc} mockbIO := &binlogIO{cm, alloc}
Params.CommonCfg.EntityExpirationTTL = 0 paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0")
iData := genInsertDataWithExpiredTS() iData := genInsertDataWithExpiredTS()
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
@ -561,7 +562,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
Field2StatslogPaths: nil, Field2StatslogPaths: nil,
Deltalogs: nil, Deltalogs: nil,
}} }}
Params.CommonCfg.EntityExpirationTTL = 0 // Turn off auto expiration paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0") // Turn off auto expiration
t.Run("Test compact invalid", func(t *testing.T) { t.Run("Test compact invalid", func(t *testing.T) {
invalidAlloc := NewAllocatorFactory(-1) invalidAlloc := NewAllocatorFactory(-1)

View File

@ -235,7 +235,7 @@ func (node *DataNode) initRateCollector() error {
// Init function does nothing now. // Init function does nothing now.
func (node *DataNode) Init() error { func (node *DataNode) Init() error {
log.Info("DataNode server initializing", log.Info("DataNode server initializing",
zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick), zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick.GetValue()),
) )
if err := node.initSession(); err != nil { if err := node.initSession(); err != nil {
log.Error("DataNode server init session failed", zap.Error(err)) log.Error("DataNode server init session failed", zap.Error(err))
@ -260,7 +260,7 @@ func (node *DataNode) Init() error {
node.factory.Init(Params) node.factory.Init(Params)
log.Info("DataNode server init succeeded", log.Info("DataNode server init succeeded",
zap.String("MsgChannelSubName", Params.CommonCfg.DataNodeSubName)) zap.String("MsgChannelSubName", Params.CommonCfg.DataNodeSubName.GetValue()))
return nil return nil
} }
@ -270,7 +270,7 @@ func (node *DataNode) StartWatchChannels(ctx context.Context) {
defer logutil.LogPanic() defer logutil.LogPanic()
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name} // REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
// TODO, this is risky, we'd better watch etcd with revision rather simply a path // TODO, this is risky, we'd better watch etcd with revision rather simply a path
watchPrefix := path.Join(Params.DataNodeCfg.ChannelWatchSubPath, fmt.Sprintf("%d", paramtable.GetNodeID())) watchPrefix := path.Join(Params.CommonCfg.DataCoordWatchSubPath.GetValue(), fmt.Sprintf("%d", paramtable.GetNodeID()))
evtChan := node.watchKv.WatchWithPrefix(watchPrefix) evtChan := node.watchKv.WatchWithPrefix(watchPrefix)
// after watch, first check all exists nodes first // after watch, first check all exists nodes first
err := node.checkWatchedList() err := node.checkWatchedList()
@ -312,7 +312,7 @@ func (node *DataNode) StartWatchChannels(ctx context.Context) {
// serves the corner case for etcd connection lost and missing some events // serves the corner case for etcd connection lost and missing some events
func (node *DataNode) checkWatchedList() error { func (node *DataNode) checkWatchedList() error {
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name} // REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
prefix := path.Join(Params.DataNodeCfg.ChannelWatchSubPath, fmt.Sprintf("%d", paramtable.GetNodeID())) prefix := path.Join(Params.CommonCfg.DataCoordWatchSubPath.GetValue(), fmt.Sprintf("%d", paramtable.GetNodeID()))
keys, values, err := node.watchKv.LoadWithPrefix(prefix) keys, values, err := node.watchKv.LoadWithPrefix(prefix)
if err != nil { if err != nil {
return err return err
@ -422,7 +422,7 @@ func (node *DataNode) handlePutEvent(watchInfo *datapb.ChannelWatchInfo, version
return fmt.Errorf("fail to marshal watchInfo with state, vChanName: %s, state: %s ,err: %w", vChanName, watchInfo.State.String(), err) return fmt.Errorf("fail to marshal watchInfo with state, vChanName: %s, state: %s ,err: %w", vChanName, watchInfo.State.String(), err)
} }
key := path.Join(Params.DataNodeCfg.ChannelWatchSubPath, fmt.Sprintf("%d", paramtable.GetNodeID()), vChanName) key := path.Join(Params.CommonCfg.DataCoordWatchSubPath.GetValue(), fmt.Sprintf("%d", paramtable.GetNodeID()), vChanName)
success, err := node.watchKv.CompareVersionAndSwap(key, version, string(v)) success, err := node.watchKv.CompareVersionAndSwap(key, version, string(v))
// etcd error, retrying // etcd error, retrying
@ -518,9 +518,6 @@ func (node *DataNode) Start() error {
// Start node watch node // Start node watch node
go node.StartWatchChannels(node.ctx) go node.StartWatchChannels(node.ctx)
Params.DataNodeCfg.CreatedTime = time.Now()
Params.DataNodeCfg.UpdatedTime = time.Now()
node.UpdateStateCode(commonpb.StateCode_Healthy) node.UpdateStateCode(commonpb.StateCode_Healthy)
return nil return nil
} }
@ -1068,7 +1065,7 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest)
} }
// parse files and generate segments // parse files and generate segments
segmentSize := int64(Params.DataCoordCfg.SegmentMaxSize) * 1024 * 1024 segmentSize := Params.DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024
importWrapper := importutil.NewImportWrapper(newCtx, colInfo.GetSchema(), colInfo.GetShardsNum(), segmentSize, node.rowIDAllocator, importWrapper := importutil.NewImportWrapper(newCtx, colInfo.GetSchema(), colInfo.GetShardsNum(), segmentSize, node.rowIDAllocator,
node.chunkManager, importResult, reportFunc) node.chunkManager, importResult, reportFunc)
importWrapper.SetCallbackFunctions(assignSegmentFunc(node, req), importWrapper.SetCallbackFunctions(assignSegmentFunc(node, req),

View File

@ -75,10 +75,9 @@ func TestMain(t *testing.M) {
os.Setenv("ROCKSMQ_PATH", path) os.Setenv("ROCKSMQ_PATH", path)
defer os.RemoveAll(path) defer os.RemoveAll(path)
Params.DataNodeCfg.InitAlias("datanode-alias-1")
Params.Init() Params.Init()
// change to specific channel for test // change to specific channel for test
Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int()) paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))
rateCol, err = newRateCollector() rateCol, err = newRateCollector()
if err != nil { if err != nil {
@ -876,15 +875,15 @@ func TestWatchChannel(t *testing.T) {
// GOOSE TODO // GOOSE TODO
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue()) kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid" oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh) path := fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), oldInvalidCh)
err = kv.Save(path, string([]byte{23})) err = kv.Save(path, string([]byte{23}))
assert.NoError(t, err) assert.NoError(t, err)
ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31()) ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31())
path = fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), ch) path = fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), ch)
c := make(chan struct{}) c := make(chan struct{})
go func() { go func() {
ec := kv.WatchWithPrefix(fmt.Sprintf("%s/%d", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID())) ec := kv.WatchWithPrefix(fmt.Sprintf("%s/%d", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID()))
c <- struct{}{} c <- struct{}{}
cnt := 0 cnt := 0
for { for {
@ -923,7 +922,7 @@ func TestWatchChannel(t *testing.T) {
exist := node.flowgraphManager.exist(ch) exist := node.flowgraphManager.exist(ch)
assert.True(t, exist) assert.True(t, exist)
err = kv.RemoveWithPrefix(fmt.Sprintf("%s/%d", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID())) err = kv.RemoveWithPrefix(fmt.Sprintf("%s/%d", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID()))
assert.Nil(t, err) assert.Nil(t, err)
//TODO there is not way to sync Release done, use sleep for now //TODO there is not way to sync Release done, use sleep for now
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
@ -935,15 +934,15 @@ func TestWatchChannel(t *testing.T) {
t.Run("Test release channel", func(t *testing.T) { t.Run("Test release channel", func(t *testing.T) {
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue()) kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid" oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh) path := fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), oldInvalidCh)
err = kv.Save(path, string([]byte{23})) err = kv.Save(path, string([]byte{23}))
assert.NoError(t, err) assert.NoError(t, err)
ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31()) ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31())
path = fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), ch) path = fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), ch)
c := make(chan struct{}) c := make(chan struct{})
go func() { go func() {
ec := kv.WatchWithPrefix(fmt.Sprintf("%s/%d", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID())) ec := kv.WatchWithPrefix(fmt.Sprintf("%s/%d", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID()))
c <- struct{}{} c <- struct{}{}
cnt := 0 cnt := 0
for { for {
@ -982,7 +981,7 @@ func TestWatchChannel(t *testing.T) {
exist := node.flowgraphManager.exist(ch) exist := node.flowgraphManager.exist(ch)
assert.False(t, exist) assert.False(t, exist)
err = kv.RemoveWithPrefix(fmt.Sprintf("%s/%d", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID())) err = kv.RemoveWithPrefix(fmt.Sprintf("%s/%d", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID()))
assert.Nil(t, err) assert.Nil(t, err)
//TODO there is not way to sync Release done, use sleep for now //TODO there is not way to sync Release done, use sleep for now
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)

View File

@ -121,7 +121,7 @@ type nodeConfig struct {
} }
func newParallelConfig() parallelConfig { func newParallelConfig() parallelConfig {
return parallelConfig{Params.DataNodeCfg.FlowGraphMaxQueueLength, Params.DataNodeCfg.FlowGraphMaxParallelism} return parallelConfig{Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(), Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()}
} }
// start starts the flow graph in datasyncservice // start starts the flow graph in datasyncservice

View File

@ -36,6 +36,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/paramtable"
) )
var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service" var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service"
@ -217,7 +218,7 @@ func TestDataSyncService_Start(t *testing.T) {
allocFactory := NewAllocatorFactory(1) allocFactory := NewAllocatorFactory(1)
factory := dependency.NewDefaultFactory(true) factory := dependency.NewDefaultFactory(true)
Params.DataNodeCfg.FlushInsertBufferSize = 1 paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "1")
ufs := []*datapb.SegmentInfo{{ ufs := []*datapb.SegmentInfo{{
CollectionID: collMeta.ID, CollectionID: collMeta.ID,

View File

@ -347,8 +347,8 @@ func newDDNode(ctx context.Context, collID UniqueID, vChannelName string, droppe
msFactory msgstream.Factory, compactor *compactionExecutor) (*ddNode, error) { msFactory msgstream.Factory, compactor *compactionExecutor) (*ddNode, error) {
baseNode := BaseNode{} baseNode := BaseNode{}
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength) baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism) baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
deltaStream, err := msFactory.NewMsgStream(ctx) deltaStream, err := msFactory.NewMsgStream(ctx)
if err != nil { if err != nil {
@ -360,7 +360,7 @@ func newDDNode(ctx context.Context, collID UniqueID, vChannelName string, droppe
zap.String("pChannelName", pChannelName), zap.String("pChannelName", pChannelName),
) )
deltaChannelName, err := funcutil.ConvertChannelName(pChannelName, Params.CommonCfg.RootCoordDml, Params.CommonCfg.RootCoordDelta) deltaChannelName, err := funcutil.ConvertChannelName(pChannelName, Params.CommonCfg.RootCoordDml.GetValue(), Params.CommonCfg.RootCoordDelta.GetValue())
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -33,6 +33,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
) )
@ -408,7 +409,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
//1. here we set buffer bytes to a relatively high level //1. here we set buffer bytes to a relatively high level
//and the sum of memory consumption in this case is 208 //and the sum of memory consumption in this case is 208
//so no segments will be flushed //so no segments will be flushed
Params.DataNodeCfg.FlushDeleteBufferBytes = 300 paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "300")
delNode.Operate([]flowgraph.Msg{fgMsg}) delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 0, len(mockFlushManager.flushedSegIDs)) assert.Equal(t, 0, len(mockFlushManager.flushedSegIDs))
assert.Equal(t, int64(208), delNode.delBufferManager.delMemorySize) assert.Equal(t, int64(208), delNode.delBufferManager.delMemorySize)
@ -420,7 +421,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
//memory consumption will be reduced to 160(under 200) //memory consumption will be reduced to 160(under 200)
msg.deleteMessages = []*msgstream.DeleteMsg{} msg.deleteMessages = []*msgstream.DeleteMsg{}
msg.segmentsToSync = []UniqueID{} msg.segmentsToSync = []UniqueID{}
Params.DataNodeCfg.FlushDeleteBufferBytes = 200 paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "200")
delNode.Operate([]flowgraph.Msg{fgMsg}) delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 1, len(mockFlushManager.flushedSegIDs)) assert.Equal(t, 1, len(mockFlushManager.flushedSegIDs))
assert.Equal(t, int64(160), delNode.delBufferManager.delMemorySize) assert.Equal(t, int64(160), delNode.delBufferManager.delMemorySize)
@ -436,7 +437,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
//5. we reset buffer bytes to 150, then we expect there would be one more //5. we reset buffer bytes to 150, then we expect there would be one more
//segment which is 48 in size to be flushed, so the remained del memory size //segment which is 48 in size to be flushed, so the remained del memory size
//will be 112 //will be 112
Params.DataNodeCfg.FlushDeleteBufferBytes = 150 paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "150")
delNode.Operate([]flowgraph.Msg{fgMsg}) delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 2, len(mockFlushManager.flushedSegIDs)) assert.Equal(t, 2, len(mockFlushManager.flushedSegIDs))
assert.Equal(t, int64(112), delNode.delBufferManager.delMemorySize) assert.Equal(t, int64(112), delNode.delBufferManager.delMemorySize)
@ -444,7 +445,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
//6. we reset buffer bytes to 60, then most of the segments will be flushed //6. we reset buffer bytes to 60, then most of the segments will be flushed
//except for the smallest entry with size equaling to 32 //except for the smallest entry with size equaling to 32
Params.DataNodeCfg.FlushDeleteBufferBytes = 60 paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "60")
delNode.Operate([]flowgraph.Msg{fgMsg}) delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 4, len(mockFlushManager.flushedSegIDs)) assert.Equal(t, 4, len(mockFlushManager.flushedSegIDs))
assert.Equal(t, int64(32), delNode.delBufferManager.delMemorySize) assert.Equal(t, int64(32), delNode.delBufferManager.delMemorySize)
@ -453,7 +454,7 @@ func TestFlowGraphDeleteNode_Operate(t *testing.T) {
//7. we reset buffer bytes to 20, then as all segment-memory consumption //7. we reset buffer bytes to 20, then as all segment-memory consumption
//is more than 20, so all five segments will be flushed and the remained //is more than 20, so all five segments will be flushed and the remained
//del memory will be lowered to zero //del memory will be lowered to zero
Params.DataNodeCfg.FlushDeleteBufferBytes = 20 paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "20")
delNode.Operate([]flowgraph.Msg{fgMsg}) delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 5, len(mockFlushManager.flushedSegIDs)) assert.Equal(t, 5, len(mockFlushManager.flushedSegIDs))
assert.Equal(t, int64(0), delNode.delBufferManager.delMemorySize) assert.Equal(t, int64(0), delNode.delBufferManager.delMemorySize)

View File

@ -40,7 +40,7 @@ import (
func newDmInputNode(ctx context.Context, seekPos *internalpb.MsgPosition, dmNodeConfig *nodeConfig) (*flowgraph.InputNode, error) { func newDmInputNode(ctx context.Context, seekPos *internalpb.MsgPosition, dmNodeConfig *nodeConfig) (*flowgraph.InputNode, error) {
// subName should be unique, since pchannelName is shared among several collections // subName should be unique, since pchannelName is shared among several collections
// use vchannel in case of reuse pchannel for same collection // use vchannel in case of reuse pchannel for same collection
consumeSubName := fmt.Sprintf("%s-%d-%s", Params.CommonCfg.DataNodeSubName, paramtable.GetNodeID(), dmNodeConfig.vChannelName) consumeSubName := fmt.Sprintf("%s-%d-%s", Params.CommonCfg.DataNodeSubName.GetValue(), paramtable.GetNodeID(), dmNodeConfig.vChannelName)
insertStream, err := dmNodeConfig.msFactory.NewTtMsgStream(ctx) insertStream, err := dmNodeConfig.msFactory.NewTtMsgStream(ctx)
if err != nil { if err != nil {
return nil, err return nil, err

View File

@ -605,9 +605,9 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl
if err != nil { if err != nil {
return nil, err return nil, err
} }
wTt.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick}) wTt.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc() metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
log.Info("datanode AsProducer", zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick)) log.Info("datanode AsProducer", zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick.GetValue()))
var wTtMsgStream msgstream.MsgStream = wTt var wTtMsgStream msgstream.MsgStream = wTt
mt := newMergedTimeTickerSender(func(ts Timestamp, segmentIDs []int64) error { mt := newMergedTimeTickerSender(func(ts Timestamp, segmentIDs []int64) error {

View File

@ -20,6 +20,7 @@ import (
"context" "context"
"errors" "errors"
"math" "math"
"strconv"
"sync" "sync"
"testing" "testing"
"time" "time"
@ -35,6 +36,7 @@ import (
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/samber/lo" "github.com/samber/lo"
@ -371,7 +373,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
t.Run("Pure auto flush", func(t *testing.T) { t.Run("Pure auto flush", func(t *testing.T) {
// iBNode.insertBuffer.maxSize = 2 // iBNode.insertBuffer.maxSize = 2
tmp := Params.DataNodeCfg.FlushInsertBufferSize tmp := Params.DataNodeCfg.FlushInsertBufferSize
Params.DataNodeCfg.FlushInsertBufferSize = 4 * 4 paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "16")
defer func() { defer func() {
Params.DataNodeCfg.FlushInsertBufferSize = tmp Params.DataNodeCfg.FlushInsertBufferSize = tmp
}() }()
@ -463,7 +465,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
t.Run("Auto with manual flush", func(t *testing.T) { t.Run("Auto with manual flush", func(t *testing.T) {
tmp := Params.DataNodeCfg.FlushInsertBufferSize tmp := Params.DataNodeCfg.FlushInsertBufferSize
Params.DataNodeCfg.FlushInsertBufferSize = 4 * 4 paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "16")
defer func() { defer func() {
Params.DataNodeCfg.FlushInsertBufferSize = tmp Params.DataNodeCfg.FlushInsertBufferSize = tmp
}() }()
@ -605,7 +607,7 @@ func TestRollBF(t *testing.T) {
t.Run("Pure roll BF", func(t *testing.T) { t.Run("Pure roll BF", func(t *testing.T) {
tmp := Params.DataNodeCfg.FlushInsertBufferSize tmp := Params.DataNodeCfg.FlushInsertBufferSize
Params.DataNodeCfg.FlushInsertBufferSize = 4 * 4 paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "16")
defer func() { defer func() {
Params.DataNodeCfg.FlushInsertBufferSize = tmp Params.DataNodeCfg.FlushInsertBufferSize = tmp
}() }()
@ -693,14 +695,14 @@ func (s *InsertBufferNodeSuit) SetupSuite() {
s.channel = newChannel("channel", s.collID, nil, rc, s.cm) s.channel = newChannel("channel", s.collID, nil, rc, s.cm)
s.cm = storage.NewLocalChunkManager(storage.RootPath(insertBufferNodeTestDir)) s.cm = storage.NewLocalChunkManager(storage.RootPath(insertBufferNodeTestDir))
s.originalConfig = Params.DataNodeCfg.FlushInsertBufferSize s.originalConfig = Params.DataNodeCfg.FlushInsertBufferSize.GetAsInt64()
// change flushing size to 2 // change flushing size to 2
Params.DataNodeCfg.FlushInsertBufferSize = 4 * 4 paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "16")
} }
func (s *InsertBufferNodeSuit) TearDownSuite() { func (s *InsertBufferNodeSuit) TearDownSuite() {
s.cm.RemoveWithPrefix(context.Background(), s.cm.RootPath()) s.cm.RemoveWithPrefix(context.Background(), s.cm.RootPath())
Params.DataNodeCfg.FlushInsertBufferSize = s.originalConfig paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, strconv.FormatInt(s.originalConfig, 10))
} }
func (s *InsertBufferNodeSuit) SetupTest() { func (s *InsertBufferNodeSuit) SetupTest() {

View File

@ -111,8 +111,8 @@ func (ttn *ttNode) updateChannelCP(ttPos *internalpb.MsgPosition) {
func newTTNode(config *nodeConfig, dc types.DataCoord) (*ttNode, error) { func newTTNode(config *nodeConfig, dc types.DataCoord) (*ttNode, error) {
baseNode := BaseNode{} baseNode := BaseNode{}
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength) baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism) baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
tt := &ttNode{ tt := &ttNode{
BaseNode: baseNode, BaseNode: baseNode,

View File

@ -10,7 +10,7 @@ var ioPool *concurrency.Pool
var ioPoolInitOnce sync.Once var ioPoolInitOnce sync.Once
func initIOPool() { func initIOPool() {
capacity := Params.DataNodeCfg.IOConcurrency capacity := Params.DataNodeCfg.IOConcurrency.GetAsInt()
if capacity > 32 { if capacity > 32 {
capacity = 32 capacity = 32
} }

View File

@ -7,12 +7,13 @@ import (
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/util/concurrency" "github.com/milvus-io/milvus/internal/util/concurrency"
"github.com/milvus-io/milvus/internal/util/paramtable"
) )
func Test_getOrCreateIOPool(t *testing.T) { func Test_getOrCreateIOPool(t *testing.T) {
Params.InitOnce() Params.InitOnce()
ioConcurrency := Params.DataNodeCfg.IOConcurrency ioConcurrency := Params.DataNodeCfg.IOConcurrency
Params.DataNodeCfg.IOConcurrency = 64 paramtable.Get().Save(Params.DataNodeCfg.IOConcurrency.Key, "64")
defer func() { Params.DataNodeCfg.IOConcurrency = ioConcurrency }() defer func() { Params.DataNodeCfg.IOConcurrency = ioConcurrency }()
nP := 10 nP := 10
nTask := 10 nTask := 10

View File

@ -64,7 +64,7 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro
//getComponentConfigurations returns the configurations of dataNode matching req.Pattern //getComponentConfigurations returns the configurations of dataNode matching req.Pattern
func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse { func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse {
prefix := "datanode." prefix := "datanode."
matchedConfig := Params.DataNodeCfg.Base.GetByPattern(prefix + req.Pattern) matchedConfig := Params.GetByPattern(prefix + req.Pattern)
configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig)) configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig))
for key, value := range matchedConfig { for key, value := range matchedConfig {
configList = append(configList, configList = append(configList,
@ -114,13 +114,13 @@ func (node *DataNode) getSystemInfoMetrics(ctx context.Context, req *milvuspb.Ge
Name: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, paramtable.GetNodeID()), Name: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, paramtable.GetNodeID()),
HardwareInfos: hardwareMetrics, HardwareInfos: hardwareMetrics,
SystemInfo: metricsinfo.DeployMetrics{}, SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.DataNodeCfg.CreatedTime.String(), CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: Params.DataNodeCfg.UpdatedTime.String(), UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.DataNodeRole, Type: typeutil.DataNodeRole,
ID: node.session.ServerID, ID: node.session.ServerID,
}, },
SystemConfigurations: metricsinfo.DataNodeConfiguration{ SystemConfigurations: metricsinfo.DataNodeConfiguration{
FlushInsertBufferSize: Params.DataNodeCfg.FlushInsertBufferSize, FlushInsertBufferSize: Params.DataNodeCfg.FlushInsertBufferSize.GetAsInt64(),
}, },
QuotaMetrics: quotaMetrics, QuotaMetrics: quotaMetrics,
} }

View File

@ -17,6 +17,8 @@
package datanode package datanode
import ( import (
"time"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
) )
@ -28,7 +30,7 @@ func syncPeriodically() segmentSyncPolicy {
return func(segment *Segment, ts Timestamp) bool { return func(segment *Segment, ts Timestamp) bool {
endTime := tsoutil.PhysicalTime(ts) endTime := tsoutil.PhysicalTime(ts)
lastSyncTime := tsoutil.PhysicalTime(segment.lastSyncTs) lastSyncTime := tsoutil.PhysicalTime(segment.lastSyncTs)
return endTime.Sub(lastSyncTime) >= Params.DataNodeCfg.SyncPeriod && return endTime.Sub(lastSyncTime) >= Params.DataNodeCfg.SyncPeriod.GetAsDuration(time.Second) &&
!segment.isBufferEmpty() !segment.isBufferEmpty()
} }
} }

View File

@ -35,10 +35,10 @@ func TestSyncPeriodically(t *testing.T) {
isBufferEmpty bool isBufferEmpty bool
shouldSync bool shouldSync bool
}{ }{
{"test buffer empty and stale", t0, t0.Add(Params.DataNodeCfg.SyncPeriod), true, false}, {"test buffer empty and stale", t0, t0.Add(Params.DataNodeCfg.SyncPeriod.GetAsDuration(time.Second)), true, false},
{"test buffer empty and not stale", t0, t0.Add(Params.DataNodeCfg.SyncPeriod / 2), true, false}, {"test buffer empty and not stale", t0, t0.Add(Params.DataNodeCfg.SyncPeriod.GetAsDuration(time.Second) / 2), true, false},
{"test buffer not empty and stale", t0, t0.Add(Params.DataNodeCfg.SyncPeriod), false, true}, {"test buffer not empty and stale", t0, t0.Add(Params.DataNodeCfg.SyncPeriod.GetAsDuration(time.Second)), false, true},
{"test buffer not empty and not stale", t0, t0.Add(Params.DataNodeCfg.SyncPeriod / 2), false, false}, {"test buffer not empty and not stale", t0, t0.Add(Params.DataNodeCfg.SyncPeriod.GetAsDuration(time.Second) / 2), false, false},
} }
for _, test := range tests { for _, test := range tests {

View File

@ -117,7 +117,7 @@ func (s *Server) registerHTTPServer() {
// (Embedded Milvus Only) Discard gin logs if logging is disabled. // (Embedded Milvus Only) Discard gin logs if logging is disabled.
// We might need to put these logs in some files in the further. // We might need to put these logs in some files in the further.
// But we don't care about these logs now, at least not in embedded Milvus. // But we don't care about these logs now, at least not in embedded Milvus.
if !proxy.Params.ProxyCfg.GinLogging { if !proxy.Params.ProxyCfg.GinLogging.GetAsBool() {
gin.DefaultWriter = io.Discard gin.DefaultWriter = io.Discard
gin.DefaultErrorWriter = io.Discard gin.DefaultErrorWriter = io.Discard
} }

View File

@ -1357,7 +1357,7 @@ func Test_NewServer(t *testing.T) {
// Update config and start server again to test with different config set. // Update config and start server again to test with different config set.
// This works as config will be initialized only once // This works as config will be initialized only once
proxy.Params.ProxyCfg.GinLogging = false paramtable.Get().Save(proxy.Params.ProxyCfg.GinLogging.Key, "false")
err = runAndWaitForServerReady(server) err = runAndWaitForServerReady(server)
assert.Nil(t, err) assert.Nil(t, err)
err = server.Stop() err = server.Stop()

View File

@ -19,9 +19,11 @@ package grpcquerynode
import ( import (
"context" "context"
"errors" "errors"
"os"
"testing" "testing"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
@ -226,6 +228,12 @@ func (m *MockIndexCoord) GetComponentStates(ctx context.Context) (*milvuspb.Comp
} }
/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
func TestMain(m *testing.M) {
paramtable.Init()
os.Exit(m.Run())
}
func Test_NewServer(t *testing.T) { func Test_NewServer(t *testing.T) {
ctx := context.Background() ctx := context.Background()
server, err := NewServer(ctx, nil) server, err := NewServer(ctx, nil)

View File

@ -52,7 +52,7 @@ func newGarbageCollector(ctx context.Context, meta *metaTable, chunkManager stor
return &garbageCollector{ return &garbageCollector{
ctx: ctx, ctx: ctx,
cancel: cancel, cancel: cancel,
gcFileDuration: Params.IndexCoordCfg.GCInterval, gcFileDuration: Params.IndexCoordCfg.GCInterval.GetAsDuration(time.Second),
gcMetaDuration: time.Minute, gcMetaDuration: time.Minute,
metaTable: meta, metaTable: meta,
chunkManager: chunkManager, chunkManager: chunkManager,

View File

@ -213,7 +213,7 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
return true return true
} }
indexParams := ib.meta.GetIndexParams(meta.CollectionID, meta.IndexID) indexParams := ib.meta.GetIndexParams(meta.CollectionID, meta.IndexID)
if isFlatIndex(getIndexType(indexParams)) || meta.NumRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex { if isFlatIndex(getIndexType(indexParams)) || meta.NumRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex.GetAsInt64() {
log.Ctx(ib.ctx).Debug("segment does not need index really", zap.Int64("buildID", buildID), log.Ctx(ib.ctx).Debug("segment does not need index really", zap.Int64("buildID", buildID),
zap.Int64("segID", meta.SegmentID), zap.Int64("num rows", meta.NumRows)) zap.Int64("segID", meta.SegmentID), zap.Int64("num rows", meta.NumRows))
if err := ib.meta.FinishTask(&indexpb.IndexTaskInfo{ if err := ib.meta.FinishTask(&indexpb.IndexTaskInfo{
@ -274,10 +274,10 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
typeParams := ib.meta.GetTypeParams(meta.CollectionID, meta.IndexID) typeParams := ib.meta.GetTypeParams(meta.CollectionID, meta.IndexID)
var storageConfig *indexpb.StorageConfig var storageConfig *indexpb.StorageConfig
if Params.CommonCfg.StorageType == "local" { if Params.CommonCfg.StorageType.GetValue() == "local" {
storageConfig = &indexpb.StorageConfig{ storageConfig = &indexpb.StorageConfig{
RootPath: Params.LocalStorageCfg.Path.GetValue(), RootPath: Params.LocalStorageCfg.Path.GetValue(),
StorageType: Params.CommonCfg.StorageType, StorageType: Params.CommonCfg.StorageType.GetValue(),
} }
} else { } else {
storageConfig = &indexpb.StorageConfig{ storageConfig = &indexpb.StorageConfig{
@ -289,11 +289,11 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
RootPath: Params.MinioCfg.RootPath.GetValue(), RootPath: Params.MinioCfg.RootPath.GetValue(),
UseIAM: Params.MinioCfg.UseIAM.GetAsBool(), UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(), IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
StorageType: Params.CommonCfg.StorageType, StorageType: Params.CommonCfg.StorageType.GetValue(),
} }
} }
req := &indexpb.CreateJobRequest{ req := &indexpb.CreateJobRequest{
ClusterID: Params.CommonCfg.ClusterPrefix, ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
IndexFilePrefix: path.Join(ib.ic.chunkManager.RootPath(), common.SegmentIndexPath), IndexFilePrefix: path.Join(ib.ic.chunkManager.RootPath(), common.SegmentIndexPath),
BuildID: buildID, BuildID: buildID,
DataPaths: binLogs, DataPaths: binLogs,
@ -390,7 +390,7 @@ func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval) ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
defer cancel() defer cancel()
response, err := client.QueryJobs(ctx1, &indexpb.QueryJobsRequest{ response, err := client.QueryJobs(ctx1, &indexpb.QueryJobsRequest{
ClusterID: Params.CommonCfg.ClusterPrefix, ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
BuildIDs: []int64{buildID}, BuildIDs: []int64{buildID},
}) })
if err != nil { if err != nil {
@ -439,7 +439,7 @@ func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval) ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
defer cancel() defer cancel()
status, err := client.DropJobs(ctx1, &indexpb.DropJobsRequest{ status, err := client.DropJobs(ctx1, &indexpb.DropJobsRequest{
ClusterID: Params.CommonCfg.ClusterPrefix, ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
BuildIDs: []UniqueID{buildID}, BuildIDs: []UniqueID{buildID},
}) })
if err != nil { if err != nil {

View File

@ -24,6 +24,7 @@ import (
"time" "time"
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
@ -629,7 +630,7 @@ func TestIndexBuilder_Error(t *testing.T) {
}) })
t.Run("assign task fail", func(t *testing.T) { t.Run("assign task fail", func(t *testing.T) {
Params.CommonCfg.StorageType = "local" paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "local")
ib.tasks[buildID] = indexTaskInit ib.tasks[buildID] = indexTaskInit
ib.ic.dataCoordClient = NewDataCoordMock() ib.ic.dataCoordClient = NewDataCoordMock()
ib.meta = createMetaTable(&indexcoord.Catalog{ ib.meta = createMetaTable(&indexcoord.Catalog{

View File

@ -129,7 +129,7 @@ func NewIndexCoord(ctx context.Context, factory dependency.Factory) (*IndexCoord
loopCancel: cancel, loopCancel: cancel,
reqTimeoutInterval: time.Second * 10, reqTimeoutInterval: time.Second * 10,
factory: factory, factory: factory,
enableActiveStandBy: Params.IndexCoordCfg.EnableActiveStandby, enableActiveStandBy: Params.IndexCoordCfg.EnableActiveStandby.GetAsBool(),
} }
i.UpdateStateCode(commonpb.StateCode_Abnormal) i.UpdateStateCode(commonpb.StateCode_Abnormal)
return i, nil return i, nil
@ -205,19 +205,20 @@ func (i *IndexCoord) Init() error {
initErr = err initErr = err
return return
} }
log.Info("IndexCoord get node sessions from etcd", zap.Bool("bind mode", Params.IndexCoordCfg.BindIndexNodeMode), log.Info("IndexCoord get node sessions from etcd",
zap.String("node address", Params.IndexCoordCfg.IndexNodeAddress)) zap.String("bind mode", Params.IndexCoordCfg.BindIndexNodeMode.GetValue()),
zap.String("node address", Params.IndexCoordCfg.IndexNodeAddress.GetValue()))
aliveNodeID := make([]UniqueID, 0) aliveNodeID := make([]UniqueID, 0)
if Params.IndexCoordCfg.BindIndexNodeMode { if Params.IndexCoordCfg.BindIndexNodeMode.GetAsBool() {
if err = i.nodeManager.AddNode(Params.IndexCoordCfg.IndexNodeID, Params.IndexCoordCfg.IndexNodeAddress); err != nil { if err = i.nodeManager.AddNode(Params.IndexCoordCfg.IndexNodeID.GetAsInt64(), Params.IndexCoordCfg.IndexNodeAddress.GetValue()); err != nil {
log.Error("IndexCoord add node fail", zap.Int64("ServerID", Params.IndexCoordCfg.IndexNodeID), log.Error("IndexCoord add node fail", zap.Int64("ServerID", Params.IndexCoordCfg.IndexNodeID.GetAsInt64()),
zap.String("address", Params.IndexCoordCfg.IndexNodeAddress), zap.Error(err)) zap.String("address", Params.IndexCoordCfg.IndexNodeAddress.GetValue()), zap.Error(err))
initErr = err initErr = err
return return
} }
log.Info("IndexCoord add node success", zap.String("IndexNode address", Params.IndexCoordCfg.IndexNodeAddress), log.Info("IndexCoord add node success", zap.String("IndexNode address", Params.IndexCoordCfg.IndexNodeAddress.GetValue()),
zap.Int64("nodeID", Params.IndexCoordCfg.IndexNodeID)) zap.Int64("nodeID", Params.IndexCoordCfg.IndexNodeID.GetAsInt64()))
aliveNodeID = append(aliveNodeID, Params.IndexCoordCfg.IndexNodeID) aliveNodeID = append(aliveNodeID, Params.IndexCoordCfg.IndexNodeID.GetAsInt64())
metrics.IndexCoordIndexNodeNum.WithLabelValues().Inc() metrics.IndexCoordIndexNodeNum.WithLabelValues().Inc()
} else { } else {
for _, session := range sessions { for _, session := range sessions {
@ -293,9 +294,6 @@ func (i *IndexCoord) Start() error {
cb() cb()
} }
Params.IndexCoordCfg.CreatedTime = time.Now()
Params.IndexCoordCfg.UpdatedTime = time.Now()
if i.enableActiveStandBy { if i.enableActiveStandBy {
i.activateFunc = func() { i.activateFunc = func() {
log.Info("IndexCoord switch from standby to active, reload the KV") log.Info("IndexCoord switch from standby to active, reload the KV")
@ -1085,7 +1083,7 @@ func (i *IndexCoord) watchNodeLoop() {
} }
return return
} }
if Params.IndexCoordCfg.BindIndexNodeMode { if Params.IndexCoordCfg.BindIndexNodeMode.GetAsBool() {
continue continue
} }
switch event.EventType { switch event.EventType {

View File

@ -21,6 +21,7 @@ import (
"errors" "errors"
"fmt" "fmt"
"math/rand" "math/rand"
"os"
"path" "path"
"strconv" "strconv"
"sync" "sync"
@ -43,10 +44,17 @@ import (
"github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
func TestMain(m *testing.M) {
paramtable.Init()
rand.Seed(time.Now().UnixNano())
os.Exit(m.Run())
}
func TestMockEtcd(t *testing.T) { func TestMockEtcd(t *testing.T) {
Params.InitOnce() Params.InitOnce()
Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root/indexcoord-mock") Params.BaseTable.Save("etcd.rootPath", "/test/datanode/root/indexcoord-mock")
@ -528,16 +536,16 @@ func testIndexCoord(t *testing.T) {
func TestIndexCoord_DisableActiveStandby(t *testing.T) { func TestIndexCoord_DisableActiveStandby(t *testing.T) {
Params.InitOnce() Params.InitOnce()
indexnode.Params.InitOnce() // indexnode.Params.InitOnce()
Params.IndexCoordCfg.EnableActiveStandby = false paramtable.Get().Save(Params.IndexCoordCfg.EnableActiveStandby.Key, "false")
testIndexCoord(t) testIndexCoord(t)
} }
// make sure the main functions work well when EnableActiveStandby=true // make sure the main functions work well when EnableActiveStandby=true
func TestIndexCoord_EnableActiveStandby(t *testing.T) { func TestIndexCoord_EnableActiveStandby(t *testing.T) {
Params.InitOnce() Params.InitOnce()
indexnode.Params.InitOnce() // indexnode.Params.InitOnce()
Params.IndexCoordCfg.EnableActiveStandby = true paramtable.Get().Save(Params.IndexCoordCfg.EnableActiveStandby.Key, "true")
testIndexCoord(t) testIndexCoord(t)
} }

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/hardware" "github.com/milvus-io/milvus/internal/util/hardware"
"github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus/internal/util/uniquegenerator" "github.com/milvus-io/milvus/internal/util/uniquegenerator"
) )
@ -34,7 +35,7 @@ import (
//getComponentConfigurations returns the configurations of indexCoord matching req.Pattern //getComponentConfigurations returns the configurations of indexCoord matching req.Pattern
func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse { func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse {
prefix := "indexcoord." prefix := "indexcoord."
matchedConfig := Params.IndexCoordCfg.Base.GetByPattern(prefix + req.Pattern) matchedConfig := Params.GetByPattern(prefix + req.Pattern)
configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig)) configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig))
for key, value := range matchedConfig { for key, value := range matchedConfig {
configList = append(configList, configList = append(configList,
@ -74,8 +75,8 @@ func getSystemInfoMetrics(
DiskUsage: hardware.GetDiskUsage(), DiskUsage: hardware.GetDiskUsage(),
}, },
SystemInfo: metricsinfo.DeployMetrics{}, SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.IndexCoordCfg.CreatedTime.String(), CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: Params.IndexCoordCfg.UpdatedTime.String(), UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.IndexCoordRole, Type: typeutil.IndexCoordRole,
ID: coord.session.ServerID, ID: coord.session.ServerID,
}, },

View File

@ -91,7 +91,7 @@ func (nm *NodeManager) AddNode(nodeID UniqueID, address string) error {
err error err error
) )
nodeClient, err = grpcindexnodeclient.NewClient(context.TODO(), address, Params.IndexCoordCfg.WithCredential) nodeClient, err = grpcindexnodeclient.NewClient(context.TODO(), address, Params.IndexCoordCfg.WithCredential.GetAsBool())
if err != nil { if err != nil {
log.Error("IndexCoord NodeManager", zap.Any("Add node err", err)) log.Error("IndexCoord NodeManager", zap.Any("Add node err", err))
return err return err

View File

@ -147,15 +147,15 @@ func (i *IndexNode) initKnowhere() {
C.free(unsafe.Pointer(cEasyloggingYaml)) C.free(unsafe.Pointer(cEasyloggingYaml))
// override index builder SIMD type // override index builder SIMD type
cSimdType := C.CString(Params.CommonCfg.SimdType) cSimdType := C.CString(Params.CommonCfg.SimdType.GetValue())
C.IndexBuilderSetSimdType(cSimdType) C.IndexBuilderSetSimdType(cSimdType)
C.free(unsafe.Pointer(cSimdType)) C.free(unsafe.Pointer(cSimdType))
// override segcore index slice size // override segcore index slice size
cIndexSliceSize := C.int64_t(Params.CommonCfg.IndexSliceSize) cIndexSliceSize := C.int64_t(Params.CommonCfg.IndexSliceSize.GetAsInt64())
C.InitIndexSliceSize(cIndexSliceSize) C.InitIndexSliceSize(cIndexSliceSize)
cThreadCoreCoefficient := C.int64_t(Params.CommonCfg.ThreadCoreCoefficient) cThreadCoreCoefficient := C.int64_t(Params.CommonCfg.ThreadCoreCoefficient.GetAsInt64())
C.InitThreadCoreCoefficient(cThreadCoreCoefficient) C.InitThreadCoreCoefficient(cThreadCoreCoefficient)
cCPUNum := C.int(hardware.GetCPUNum()) cCPUNum := C.int(hardware.GetCPUNum())
@ -210,9 +210,6 @@ func (i *IndexNode) Start() error {
i.once.Do(func() { i.once.Do(func() {
startErr = i.sched.Start() startErr = i.sched.Start()
Params.IndexNodeCfg.CreatedTime = time.Now()
Params.IndexNodeCfg.UpdatedTime = time.Now()
i.UpdateStateCode(commonpb.StateCode_Healthy) i.UpdateStateCode(commonpb.StateCode_Healthy)
log.Info("IndexNode", zap.Any("State", i.stateCode.Load())) log.Info("IndexNode", zap.Any("State", i.stateCode.Load()))
}) })

View File

@ -232,13 +232,13 @@ func getMockSystemInfoMetrics(
DiskUsage: hardware.GetDiskUsage(), DiskUsage: hardware.GetDiskUsage(),
}, },
SystemInfo: metricsinfo.DeployMetrics{}, SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.IndexNodeCfg.CreatedTime.String(), CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: Params.IndexNodeCfg.UpdatedTime.String(), UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.IndexNodeRole, Type: typeutil.IndexNodeRole,
}, },
SystemConfigurations: metricsinfo.IndexNodeConfiguration{ SystemConfigurations: metricsinfo.IndexNodeConfiguration{
MinioBucketName: Params.MinioCfg.BucketName.GetValue(), MinioBucketName: Params.MinioCfg.BucketName.GetValue(),
SimdType: Params.CommonCfg.SimdType, SimdType: Params.CommonCfg.SimdType.GetValue(),
}, },
} }

View File

@ -221,7 +221,7 @@ func (i *IndexNode) GetJobStats(ctx context.Context, req *indexpb.GetJobStatsReq
EnqueueJobNum: int64(unissued), EnqueueJobNum: int64(unissued),
TaskSlots: int64(slots), TaskSlots: int64(slots),
JobInfos: jobInfos, JobInfos: jobInfos,
EnableDisk: Params.IndexNodeCfg.EnableDisk, EnableDisk: Params.IndexNodeCfg.EnableDisk.GetAsBool(),
}, nil }, nil
} }

View File

@ -31,7 +31,7 @@ import (
//getComponentConfigurations returns the configurations of queryNode matching req.Pattern //getComponentConfigurations returns the configurations of queryNode matching req.Pattern
func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse { func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse {
prefix := "indexnode." prefix := "indexnode."
matchedConfig := Params.IndexNodeCfg.Base.GetByPattern(prefix + req.Pattern) matchedConfig := Params.GetByPattern(prefix + req.Pattern)
configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig)) configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig))
for key, value := range matchedConfig { for key, value := range matchedConfig {
configList = append(configList, configList = append(configList,
@ -70,14 +70,14 @@ func getSystemInfoMetrics(
DiskUsage: hardware.GetDiskUsage(), DiskUsage: hardware.GetDiskUsage(),
}, },
SystemInfo: metricsinfo.DeployMetrics{}, SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.IndexNodeCfg.CreatedTime.String(), CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: Params.IndexNodeCfg.UpdatedTime.String(), UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.IndexNodeRole, Type: typeutil.IndexNodeRole,
ID: node.session.ServerID, ID: node.session.ServerID,
}, },
SystemConfigurations: metricsinfo.IndexNodeConfiguration{ SystemConfigurations: metricsinfo.IndexNodeConfiguration{
MinioBucketName: Params.MinioCfg.BucketName.GetValue(), MinioBucketName: Params.MinioCfg.BucketName.GetValue(),
SimdType: Params.CommonCfg.SimdType, SimdType: Params.CommonCfg.SimdType.GetValue(),
}, },
} }

View File

@ -306,10 +306,10 @@ func (it *indexBuildTask) BuildIndex(ctx context.Context) error {
func (it *indexBuildTask) BuildDiskAnnIndex(ctx context.Context) error { func (it *indexBuildTask) BuildDiskAnnIndex(ctx context.Context) error {
// check index node support disk index // check index node support disk index
if !Params.IndexNodeCfg.EnableDisk { if !Params.IndexNodeCfg.EnableDisk.GetAsBool() {
log.Ctx(ctx).Error("IndexNode don't support build disk index", log.Ctx(ctx).Error("IndexNode don't support build disk index",
zap.String("index type", it.newIndexParams["index_type"]), zap.String("index type", it.newIndexParams["index_type"]),
zap.Bool("enable disk", Params.IndexNodeCfg.EnableDisk)) zap.Bool("enable disk", Params.IndexNodeCfg.EnableDisk.GetAsBool()))
return errors.New("index node don't support build disk index") return errors.New("index node don't support build disk index")
} }
@ -321,7 +321,7 @@ func (it *indexBuildTask) BuildDiskAnnIndex(ctx context.Context) error {
} }
usedLocalSizeWhenBuild := int64(float64(it.fieldData.GetMemorySize())*diskUsageRatio) + localUsedSize usedLocalSizeWhenBuild := int64(float64(it.fieldData.GetMemorySize())*diskUsageRatio) + localUsedSize
maxUsedLocalSize := int64(float64(Params.IndexNodeCfg.DiskCapacityLimit) * Params.IndexNodeCfg.MaxDiskUsagePercentage) maxUsedLocalSize := int64(Params.IndexNodeCfg.DiskCapacityLimit.GetAsFloat() * Params.IndexNodeCfg.MaxDiskUsagePercentage.GetAsFloat())
if usedLocalSizeWhenBuild > maxUsedLocalSize { if usedLocalSizeWhenBuild > maxUsedLocalSize {
log.Ctx(ctx).Error("IndexNode don't has enough disk size to build disk ann index", log.Ctx(ctx).Error("IndexNode don't has enough disk size to build disk ann index",

View File

@ -177,7 +177,7 @@ func NewTaskScheduler(ctx context.Context) (*TaskScheduler, error) {
s := &TaskScheduler{ s := &TaskScheduler{
ctx: ctx1, ctx: ctx1,
cancel: cancel, cancel: cancel,
buildParallel: Params.IndexNodeCfg.BuildParallel, buildParallel: Params.IndexNodeCfg.BuildParallel.GetAsInt(),
} }
s.IndexBuildQueue = NewIndexBuildTaskQueue(s) s.IndexBuildQueue = NewIndexBuildTaskQueue(s)

View File

@ -19,10 +19,11 @@ var (
func Connect(cfg *paramtable.MetaDBConfig) error { func Connect(cfg *paramtable.MetaDBConfig) error {
// load config // load config
dsn := fmt.Sprintf("%s:%s@tcp(%s:%d)/%s?charset=utf8mb4&parseTime=True&loc=Local", cfg.Username, cfg.Password, cfg.Address, cfg.Port, cfg.DBName) dsn := fmt.Sprintf("%s:%s@tcp(%s:%d)/%s?charset=utf8mb4&parseTime=True&loc=Local",
cfg.Username.GetValue(), cfg.Password.GetValue(), cfg.Address.GetValue(), cfg.Port.GetAsInt(), cfg.DBName.GetValue())
var ormLogger logger.Interface var ormLogger logger.Interface
if cfg.Base.Log.Level == "debug" { if cfg.LogLevel.GetValue() == "debug" {
ormLogger = logger.Default.LogMode(logger.Info) ormLogger = logger.Default.LogMode(logger.Info)
} else { } else {
ormLogger = logger.Default ormLogger = logger.Default
@ -33,21 +34,33 @@ func Connect(cfg *paramtable.MetaDBConfig) error {
CreateBatchSize: 100, CreateBatchSize: 100,
}) })
if err != nil { if err != nil {
log.Error("fail to connect db", zap.String("host", cfg.Address), zap.Int("port", cfg.Port), zap.String("database", cfg.DBName), zap.Error(err)) log.Error("fail to connect db",
zap.String("host", cfg.Address.GetValue()),
zap.Int("port", cfg.Port.GetAsInt()),
zap.String("database", cfg.DBName.GetValue()),
zap.Error(err))
return err return err
} }
idb, err := db.DB() idb, err := db.DB()
if err != nil { if err != nil {
log.Error("fail to create db instance", zap.String("host", cfg.Address), zap.Int("port", cfg.Port), zap.String("database", cfg.DBName), zap.Error(err)) log.Error("fail to create db instance",
zap.String("host", cfg.Address.GetValue()),
zap.Int("port", cfg.Port.GetAsInt()),
zap.String("database", cfg.DBName.GetValue()),
zap.Error(err))
return err return err
} }
idb.SetMaxIdleConns(cfg.MaxIdleConns) idb.SetMaxIdleConns(cfg.MaxIdleConns.GetAsInt())
idb.SetMaxOpenConns(cfg.MaxOpenConns) idb.SetMaxOpenConns(cfg.MaxOpenConns.GetAsInt())
globalDB = db globalDB = db
log.Info("db connected success", zap.String("host", cfg.Address), zap.Int("port", cfg.Port), zap.String("database", cfg.DBName)) log.Info("db connected success",
zap.String("host", cfg.Address.GetValue()),
zap.Int("port", cfg.Port.GetAsInt()),
zap.String("database", cfg.DBName.GetValue()),
zap.Error(err))
return nil return nil
} }

View File

@ -58,12 +58,12 @@ func SetupAccseeLog(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.Min
func InitAccessLogger(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) (*RotateLogger, error) { func InitAccessLogger(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) (*RotateLogger, error) {
var lg *RotateLogger var lg *RotateLogger
var err error var err error
if !logCfg.Enable { if !logCfg.Enable.GetAsBool() {
return nil, nil return nil, nil
} }
var writeSyncer zapcore.WriteSyncer var writeSyncer zapcore.WriteSyncer
if len(logCfg.Filename) > 0 { if len(logCfg.Filename.GetValue()) > 0 {
lg, err = NewRotateLogger(logCfg, minioCfg) lg, err = NewRotateLogger(logCfg, minioCfg)
if err != nil { if err != nil {
return nil, err return nil, err

View File

@ -39,7 +39,7 @@ func TestAccessLogger_NotEnable(t *testing.T) {
defer closer.Close() defer closer.Close()
Params.Init() Params.Init()
Params.ProxyCfg.AccessLog.Enable = false Params.Save(Params.ProxyCfg.AccessLog.Enable.Key, "false")
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -73,7 +73,7 @@ func TestAccessLogger_Basic(t *testing.T) {
Params.Init() Params.Init()
testPath := "/tmp/accesstest" testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
defer os.RemoveAll(testPath) defer os.RemoveAll(testPath)
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -107,7 +107,7 @@ func TestAccessLogger_Stdout(t *testing.T) {
defer closer.Close() defer closer.Close()
Params.Init() Params.Init()
Params.ProxyCfg.AccessLog.Filename = "" Params.Save(Params.ProxyCfg.AccessLog.Filename.Key, "")
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -140,9 +140,10 @@ func TestAccessLogger_WithMinio(t *testing.T) {
Params.Init() Params.Init()
testPath := "/tmp/accesstest" testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
Params.ProxyCfg.AccessLog.MinioEnable = true Params.Save(Params.ProxyCfg.AccessLog.MinioEnable.Key, "true")
Params.ProxyCfg.AccessLog.RemotePath = "access_log/" Params.Save(Params.ProxyCfg.AccessLog.RemotePath.Key, "access_log/")
Params.Save(Params.ProxyCfg.AccessLog.MaxSize.Key, "1")
defer os.RemoveAll(testPath) defer os.RemoveAll(testPath)
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -185,7 +186,7 @@ func TestAccessLogger_Error(t *testing.T) {
Params.Init() Params.Init()
testPath := "/tmp/accesstest" testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, "testPath")
defer os.RemoveAll(testPath) defer os.RemoveAll(testPath)
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)

View File

@ -29,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"go.uber.org/zap"
) )
const megabyte = 1024 * 1024 const megabyte = 1024 * 1024
@ -66,17 +67,19 @@ type RotateLogger struct {
func NewRotateLogger(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) (*RotateLogger, error) { func NewRotateLogger(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) (*RotateLogger, error) {
logger := &RotateLogger{ logger := &RotateLogger{
localPath: logCfg.LocalPath, localPath: logCfg.LocalPath.GetValue(),
fileName: logCfg.Filename, fileName: logCfg.Filename.GetValue(),
rotatedTime: logCfg.RotatedTime, rotatedTime: logCfg.RotatedTime.GetAsInt64(),
maxSize: logCfg.MaxSize, maxSize: logCfg.MaxSize.GetAsInt(),
maxBackups: logCfg.MaxBackups, maxBackups: logCfg.MaxBackups.GetAsInt(),
} }
log.Info("Access log save to " + logger.dir()) log.Info("Access log save to " + logger.dir())
if logCfg.MinioEnable { if logCfg.MinioEnable.GetAsBool() {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel() defer cancel()
handler, err := NewMinioHandler(ctx, minioCfg, logCfg.RemotePath, logCfg.MaxBackups) log.Debug("remtepath", zap.Any("remote", logCfg.RemotePath.GetValue()))
log.Debug("maxBackups", zap.Any("maxBackups", logCfg.MaxBackups.GetValue()))
handler, err := NewMinioHandler(ctx, minioCfg, logCfg.RemotePath.GetValue(), logCfg.MaxBackups.GetAsInt())
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -22,8 +22,10 @@ import (
"testing" "testing"
"time" "time"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"go.uber.org/zap"
) )
func getText(size int) []byte { func getText(size int) []byte {
@ -38,9 +40,9 @@ func TestRotateLogger_Basic(t *testing.T) {
var Params paramtable.ComponentParam var Params paramtable.ComponentParam
Params.Init() Params.Init()
testPath := "/tmp/accesstest" testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
Params.ProxyCfg.AccessLog.MinioEnable = true Params.Save(Params.ProxyCfg.AccessLog.MinioEnable.Key, "true")
Params.ProxyCfg.AccessLog.RemotePath = "access_log/" Params.Save(Params.ProxyCfg.AccessLog.RemotePath.Key, "access_log/")
defer os.RemoveAll(testPath) defer os.RemoveAll(testPath)
logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -67,12 +69,11 @@ func TestRotateLogger_TimeRotate(t *testing.T) {
var Params paramtable.ComponentParam var Params paramtable.ComponentParam
Params.Init() Params.Init()
testPath := "/tmp/accesstest" testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
Params.ProxyCfg.AccessLog.MinioEnable = true Params.Save(Params.ProxyCfg.AccessLog.MinioEnable.Key, "true")
Params.ProxyCfg.AccessLog.RemotePath = "access_log/" Params.Save(Params.ProxyCfg.AccessLog.RemotePath.Key, "access_log/")
Params.ProxyCfg.AccessLog.RotatedTime = 2 Params.Save(Params.ProxyCfg.AccessLog.RotatedTime.Key, "2")
//close file retention Params.Save(Params.ProxyCfg.AccessLog.MaxBackups.Key, "0")
Params.ProxyCfg.AccessLog.MaxBackups = 0
defer os.RemoveAll(testPath) defer os.RemoveAll(testPath)
logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -96,10 +97,10 @@ func TestRotateLogger_SizeRotate(t *testing.T) {
var Params paramtable.ComponentParam var Params paramtable.ComponentParam
Params.Init() Params.Init()
testPath := "/tmp/accesstest" testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
Params.ProxyCfg.AccessLog.MinioEnable = true Params.Save(Params.ProxyCfg.AccessLog.MinioEnable.Key, "true")
Params.ProxyCfg.AccessLog.RemotePath = "access_log/" Params.Save(Params.ProxyCfg.AccessLog.RemotePath.Key, "access_log/")
Params.ProxyCfg.AccessLog.MaxSize = 1 Params.Save(Params.ProxyCfg.AccessLog.MaxSize.Key, "1")
defer os.RemoveAll(testPath) defer os.RemoveAll(testPath)
logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -110,6 +111,7 @@ func TestRotateLogger_SizeRotate(t *testing.T) {
num := 1024 * 1024 num := 1024 * 1024
text := getText(num + 1) text := getText(num + 1)
_, err = logger.Write(text) _, err = logger.Write(text)
log.Error("write failed", zap.Error(err))
assert.Error(t, err) assert.Error(t, err)
for i := 1; i <= 2; i++ { for i := 1; i <= 2; i++ {
@ -129,8 +131,8 @@ func TestRotateLogger_LocalRetention(t *testing.T) {
var Params paramtable.ComponentParam var Params paramtable.ComponentParam
Params.Init() Params.Init()
testPath := "/tmp/accesstest" testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
Params.ProxyCfg.AccessLog.MaxBackups = 1 Params.Save(Params.ProxyCfg.AccessLog.MaxBackups.Key, "1")
defer os.RemoveAll(testPath) defer os.RemoveAll(testPath)
logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -150,7 +152,7 @@ func TestRotateLogger_BasicError(t *testing.T) {
var Params paramtable.ComponentParam var Params paramtable.ComponentParam
Params.Init() Params.Init()
testPath := "" testPath := ""
Params.ProxyCfg.AccessLog.LocalPath = testPath Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg) logger, err := NewRotateLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
assert.NoError(t, err) assert.NoError(t, err)
@ -174,8 +176,8 @@ func TestRotateLogger_InitError(t *testing.T) {
var params paramtable.ComponentParam var params paramtable.ComponentParam
params.Init() params.Init()
testPath := "" testPath := ""
params.ProxyCfg.AccessLog.LocalPath = testPath params.Save(params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
params.ProxyCfg.AccessLog.MinioEnable = true params.Save(params.ProxyCfg.AccessLog.MinioEnable.Key, "true")
params.Save(params.MinioCfg.Address.Key, "") params.Save(params.MinioCfg.Address.Key, "")
//init err with invalid minio address //init err with invalid minio address
_, err := NewRotateLogger(&params.ProxyCfg.AccessLog, &params.MinioCfg) _, err := NewRotateLogger(&params.ProxyCfg.AccessLog, &params.MinioCfg)

View File

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

View File

@ -59,7 +59,7 @@ func AuthenticationInterceptor(ctx context.Context) (context.Context, error) {
// check: // check:
// 1. if rpc call from a member (like index/query/data component) // 1. if rpc call from a member (like index/query/data component)
// 2. if rpc call from sdk // 2. if rpc call from sdk
if Params.CommonCfg.AuthorizationEnabled { if Params.CommonCfg.AuthorizationEnabled.GetAsBool() {
if !validSourceID(ctx, md[strings.ToLower(util.HeaderSourceID)]) && if !validSourceID(ctx, md[strings.ToLower(util.HeaderSourceID)]) &&
!validAuth(ctx, md[strings.ToLower(util.HeaderAuthorize)]) { !validAuth(ctx, md[strings.ToLower(util.HeaderAuthorize)]) {
return nil, ErrUnauthenticated() return nil, ErrUnauthenticated()

View File

@ -9,6 +9,7 @@ import (
"github.com/milvus-io/milvus/internal/util" "github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/util/crypto" "github.com/milvus-io/milvus/internal/util/crypto"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -46,7 +47,8 @@ func TestValidSourceID(t *testing.T) {
func TestAuthenticationInterceptor(t *testing.T) { func TestAuthenticationInterceptor(t *testing.T) {
ctx := context.Background() ctx := context.Background()
Params.CommonCfg.AuthorizationEnabled = true // mock authorization is turned on paramtable.Get().Save(Params.CommonCfg.AuthorizationEnabled.Key, "true") // mock authorization is turned on
defer paramtable.Get().Reset(Params.CommonCfg.AuthorizationEnabled.Key) // mock authorization is turned on
// no metadata // no metadata
_, err := AuthenticationInterceptor(ctx) _, err := AuthenticationInterceptor(ctx)
assert.NotNil(t, err) assert.NotNil(t, err)

View File

@ -118,7 +118,7 @@ func (ticker *channelsTimeTickerImpl) tick() error {
} else { } else {
if stat.minTs > current { if stat.minTs > current {
ticker.minTsStatistics[pchan] = stat.minTs - 1 ticker.minTsStatistics[pchan] = stat.minTs - 1
next := now + Timestamp(Params.ProxyCfg.TimeTickInterval) next := now + Timestamp(Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond))
if next > stat.maxTs { if next > stat.maxTs {
next = stat.maxTs next = stat.maxTs
} }

View File

@ -36,7 +36,7 @@ func (d defaultHook) Release() {}
var hoo hook.Hook var hoo hook.Hook
func initHook() error { func initHook() error {
path := Params.ProxyCfg.SoPath path := Params.ProxyCfg.SoPath.GetValue()
if path == "" { if path == "" {
hoo = defaultHook{} hoo = defaultHook{}
return nil return nil
@ -59,7 +59,7 @@ func initHook() error {
if !ok { if !ok {
return fmt.Errorf("fail to convert the `Hook` interface") return fmt.Errorf("fail to convert the `Hook` interface")
} }
if err = hoo.Init(Params.HookCfg.SoConfig); err != nil { if err = hoo.Init(Params.HookCfg.SoConfig.GetValue()); err != nil {
return fmt.Errorf("fail to init configs for the hook, error: %s", err.Error()) return fmt.Errorf("fail to init configs for the hook, error: %s", err.Error())
} }
return nil return nil
@ -67,7 +67,7 @@ func initHook() error {
func UnaryServerHookInterceptor() grpc.UnaryServerInterceptor { func UnaryServerHookInterceptor() grpc.UnaryServerInterceptor {
if hookError := initHook(); hookError != nil { if hookError := initHook(); hookError != nil {
logger.Error("hook error", zap.String("path", Params.ProxyCfg.SoPath), zap.Error(hookError)) logger.Error("hook error", zap.String("path", Params.ProxyCfg.SoPath.GetValue()), zap.Error(hookError))
hoo = defaultHook{} hoo = defaultHook{}
} }
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {

View File

@ -7,18 +7,19 @@ import (
"google.golang.org/grpc" "google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
func TestInitHook(t *testing.T) { func TestInitHook(t *testing.T) {
Params.ProxyCfg.SoPath = "" paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "")
initHook() initHook()
assert.IsType(t, defaultHook{}, hoo) assert.IsType(t, defaultHook{}, hoo)
Params.ProxyCfg.SoPath = "/a/b/hook.so" paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "/a/b/hook.so")
err := initHook() err := initHook()
assert.NotNil(t, err) assert.NotNil(t, err)
Params.ProxyCfg.SoPath = "" paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "")
} }
type mockHook struct { type mockHook struct {

View File

@ -2008,7 +2008,7 @@ func (node *Proxy) Insert(ctx context.Context, request *milvuspb.InsertRequest)
} }
if len(it.PartitionName) <= 0 { if len(it.PartitionName) <= 0 {
it.PartitionName = Params.CommonCfg.DefaultPartitionName it.PartitionName = Params.CommonCfg.DefaultPartitionName.GetValue()
} }
constructFailedResponse := func(err error) *milvuspb.MutationResult { constructFailedResponse := func(err error) *milvuspb.MutationResult {

View File

@ -88,14 +88,14 @@ func getProxyMetrics(ctx context.Context, request *milvuspb.GetMetricsRequest, n
Name: proxyRoleName, Name: proxyRoleName,
HardwareInfos: hardwareMetrics, HardwareInfos: hardwareMetrics,
SystemInfo: metricsinfo.DeployMetrics{}, SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.ProxyCfg.CreatedTime.String(), CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: Params.ProxyCfg.UpdatedTime.String(), UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.ProxyRole, Type: typeutil.ProxyRole,
ID: node.session.ServerID, ID: node.session.ServerID,
}, },
SystemConfigurations: metricsinfo.ProxyConfiguration{ SystemConfigurations: metricsinfo.ProxyConfiguration{
DefaultPartitionName: Params.CommonCfg.DefaultPartitionName, DefaultPartitionName: Params.CommonCfg.DefaultPartitionName.GetValue(),
DefaultIndexName: Params.CommonCfg.DefaultIndexName, DefaultIndexName: Params.CommonCfg.DefaultIndexName.GetValue(),
}, },
QuotaMetrics: quotaMetrics, QuotaMetrics: quotaMetrics,
} }
@ -148,14 +148,14 @@ func getSystemInfoMetrics(
DiskUsage: hardware.GetDiskUsage(), DiskUsage: hardware.GetDiskUsage(),
}, },
SystemInfo: metricsinfo.DeployMetrics{}, SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.ProxyCfg.CreatedTime.String(), CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: Params.ProxyCfg.UpdatedTime.String(), UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.ProxyRole, Type: typeutil.ProxyRole,
ID: node.session.ServerID, ID: node.session.ServerID,
}, },
SystemConfigurations: metricsinfo.ProxyConfiguration{ SystemConfigurations: metricsinfo.ProxyConfiguration{
DefaultPartitionName: Params.CommonCfg.DefaultPartitionName, DefaultPartitionName: Params.CommonCfg.DefaultPartitionName.GetValue(),
DefaultIndexName: Params.CommonCfg.DefaultIndexName, DefaultIndexName: Params.CommonCfg.DefaultIndexName.GetValue(),
}, },
}, },
} }

View File

@ -46,7 +46,7 @@ func NewMultiRateLimiter() *MultiRateLimiter {
// Limit returns true, the request will be rejected. // Limit returns true, the request will be rejected.
// Otherwise, the request will pass. Limit also returns limit of limiter. // Otherwise, the request will pass. Limit also returns limit of limiter.
func (m *MultiRateLimiter) Limit(rt internalpb.RateType, n int) (bool, float64) { func (m *MultiRateLimiter) Limit(rt internalpb.RateType, n int) (bool, float64) {
if !Params.QuotaConfig.QuotaAndLimitsEnabled { if !Params.QuotaConfig.QuotaAndLimitsEnabled.GetAsBool() {
return false, 1 // no limit return false, 1 // no limit
} }
// TODO: call other rate limiters // TODO: call other rate limiters
@ -103,25 +103,25 @@ func (rl *rateLimiter) registerLimiters() {
var r float64 var r float64
switch internalpb.RateType(rt) { switch internalpb.RateType(rt) {
case internalpb.RateType_DDLCollection: case internalpb.RateType_DDLCollection:
r = Params.QuotaConfig.DDLCollectionRate r = Params.QuotaConfig.DDLCollectionRate.GetAsFloat()
case internalpb.RateType_DDLPartition: case internalpb.RateType_DDLPartition:
r = Params.QuotaConfig.DDLPartitionRate r = Params.QuotaConfig.DDLPartitionRate.GetAsFloat()
case internalpb.RateType_DDLIndex: case internalpb.RateType_DDLIndex:
r = Params.QuotaConfig.MaxIndexRate r = Params.QuotaConfig.MaxIndexRate.GetAsFloat()
case internalpb.RateType_DDLFlush: case internalpb.RateType_DDLFlush:
r = Params.QuotaConfig.MaxFlushRate r = Params.QuotaConfig.MaxFlushRate.GetAsFloat()
case internalpb.RateType_DDLCompaction: case internalpb.RateType_DDLCompaction:
r = Params.QuotaConfig.MaxCompactionRate r = Params.QuotaConfig.MaxCompactionRate.GetAsFloat()
case internalpb.RateType_DMLInsert: case internalpb.RateType_DMLInsert:
r = Params.QuotaConfig.DMLMaxInsertRate r = Params.QuotaConfig.DMLMaxInsertRate.GetAsFloat()
case internalpb.RateType_DMLDelete: case internalpb.RateType_DMLDelete:
r = Params.QuotaConfig.DMLMaxDeleteRate r = Params.QuotaConfig.DMLMaxDeleteRate.GetAsFloat()
case internalpb.RateType_DMLBulkLoad: case internalpb.RateType_DMLBulkLoad:
r = Params.QuotaConfig.DMLMaxBulkLoadRate r = Params.QuotaConfig.DMLMaxBulkLoadRate.GetAsFloat()
case internalpb.RateType_DQLSearch: case internalpb.RateType_DQLSearch:
r = Params.QuotaConfig.DQLMaxSearchRate r = Params.QuotaConfig.DQLMaxSearchRate.GetAsFloat()
case internalpb.RateType_DQLQuery: case internalpb.RateType_DQLQuery:
r = Params.QuotaConfig.DQLMaxQueryRate r = Params.QuotaConfig.DQLMaxQueryRate.GetAsFloat()
} }
limit := ratelimitutil.Limit(r) limit := ratelimitutil.Limit(r)
burst := r // use rate as burst, because Limiter is with punishment mechanism, burst is insignificant. burst := r // use rate as burst, because Limiter is with punishment mechanism, burst is insignificant.

View File

@ -17,10 +17,12 @@
package proxy package proxy
import ( import (
"fmt"
"math" "math"
"testing" "testing"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/ratelimitutil" "github.com/milvus-io/milvus/internal/util/ratelimitutil"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -28,7 +30,7 @@ import (
func TestMultiRateLimiter(t *testing.T) { func TestMultiRateLimiter(t *testing.T) {
t.Run("test multiRateLimiter", func(t *testing.T) { t.Run("test multiRateLimiter", func(t *testing.T) {
bak := Params.QuotaConfig.QuotaAndLimitsEnabled bak := Params.QuotaConfig.QuotaAndLimitsEnabled
Params.QuotaConfig.QuotaAndLimitsEnabled = true paramtable.Get().Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, "true")
multiLimiter := NewMultiRateLimiter() multiLimiter := NewMultiRateLimiter()
for _, rt := range internalpb.RateType_value { for _, rt := range internalpb.RateType_value {
multiLimiter.globalRateLimiter.limiters[internalpb.RateType(rt)] = ratelimitutil.NewLimiter(ratelimitutil.Limit(1000), 1) multiLimiter.globalRateLimiter.limiters[internalpb.RateType(rt)] = ratelimitutil.NewLimiter(ratelimitutil.Limit(1000), 1)
@ -47,7 +49,7 @@ func TestMultiRateLimiter(t *testing.T) {
t.Run("not enable quotaAndLimit", func(t *testing.T) { t.Run("not enable quotaAndLimit", func(t *testing.T) {
multiLimiter := NewMultiRateLimiter() multiLimiter := NewMultiRateLimiter()
bak := Params.QuotaConfig.QuotaAndLimitsEnabled bak := Params.QuotaConfig.QuotaAndLimitsEnabled
Params.QuotaConfig.QuotaAndLimitsEnabled = false paramtable.Get().Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, "false")
for _, rt := range internalpb.RateType_value { for _, rt := range internalpb.RateType_value {
ok, r := multiLimiter.Limit(internalpb.RateType(rt), 1) ok, r := multiLimiter.Limit(internalpb.RateType(rt), 1)
assert.False(t, ok) assert.False(t, ok)
@ -59,10 +61,10 @@ func TestMultiRateLimiter(t *testing.T) {
t.Run("test limit", func(t *testing.T) { t.Run("test limit", func(t *testing.T) {
run := func(insertRate float64) { run := func(insertRate float64) {
bakInsertRate := Params.QuotaConfig.DMLMaxInsertRate bakInsertRate := Params.QuotaConfig.DMLMaxInsertRate
Params.QuotaConfig.DMLMaxInsertRate = insertRate paramtable.Get().Save(Params.QuotaConfig.DMLMaxInsertRate.Key, fmt.Sprintf("%f", insertRate))
multiLimiter := NewMultiRateLimiter() multiLimiter := NewMultiRateLimiter()
bak := Params.QuotaConfig.QuotaAndLimitsEnabled bak := Params.QuotaConfig.QuotaAndLimitsEnabled
Params.QuotaConfig.QuotaAndLimitsEnabled = true paramtable.Get().Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, "true")
ok, r := multiLimiter.Limit(internalpb.RateType_DMLInsert, 1*1024*1024) ok, r := multiLimiter.Limit(internalpb.RateType_DMLInsert, 1*1024*1024)
assert.False(t, ok) assert.False(t, ok)
assert.NotEqual(t, float64(0), r) assert.NotEqual(t, float64(0), r)

View File

@ -72,7 +72,7 @@ func UnaryServerInterceptor(privilegeFunc PrivilegeFunc) grpc.UnaryServerInterce
} }
func PrivilegeInterceptor(ctx context.Context, req interface{}) (context.Context, error) { func PrivilegeInterceptor(ctx context.Context, req interface{}) (context.Context, error) {
if !Params.CommonCfg.AuthorizationEnabled { if !Params.CommonCfg.AuthorizationEnabled.GetAsBool() {
return ctx, nil return ctx, nil
} }
log.Debug("PrivilegeInterceptor", zap.String("type", reflect.TypeOf(req).String())) log.Debug("PrivilegeInterceptor", zap.String("type", reflect.TypeOf(req).String()))

View File

@ -5,6 +5,7 @@ import (
"testing" "testing"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus-proto/go-api/commonpb" "github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb" "github.com/milvus-io/milvus-proto/go-api/milvuspb"
@ -20,7 +21,7 @@ func TestUnaryServerInterceptor(t *testing.T) {
func TestPrivilegeInterceptor(t *testing.T) { func TestPrivilegeInterceptor(t *testing.T) {
ctx := context.Background() ctx := context.Background()
t.Run("Authorization Disabled", func(t *testing.T) { t.Run("Authorization Disabled", func(t *testing.T) {
Params.CommonCfg.AuthorizationEnabled = false paramtable.Get().Save(Params.CommonCfg.AuthorizationEnabled.Key, "false")
_, err := PrivilegeInterceptor(ctx, &milvuspb.LoadCollectionRequest{ _, err := PrivilegeInterceptor(ctx, &milvuspb.LoadCollectionRequest{
DbName: "db_test", DbName: "db_test",
CollectionName: "col1", CollectionName: "col1",
@ -29,7 +30,7 @@ func TestPrivilegeInterceptor(t *testing.T) {
}) })
t.Run("Authorization Enabled", func(t *testing.T) { t.Run("Authorization Enabled", func(t *testing.T) {
Params.CommonCfg.AuthorizationEnabled = true paramtable.Get().Save(Params.CommonCfg.AuthorizationEnabled.Key, "true")
_, err := PrivilegeInterceptor(ctx, &milvuspb.HasCollectionRequest{}) _, err := PrivilegeInterceptor(ctx, &milvuspb.HasCollectionRequest{})
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -243,10 +243,10 @@ func (node *Proxy) Init() error {
} }
log.Debug("create task scheduler done", zap.String("role", typeutil.ProxyRole)) log.Debug("create task scheduler done", zap.String("role", typeutil.ProxyRole))
syncTimeTickInterval := Params.ProxyCfg.TimeTickInterval / 2 syncTimeTickInterval := Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond) / 2
log.Debug("create channels time ticker", log.Debug("create channels time ticker",
zap.String("role", typeutil.ProxyRole), zap.Duration("syncTimeTickInterval", syncTimeTickInterval)) zap.String("role", typeutil.ProxyRole), zap.Duration("syncTimeTickInterval", syncTimeTickInterval))
node.chTicker = newChannelsTimeTicker(node.ctx, Params.ProxyCfg.TimeTickInterval/2, []string{}, node.sched.getPChanStatistics, tsoAllocator) node.chTicker = newChannelsTimeTicker(node.ctx, Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond)/2, []string{}, node.sched.getPChanStatistics, tsoAllocator)
log.Debug("create channels time ticker done", zap.String("role", typeutil.ProxyRole)) log.Debug("create channels time ticker done", zap.String("role", typeutil.ProxyRole))
log.Debug("create metrics cache manager", zap.String("role", typeutil.ProxyRole)) log.Debug("create metrics cache manager", zap.String("role", typeutil.ProxyRole))
@ -269,7 +269,7 @@ func (node *Proxy) sendChannelsTimeTickLoop() {
go func() { go func() {
defer node.wg.Done() defer node.wg.Done()
timer := time.NewTicker(Params.ProxyCfg.TimeTickInterval) timer := time.NewTicker(Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond))
for { for {
select { select {
@ -365,10 +365,6 @@ func (node *Proxy) Start() error {
cb() cb()
} }
now := time.Now()
Params.ProxyCfg.CreatedTime = now
Params.ProxyCfg.UpdatedTime = now
log.Debug("update state code", zap.String("role", typeutil.ProxyRole), zap.String("State", commonpb.StateCode_Healthy.String())) log.Debug("update state code", zap.String("role", typeutil.ProxyRole), zap.String("State", commonpb.StateCode_Healthy.String()))
node.UpdateStateCode(commonpb.StateCode_Healthy) node.UpdateStateCode(commonpb.StateCode_Healthy)

View File

@ -159,12 +159,12 @@ func (cct *createCollectionTask) PreExecute(ctx context.Context) error {
} }
cct.schema.AutoID = false cct.schema.AutoID = false
if cct.ShardsNum > Params.ProxyCfg.MaxShardNum { if cct.ShardsNum > Params.ProxyCfg.MaxShardNum.GetAsInt32() {
return fmt.Errorf("maximum shards's number should be limited to %d", Params.ProxyCfg.MaxShardNum) return fmt.Errorf("maximum shards's number should be limited to %d", Params.ProxyCfg.MaxShardNum.GetAsInt())
} }
if int64(len(cct.schema.Fields)) > Params.ProxyCfg.MaxFieldNum { if len(cct.schema.Fields) > Params.ProxyCfg.MaxFieldNum.GetAsInt() {
return fmt.Errorf("maximum field's number should be limited to %d", Params.ProxyCfg.MaxFieldNum) return fmt.Errorf("maximum field's number should be limited to %d", Params.ProxyCfg.MaxFieldNum.GetAsInt())
} }
// validate collection name // validate collection name

View File

@ -116,7 +116,7 @@ func (cit *createIndexTask) parseIndexParams() error {
for _, kv := range cit.req.GetExtraParams() { for _, kv := range cit.req.GetExtraParams() {
if kv.Key == common.IndexParamsKey { if kv.Key == common.IndexParamsKey {
params, err := funcutil.ParseIndexParamsMap(kv.Value) params, err := funcutil.JSONToMap(kv.Value)
if err != nil { if err != nil {
return err return err
} }
@ -130,16 +130,16 @@ func (cit *createIndexTask) parseIndexParams() error {
if isVecIndex { if isVecIndex {
specifyIndexType, exist := indexParamsMap[common.IndexTypeKey] specifyIndexType, exist := indexParamsMap[common.IndexTypeKey]
if Params.AutoIndexConfig.Enable { if Params.AutoIndexConfig.Enable.GetAsBool() {
if exist { if exist {
if specifyIndexType != AutoIndexName { if specifyIndexType != AutoIndexName {
return fmt.Errorf("IndexType should be %s", AutoIndexName) return fmt.Errorf("IndexType should be %s", AutoIndexName)
} }
} }
log.Debug("create index trigger AutoIndex", log.Debug("create index trigger AutoIndex",
zap.String("type", Params.AutoIndexConfig.AutoIndexTypeName)) zap.String("type", Params.AutoIndexConfig.AutoIndexTypeName.GetValue()))
// override params // override params
for k, v := range Params.AutoIndexConfig.IndexParams { for k, v := range Params.AutoIndexConfig.IndexParams.GetAsJSONMap() {
indexParamsMap[k] = v indexParamsMap[k] = v
} }
} else { } else {
@ -307,7 +307,7 @@ func (cit *createIndexTask) Execute(ctx context.Context) error {
zap.Any("indexParams", cit.req.GetExtraParams())) zap.Any("indexParams", cit.req.GetExtraParams()))
if cit.req.GetIndexName() == "" { if cit.req.GetIndexName() == "" {
cit.req.IndexName = Params.CommonCfg.DefaultIndexName + "_" + strconv.FormatInt(cit.fieldSchema.GetFieldID(), 10) cit.req.IndexName = Params.CommonCfg.DefaultIndexName.GetValue() + "_" + strconv.FormatInt(cit.fieldSchema.GetFieldID(), 10)
} }
var err error var err error
req := &indexpb.CreateIndexRequest{ req := &indexpb.CreateIndexRequest{
@ -620,7 +620,7 @@ func (gibpt *getIndexBuildProgressTask) Execute(ctx context.Context) error {
gibpt.collectionID = collectionID gibpt.collectionID = collectionID
if gibpt.IndexName == "" { if gibpt.IndexName == "" {
gibpt.IndexName = Params.CommonCfg.DefaultIndexName gibpt.IndexName = Params.CommonCfg.DefaultIndexName.GetValue()
} }
resp, err := gibpt.indexCoord.GetIndexBuildProgress(ctx, &indexpb.GetIndexBuildProgressRequest{ resp, err := gibpt.indexCoord.GetIndexBuildProgress(ctx, &indexpb.GetIndexBuildProgressRequest{
@ -707,7 +707,7 @@ func (gist *getIndexStateTask) PreExecute(ctx context.Context) error {
func (gist *getIndexStateTask) Execute(ctx context.Context) error { func (gist *getIndexStateTask) Execute(ctx context.Context) error {
if gist.IndexName == "" { if gist.IndexName == "" {
gist.IndexName = Params.CommonCfg.DefaultIndexName gist.IndexName = Params.CommonCfg.DefaultIndexName.GetValue()
} }
collectionID, err := globalMetaCache.GetCollectionID(ctx, gist.CollectionName) collectionID, err := globalMetaCache.GetCollectionID(ctx, gist.CollectionName)
if err != nil { if err != nil {

View File

@ -433,7 +433,7 @@ func (it *insertTask) Execute(ctx context.Context) error {
return err return err
} }
} else { } else {
partitionID, err = globalMetaCache.GetPartitionID(ctx, collectionName, Params.CommonCfg.DefaultPartitionName) partitionID, err = globalMetaCache.GetPartitionID(ctx, collectionName, Params.CommonCfg.DefaultPartitionName.GetValue())
if err != nil { if err != nil {
return err return err
} }

View File

@ -201,8 +201,8 @@ func newBaseTaskQueue(tsoAllocatorIns tsoAllocator) *baseTaskQueue {
activeTasks: make(map[UniqueID]task), activeTasks: make(map[UniqueID]task),
utLock: sync.RWMutex{}, utLock: sync.RWMutex{},
atLock: sync.RWMutex{}, atLock: sync.RWMutex{},
maxTaskNum: Params.ProxyCfg.MaxTaskNum, maxTaskNum: Params.ProxyCfg.MaxTaskNum.GetAsInt64(),
utBufChan: make(chan int, Params.ProxyCfg.MaxTaskNum), utBufChan: make(chan int, Params.ProxyCfg.MaxTaskNum.GetAsInt()),
tsoAllocatorIns: tsoAllocatorIns, tsoAllocatorIns: tsoAllocatorIns,
} }
} }

View File

@ -18,6 +18,7 @@ import (
"github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/autoindex"
"github.com/milvus-io/milvus/internal/util/commonpbutil" "github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/distance" "github.com/milvus-io/milvus/internal/util/distance"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
@ -100,7 +101,7 @@ func getPartitionIDs(ctx context.Context, collectionName string, partitionNames
func parseSearchParams(searchParamsPair []*commonpb.KeyValuePair) (string, error) { func parseSearchParams(searchParamsPair []*commonpb.KeyValuePair) (string, error) {
searchParamStr, err := funcutil.GetAttrByKeyFromRepeatedKV(SearchParamsKey, searchParamsPair) searchParamStr, err := funcutil.GetAttrByKeyFromRepeatedKV(SearchParamsKey, searchParamsPair)
if Params.AutoIndexConfig.Enable { if Params.AutoIndexConfig.Enable.GetAsBool() {
searchParamMap := make(map[string]interface{}) searchParamMap := make(map[string]interface{})
var level int var level int
if err == nil { // if specified params, we try to parse params if err == nil { // if specified params, we try to parse params
@ -129,7 +130,8 @@ func parseSearchParams(searchParamsPair []*commonpb.KeyValuePair) (string, error
} else { } else {
level = 1 level = 1
} }
calculator := Params.AutoIndexConfig.GetSearchParamStrCalculator(level) paramsStr := Params.AutoIndexConfig.SearchParamsYamlStr.GetValue()
calculator := autoindex.GetSearchCalculator(paramsStr, level)
if calculator == nil { if calculator == nil {
return "", fmt.Errorf("search params calculator not found for level:%d", level) return "", fmt.Errorf("search params calculator not found for level:%d", level)
} }

View File

@ -11,7 +11,6 @@ import (
"time" "time"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/util/autoindex"
"github.com/milvus-io/milvus/internal/util/indexparamcheck" "github.com/milvus-io/milvus/internal/util/indexparamcheck"
"github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -1897,13 +1896,17 @@ func TestTaskSearch_parseSearchParams_AutoIndexEnable(t *testing.T) {
oldIndexType := Params.AutoIndexConfig.IndexType oldIndexType := Params.AutoIndexConfig.IndexType
oldIndexParams := Params.AutoIndexConfig.IndexParams oldIndexParams := Params.AutoIndexConfig.IndexParams
oldSearchParamYamStr := Params.AutoIndexConfig.SearchParamsYamlStr oldSearchParamYamStr := Params.AutoIndexConfig.SearchParamsYamlStr
oldParser := Params.AutoIndexConfig.Parser
//parseSearchParams //parseSearchParams
Params.AutoIndexConfig.Enable = true paramtable.Get().Save(Params.AutoIndexConfig.Enable.Key, "true")
Params.AutoIndexConfig.IndexType = indexparamcheck.IndexHNSW paramtable.Get().Save(Params.AutoIndexConfig.IndexType.Key, indexparamcheck.IndexHNSW)
Params.AutoIndexConfig.IndexParams = make(map[string]string) paramtable.Get().Save(Params.AutoIndexConfig.IndexParams.Key, "{}")
defer func() {
paramtable.Get().Reset(Params.AutoIndexConfig.Enable.Key)
paramtable.Get().Reset(Params.AutoIndexConfig.IndexType.Key)
paramtable.Get().Reset(Params.AutoIndexConfig.IndexParams.Key)
}()
buildParams := map[string]interface{}{ buildParams := map[string]any{
common.MetricTypeKey: indexparamcheck.L2, common.MetricTypeKey: indexparamcheck.L2,
common.IndexTypeKey: indexparamcheck.IndexHNSW, common.IndexTypeKey: indexparamcheck.IndexHNSW,
"M": 8, "M": 8,
@ -1911,7 +1914,8 @@ func TestTaskSearch_parseSearchParams_AutoIndexEnable(t *testing.T) {
} }
buildParamsJSONValue, err := json.Marshal(buildParams) buildParamsJSONValue, err := json.Marshal(buildParams)
assert.NoError(t, err) assert.NoError(t, err)
Params.AutoIndexConfig.IndexParams, err = funcutil.ParseIndexParamsMap(string(buildParamsJSONValue)) paramtable.Get().Save(Params.AutoIndexConfig.IndexParams.Key, string(buildParamsJSONValue))
defer paramtable.Get().Reset(Params.AutoIndexConfig.IndexParams.Key)
assert.NoError(t, err) assert.NoError(t, err)
jsonStr := ` jsonStr := `
@ -1941,8 +1945,8 @@ func TestTaskSearch_parseSearchParams_AutoIndexEnable(t *testing.T) {
] ]
} }
}` }`
Params.AutoIndexConfig.Parser = autoindex.NewParser() paramtable.Get().Save(Params.AutoIndexConfig.SearchParamsYamlStr.Key, jsonStr)
Params.AutoIndexConfig.Parser.InitFromJSONStr(jsonStr) defer paramtable.Get().Reset(Params.AutoIndexConfig.SearchParamsYamlStr.Key)
normalKVPairs := []*commonpb.KeyValuePair{ normalKVPairs := []*commonpb.KeyValuePair{
{ {
@ -2085,7 +2089,6 @@ func TestTaskSearch_parseSearchParams_AutoIndexEnable(t *testing.T) {
Params.AutoIndexConfig.IndexType = oldIndexType Params.AutoIndexConfig.IndexType = oldIndexType
Params.AutoIndexConfig.IndexParams = oldIndexParams Params.AutoIndexConfig.IndexParams = oldIndexParams
Params.AutoIndexConfig.SearchParamsYamlStr = oldSearchParamYamStr Params.AutoIndexConfig.SearchParamsYamlStr = oldSearchParamYamStr
Params.AutoIndexConfig.Parser = oldParser
} }

View File

@ -570,7 +570,7 @@ func TestCreateCollectionTask(t *testing.T) {
assert.Error(t, err) assert.Error(t, err)
task.Schema = marshaledSchema task.Schema = marshaledSchema
task.ShardsNum = Params.ProxyCfg.MaxShardNum + 1 task.ShardsNum = Params.ProxyCfg.MaxShardNum.GetAsInt32() + 1
err = task.PreExecute(ctx) err = task.PreExecute(ctx)
assert.Error(t, err) assert.Error(t, err)
task.ShardsNum = shardsNum task.ShardsNum = shardsNum
@ -582,7 +582,7 @@ func TestCreateCollectionTask(t *testing.T) {
Name: collectionName, Name: collectionName,
Description: "", Description: "",
AutoID: false, AutoID: false,
Fields: make([]*schemapb.FieldSchema, Params.ProxyCfg.MaxFieldNum+1), Fields: make([]*schemapb.FieldSchema, Params.ProxyCfg.MaxFieldNum.GetAsInt32()+1),
} }
marshaledSchemaWithTooManyFields, err := proto.Marshal(schemaWithTooManyFields) marshaledSchemaWithTooManyFields, err := proto.Marshal(schemaWithTooManyFields)
assert.NoError(t, err) assert.NoError(t, err)
@ -602,7 +602,7 @@ func TestCreateCollectionTask(t *testing.T) {
assert.Error(t, err) assert.Error(t, err)
schema.Name = prefix schema.Name = prefix
for i := 0; i < int(Params.ProxyCfg.MaxNameLength); i++ { for i := 0; i < Params.ProxyCfg.MaxNameLength.GetAsInt(); i++ {
schema.Name += strconv.Itoa(i % 10) schema.Name += strconv.Itoa(i % 10)
} }
tooLongNameSchema, err := proto.Marshal(schema) tooLongNameSchema, err := proto.Marshal(schema)
@ -701,7 +701,7 @@ func TestCreateCollectionTask(t *testing.T) {
schema.Fields[idx].TypeParams = []*commonpb.KeyValuePair{ schema.Fields[idx].TypeParams = []*commonpb.KeyValuePair{
{ {
Key: "dim", Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension) + 1), Value: strconv.Itoa(Params.ProxyCfg.MaxDimension.GetAsInt() + 1),
}, },
} }
} }
@ -717,7 +717,7 @@ func TestCreateCollectionTask(t *testing.T) {
schema.Fields[1].TypeParams = []*commonpb.KeyValuePair{ schema.Fields[1].TypeParams = []*commonpb.KeyValuePair{
{ {
Key: "dim", Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension) + 1), Value: strconv.Itoa(Params.ProxyCfg.MaxDimension.GetAsInt() + 1),
}, },
} }
binaryTooLargeDimSchema, err := proto.Marshal(schema) binaryTooLargeDimSchema, err := proto.Marshal(schema)

View File

@ -99,9 +99,8 @@ func validateCollectionNameOrAlias(entity, entityType string) error {
} }
invalidMsg := fmt.Sprintf("Invalid collection %s: %s. ", entityType, entity) invalidMsg := fmt.Sprintf("Invalid collection %s: %s. ", entityType, entity)
if int64(len(entity)) > Params.ProxyCfg.MaxNameLength { if len(entity) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + fmt.Sprintf("The length of a collection %s must be less than ", entityType) + msg := invalidMsg + fmt.Sprintf("The length of a collection %s must be less than ", entityType) + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
return errors.New(msg) return errors.New(msg)
} }
@ -139,9 +138,8 @@ func validatePartitionTag(partitionTag string, strictCheck bool) error {
return errors.New(msg) return errors.New(msg)
} }
if int64(len(partitionTag)) > Params.ProxyCfg.MaxNameLength { if len(partitionTag) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + "The length of a partition name must be less than " + msg := invalidMsg + "The length of a partition name must be less than " + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
return errors.New(msg) return errors.New(msg)
} }
@ -173,9 +171,8 @@ func validateFieldName(fieldName string) error {
} }
invalidMsg := "Invalid field name: " + fieldName + ". " invalidMsg := "Invalid field name: " + fieldName + ". "
if int64(len(fieldName)) > Params.ProxyCfg.MaxNameLength { if len(fieldName) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + "The length of a field name must be less than " + msg := invalidMsg + "The length of a field name must be less than " + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
return errors.New(msg) return errors.New(msg)
} }
@ -214,8 +211,8 @@ func validateDimension(field *schemapb.FieldSchema) error {
return errors.New("dimension is not defined in field type params, check type param `dim` for vector field") return errors.New("dimension is not defined in field type params, check type param `dim` for vector field")
} }
if dim <= 0 || dim > Params.ProxyCfg.MaxDimension { if dim <= 0 || dim > Params.ProxyCfg.MaxDimension.GetAsInt64() {
return fmt.Errorf("invalid dimension: %d. should be in range 1 ~ %d", dim, Params.ProxyCfg.MaxDimension) return fmt.Errorf("invalid dimension: %d. should be in range 1 ~ %d", dim, Params.ProxyCfg.MaxDimension.GetAsInt())
} }
if field.DataType == schemapb.DataType_BinaryVector && dim%8 != 0 { if field.DataType == schemapb.DataType_BinaryVector && dim%8 != 0 {
return fmt.Errorf("invalid dimension: %d. should be multiple of 8. ", dim) return fmt.Errorf("invalid dimension: %d. should be multiple of 8. ", dim)
@ -566,9 +563,8 @@ func ValidateUsername(username string) error {
} }
invalidMsg := "Invalid username: " + username + ". " invalidMsg := "Invalid username: " + username + ". "
if int64(len(username)) > Params.ProxyCfg.MaxUsernameLength { if len(username) > Params.ProxyCfg.MaxUsernameLength.GetAsInt() {
msg := invalidMsg + "The length of username must be less than " + msg := invalidMsg + "The length of username must be less than " + Params.ProxyCfg.MaxUsernameLength.GetValue() + " characters."
strconv.FormatInt(Params.ProxyCfg.MaxUsernameLength, 10) + " characters."
return errors.New(msg) return errors.New(msg)
} }
@ -590,9 +586,9 @@ func ValidateUsername(username string) error {
} }
func ValidatePassword(password string) error { func ValidatePassword(password string) error {
if int64(len(password)) < Params.ProxyCfg.MinPasswordLength || int64(len(password)) > Params.ProxyCfg.MaxPasswordLength { if len(password) < Params.ProxyCfg.MinPasswordLength.GetAsInt() || len(password) > Params.ProxyCfg.MaxPasswordLength.GetAsInt() {
msg := "The length of password must be great than " + strconv.FormatInt(Params.ProxyCfg.MinPasswordLength, 10) + msg := "The length of password must be great than " + Params.ProxyCfg.MinPasswordLength.GetValue() +
" and less than " + strconv.FormatInt(Params.ProxyCfg.MaxPasswordLength, 10) + " characters." " and less than " + Params.ProxyCfg.MaxPasswordLength.GetValue() + " characters."
return errors.New(msg) return errors.New(msg)
} }
return nil return nil
@ -600,10 +596,10 @@ func ValidatePassword(password string) error {
func validateTravelTimestamp(travelTs, tMax typeutil.Timestamp) error { func validateTravelTimestamp(travelTs, tMax typeutil.Timestamp) error {
durationSeconds := tsoutil.CalculateDuration(tMax, travelTs) / 1000 durationSeconds := tsoutil.CalculateDuration(tMax, travelTs) / 1000
if durationSeconds > Params.CommonCfg.RetentionDuration { if durationSeconds > Params.CommonCfg.RetentionDuration.GetAsInt64() {
durationIn := time.Second * time.Duration(durationSeconds) durationIn := time.Second * time.Duration(durationSeconds)
durationSupport := time.Second * time.Duration(Params.CommonCfg.RetentionDuration) durationSupport := time.Second * time.Duration(Params.CommonCfg.RetentionDuration.GetAsInt64())
return fmt.Errorf("only support to travel back to %v so far, but got %v", durationSupport, durationIn) return fmt.Errorf("only support to travel back to %v so far, but got %v", durationSupport, durationIn)
} }
return nil return nil
@ -618,7 +614,7 @@ func parseGuaranteeTs(ts, tMax typeutil.Timestamp) typeutil.Timestamp {
case strongTS: case strongTS:
ts = tMax ts = tMax
case boundedTS: case boundedTS:
ratio := time.Duration(-Params.CommonCfg.GracefulTime) ratio := time.Duration(-Params.CommonCfg.GracefulTime.GetAsInt64())
ts = tsoutil.AddPhysicalDurationOnTs(tMax, ratio*time.Millisecond) ts = tsoutil.AddPhysicalDurationOnTs(tMax, ratio*time.Millisecond)
} }
return ts return ts
@ -632,9 +628,8 @@ func validateName(entity string, nameType string) error {
} }
invalidMsg := fmt.Sprintf("invalid %s: %s. ", nameType, entity) invalidMsg := fmt.Sprintf("invalid %s: %s. ", nameType, entity)
if int64(len(entity)) > Params.ProxyCfg.MaxNameLength { if len(entity) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + fmt.Sprintf("the length of %s must be less than ", nameType) + msg := invalidMsg + fmt.Sprintf("the length of %s must be less than ", nameType) + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
return errors.New(msg) return errors.New(msg)
} }
@ -813,9 +808,8 @@ func validateIndexName(indexName string) error {
return nil return nil
} }
invalidMsg := "Invalid index name: " + indexName + ". " invalidMsg := "Invalid index name: " + indexName + ". "
if int64(len(indexName)) > Params.ProxyCfg.MaxNameLength { if len(indexName) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + "The length of a index name must be less than " + msg := invalidMsg + "The length of a index name must be less than " + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
return errors.New(msg) return errors.New(msg)
} }

View File

@ -35,6 +35,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util" "github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/util/crypto" "github.com/milvus-io/milvus/internal/util/crypto"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil" "github.com/milvus-io/milvus/internal/util/typeutil"
) )
@ -133,7 +134,7 @@ func TestValidateDimension(t *testing.T) {
fieldSchema.TypeParams = []*commonpb.KeyValuePair{ fieldSchema.TypeParams = []*commonpb.KeyValuePair{
{ {
Key: "dim", Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension)), Value: Params.ProxyCfg.MaxDimension.GetValue(),
}, },
} }
assert.Nil(t, validateDimension(fieldSchema)) assert.Nil(t, validateDimension(fieldSchema))
@ -149,7 +150,7 @@ func TestValidateDimension(t *testing.T) {
fieldSchema.TypeParams = []*commonpb.KeyValuePair{ fieldSchema.TypeParams = []*commonpb.KeyValuePair{
{ {
Key: "dim", Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension + 1)), Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension.GetAsInt32() + 1)),
}, },
} }
assert.NotNil(t, validateDimension(fieldSchema)) assert.NotNil(t, validateDimension(fieldSchema))
@ -165,7 +166,7 @@ func TestValidateDimension(t *testing.T) {
fieldSchema.TypeParams = []*commonpb.KeyValuePair{ fieldSchema.TypeParams = []*commonpb.KeyValuePair{
{ {
Key: "dim", Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension)), Value: strconv.Itoa(Params.ProxyCfg.MaxDimension.GetAsInt()),
}, },
} }
assert.Nil(t, validateDimension(fieldSchema)) assert.Nil(t, validateDimension(fieldSchema))
@ -784,18 +785,19 @@ func TestValidateTravelTimestamp(t *testing.T) {
travelTs := tsoutil.GetCurrentTime() travelTs := tsoutil.GetCurrentTime()
tests := []struct { tests := []struct {
description string description string
defaultRD int64 defaultRD string
nowTs typeutil.Timestamp nowTs typeutil.Timestamp
isValid bool isValid bool
}{ }{
{"one second", 100, tsoutil.AddPhysicalDurationOnTs(travelTs, time.Second), true}, {"one second", "100", tsoutil.AddPhysicalDurationOnTs(travelTs, time.Second), true},
{"retention duration", 100, tsoutil.AddPhysicalDurationOnTs(travelTs, 100*time.Second), true}, {"retention duration", "100", tsoutil.AddPhysicalDurationOnTs(travelTs, 100*time.Second), true},
{"retention duration+1", 100, tsoutil.AddPhysicalDurationOnTs(travelTs, 101*time.Second), false}, {"retention duration+1", "100", tsoutil.AddPhysicalDurationOnTs(travelTs, 101*time.Second), false},
} }
for _, test := range tests { for _, test := range tests {
t.Run(test.description, func(t *testing.T) { t.Run(test.description, func(t *testing.T) {
Params.CommonCfg.RetentionDuration = test.defaultRD paramtable.Get().Save(Params.CommonCfg.RetentionDuration.Key, test.defaultRD)
defer paramtable.Get().Reset(Params.CommonCfg.RetentionDuration.Key)
err := validateTravelTimestamp(travelTs, test.nowTs) err := validateTravelTimestamp(travelTs, test.nowTs)
if test.isValid { if test.isValid {
assert.NoError(t, err) assert.NoError(t, err)

View File

@ -18,6 +18,7 @@ package checkers
import ( import (
"context" "context"
"time"
"github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/balance"
. "github.com/milvus-io/milvus/internal/querycoordv2/params" . "github.com/milvus-io/milvus/internal/querycoordv2/params"
@ -44,7 +45,7 @@ func (b *BalanceChecker) Check(ctx context.Context) []task.Task {
ret := make([]task.Task, 0) ret := make([]task.Task, 0)
segmentPlans, channelPlans := b.Balance.Balance() segmentPlans, channelPlans := b.Balance.Balance()
tasks := balance.CreateSegmentTasksFromPlans(ctx, b.ID(), Params.QueryCoordCfg.SegmentTaskTimeout, segmentPlans) tasks := balance.CreateSegmentTasksFromPlans(ctx, b.ID(), Params.QueryCoordCfg.SegmentTaskTimeout.GetAsDuration(time.Millisecond), segmentPlans)
task.SetPriorityWithFunc(func(t task.Task) task.Priority { task.SetPriorityWithFunc(func(t task.Task) task.Priority {
if t.Priority() == task.TaskPriorityHigh { if t.Priority() == task.TaskPriorityHigh {
return task.TaskPriorityHigh return task.TaskPriorityHigh
@ -53,7 +54,7 @@ func (b *BalanceChecker) Check(ctx context.Context) []task.Task {
}, tasks...) }, tasks...)
ret = append(ret, tasks...) ret = append(ret, tasks...)
tasks = balance.CreateChannelTasksFromPlans(ctx, b.ID(), Params.QueryCoordCfg.ChannelTaskTimeout, channelPlans) tasks = balance.CreateChannelTasksFromPlans(ctx, b.ID(), Params.QueryCoordCfg.ChannelTaskTimeout.GetAsDuration(time.Millisecond), channelPlans)
ret = append(ret, tasks...) ret = append(ret, tasks...)
return ret return ret
} }

View File

@ -18,6 +18,7 @@ package checkers
import ( import (
"context" "context"
"time"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/querycoordv2/balance" "github.com/milvus-io/milvus/internal/querycoordv2/balance"
@ -176,14 +177,14 @@ func (c *ChannelChecker) createChannelLoadTask(ctx context.Context, channels []*
// log.Debug("try to subscribe channels", // log.Debug("try to subscribe channels",
// zap.Any("channels", channels), // zap.Any("channels", channels),
// zap.Any("plans", plans)) // zap.Any("plans", plans))
return balance.CreateChannelTasksFromPlans(ctx, c.ID(), Params.QueryCoordCfg.ChannelTaskTimeout, plans) return balance.CreateChannelTasksFromPlans(ctx, c.ID(), Params.QueryCoordCfg.ChannelTaskTimeout.GetAsDuration(time.Millisecond), plans)
} }
func (c *ChannelChecker) createChannelReduceTasks(ctx context.Context, channels []*meta.DmChannel, replicaID int64) []task.Task { func (c *ChannelChecker) createChannelReduceTasks(ctx context.Context, channels []*meta.DmChannel, replicaID int64) []task.Task {
ret := make([]task.Task, 0, len(channels)) ret := make([]task.Task, 0, len(channels))
for _, ch := range channels { for _, ch := range channels {
action := task.NewChannelAction(ch.Node, task.ActionTypeReduce, ch.GetChannelName()) action := task.NewChannelAction(ch.Node, task.ActionTypeReduce, ch.GetChannelName())
task, err := task.NewChannelTask(ctx, Params.QueryCoordCfg.ChannelTaskTimeout, c.ID(), ch.GetCollectionID(), replicaID, action) task, err := task.NewChannelTask(ctx, Params.QueryCoordCfg.ChannelTaskTimeout.GetAsDuration(time.Millisecond), c.ID(), ch.GetCollectionID(), replicaID, action)
if err != nil { if err != nil {
log.Warn("Create channel reduce task failed", log.Warn("Create channel reduce task failed",
zap.Int64("collection", ch.GetCollectionID()), zap.Int64("collection", ch.GetCollectionID()),

View File

@ -80,7 +80,7 @@ func NewCheckerController(
func (controller *CheckerController) Start(ctx context.Context) { func (controller *CheckerController) Start(ctx context.Context) {
go func() { go func() {
ticker := time.NewTicker(Params.QueryCoordCfg.CheckInterval) ticker := time.NewTicker(Params.QueryCoordCfg.CheckInterval.GetAsDuration(time.Millisecond))
defer ticker.Stop() defer ticker.Stop()
for { for {
select { select {
@ -98,7 +98,7 @@ func (controller *CheckerController) Start(ctx context.Context) {
case <-controller.checkCh: case <-controller.checkCh:
ticker.Stop() ticker.Stop()
controller.check(ctx) controller.check(ctx)
ticker.Reset(Params.QueryCoordCfg.CheckInterval) ticker.Reset(Params.QueryCoordCfg.CheckInterval.GetAsDuration(time.Millisecond))
} }
} }
}() }()

View File

@ -18,6 +18,7 @@ package checkers
import ( import (
"context" "context"
"time"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
@ -269,7 +270,7 @@ func (c *SegmentChecker) createSegmentLoadTasks(ctx context.Context, segments []
for i := range plans { for i := range plans {
plans[i].ReplicaID = replica.GetID() plans[i].ReplicaID = replica.GetID()
} }
return balance.CreateSegmentTasksFromPlans(ctx, c.ID(), Params.QueryCoordCfg.SegmentTaskTimeout, plans) return balance.CreateSegmentTasksFromPlans(ctx, c.ID(), Params.QueryCoordCfg.SegmentTaskTimeout.GetAsDuration(time.Millisecond), plans)
} }
func (c *SegmentChecker) createSegmentReduceTasks(ctx context.Context, segments []*meta.Segment, replicaID int64, scope querypb.DataScope) []task.Task { func (c *SegmentChecker) createSegmentReduceTasks(ctx context.Context, segments []*meta.Segment, replicaID int64, scope querypb.DataScope) []task.Task {
@ -278,7 +279,7 @@ func (c *SegmentChecker) createSegmentReduceTasks(ctx context.Context, segments
action := task.NewSegmentActionWithScope(s.Node, task.ActionTypeReduce, s.GetInsertChannel(), s.GetID(), scope) action := task.NewSegmentActionWithScope(s.Node, task.ActionTypeReduce, s.GetInsertChannel(), s.GetID(), scope)
task, err := task.NewSegmentTask( task, err := task.NewSegmentTask(
ctx, ctx,
Params.QueryCoordCfg.SegmentTaskTimeout, Params.QueryCoordCfg.SegmentTaskTimeout.GetAsDuration(time.Millisecond),
c.ID(), c.ID(),
s.GetCollectionID(), s.GetCollectionID(),
replicaID, replicaID,

View File

@ -57,7 +57,7 @@ func (dh *distHandler) start(ctx context.Context) {
defer dh.wg.Done() defer dh.wg.Done()
logger := log.With(zap.Int64("nodeID", dh.nodeID)) logger := log.With(zap.Int64("nodeID", dh.nodeID))
logger.Info("start dist handler") logger.Info("start dist handler")
ticker := time.NewTicker(Params.QueryCoordCfg.DistPullInterval) ticker := time.NewTicker(Params.QueryCoordCfg.DistPullInterval.GetAsDuration(time.Millisecond))
failures := 0 failures := 0
for { for {
select { select {

View File

@ -21,6 +21,7 @@ import (
"errors" "errors"
"fmt" "fmt"
"sync" "sync"
"time"
"github.com/samber/lo" "github.com/samber/lo"
"go.uber.org/zap" "go.uber.org/zap"
@ -133,7 +134,7 @@ func (s *Server) balanceSegments(ctx context.Context, req *querypb.LoadBalanceRe
zap.Int64("segmentID", plan.Segment.GetID()), zap.Int64("segmentID", plan.Segment.GetID()),
) )
task, err := task.NewSegmentTask(ctx, task, err := task.NewSegmentTask(ctx,
Params.QueryCoordCfg.SegmentTaskTimeout, Params.QueryCoordCfg.SegmentTaskTimeout.GetAsDuration(time.Millisecond),
req.GetBase().GetMsgID(), req.GetBase().GetMsgID(),
req.GetCollectionID(), req.GetCollectionID(),
replica.GetID(), replica.GetID(),
@ -159,7 +160,7 @@ func (s *Server) balanceSegments(ctx context.Context, req *querypb.LoadBalanceRe
} }
tasks = append(tasks, task) tasks = append(tasks, task)
} }
return task.Wait(ctx, Params.QueryCoordCfg.SegmentTaskTimeout, tasks...) return task.Wait(ctx, Params.QueryCoordCfg.SegmentTaskTimeout.GetAsDuration(time.Millisecond), tasks...)
} }
// TODO(dragondriver): add more detail metrics // TODO(dragondriver): add more detail metrics
@ -181,14 +182,14 @@ func (s *Server) getSystemInfoMetrics(
DiskUsage: hardware.GetDiskUsage(), DiskUsage: hardware.GetDiskUsage(),
}, },
SystemInfo: metricsinfo.DeployMetrics{}, SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.QueryCoordCfg.CreatedTime.String(), CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: Params.QueryCoordCfg.UpdatedTime.String(), UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.QueryCoordRole, Type: typeutil.QueryCoordRole,
ID: s.session.ServerID, ID: s.session.ServerID,
}, },
SystemConfigurations: metricsinfo.QueryCoordConfiguration{ SystemConfigurations: metricsinfo.QueryCoordConfiguration{
SearchChannelPrefix: Params.CommonCfg.QueryCoordSearch, SearchChannelPrefix: Params.CommonCfg.QueryCoordSearch.GetValue(),
SearchResultChannelPrefix: Params.CommonCfg.QueryCoordSearchResult, SearchResultChannelPrefix: Params.CommonCfg.QueryCoordSearchResult.GetValue(),
}, },
}, },
ConnectedNodes: make([]metricsinfo.QueryNodeInfos, 0), ConnectedNodes: make([]metricsinfo.QueryNodeInfos, 0),

Some files were not shown because too many files have changed in this diff Show More