Refactor all params into ParamItem (#20987)

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

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

View File

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

View File

@ -79,13 +79,11 @@ func (c *MilvusConfig) init(base *paramtable.BaseTable) {
c.EtcdCfg = &paramtable.EtcdConfig{}
c.MysqlCfg = &paramtable.MetaDBConfig{}
c.MetaStoreCfg.Base = base
c.MetaStoreCfg.LoadCfgToMemory()
c.MetaStoreCfg.Init(base)
switch c.MetaStoreCfg.MetaStoreType {
switch c.MetaStoreCfg.MetaStoreType.GetValue() {
case util.MetaStoreTypeMysql:
c.MysqlCfg.Base = base
c.MysqlCfg.LoadCfgToMemory()
c.MysqlCfg.Init(base)
default:
}
@ -96,11 +94,11 @@ func (c *MilvusConfig) String() string {
if c == nil {
return ""
}
switch c.MetaStoreCfg.MetaStoreType {
switch c.MetaStoreCfg.MetaStoreType.GetValue() {
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:
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)
for _, kv := range indexInfo.IndexParams {
if kv.Key == common.IndexParamsKey {
params, err := funcutil.ParseIndexParamsMap(kv.Value)
params, err := funcutil.JSONToMap(kv.Value)
if err != nil {
return nil, err
}

View File

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

View File

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

View File

@ -153,6 +153,12 @@ func (m *Manager) DeleteConfig(key string) {
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.
func (m *Manager) pullSourceConfigs(source string) error {
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 {
r.intervalInitOnce.Do(func() {
go r.refreshPeriodically()
go r.refreshPeriodically(name)
})
}
}
@ -52,9 +52,9 @@ func (r refresher) stop() {
r.intervalDone <- true
}
func (r refresher) refreshPeriodically() {
func (r refresher) refreshPeriodically(name string) {
ticker := time.NewTicker(r.refreshInterval)
log.Info("start refreshing configurations")
log.Info("start refreshing configurations", zap.String("source", name))
for {
select {
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) {
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
keys, values, err := c.watchkv.LoadWithPrefix(prefix)

View File

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

View File

@ -414,7 +414,7 @@ func (c *ChannelManager) unsubAttempt(ncInfo *NodeChannelInfo) {
nodeID := ncInfo.NodeID
for _, ch := range ncInfo.Channels {
// 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)
msgstream.UnsubscribeChannels(c.ctx, c.msgstreamFactory, subName, []string{pchannelName})
}
@ -449,7 +449,7 @@ func (c *ChannelManager) fillChannelWatchInfo(op *ChannelOp) {
Vchan: vcInfo,
StartTs: time.Now().Unix(),
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,
}
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 {
var channelsWithTimer = []string{}
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 {
vcInfo := c.h.GetDataVChanPositions(ch, allPartitionID)
info := &datapb.ChannelWatchInfo{
@ -653,7 +653,7 @@ func (c *ChannelManager) watchChannelStatesLoop(ctx context.Context) {
defer logutil.LogPanic()
// 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
etcdWatcher, timeoutWatcher := c.stateTimer.getWatchers(watchPrefix)
@ -788,7 +788,7 @@ func (c *ChannelManager) CleanupAndReassign(nodeID UniqueID, channelName string)
if c.msgstreamFactory == nil {
log.Warn("msgstream factory is not set, unable to clean up topics")
} 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)
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
func waitAndCheckState(t *testing.T, kv kv.MetaKv, expectedState datapb.ChannelWatchState, nodeID UniqueID, channelName string, collectionID UniqueID) {
for {
prefix := Params.DataCoordCfg.ChannelWatchSubPath
prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
v, err := kv.Load(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName))
if err == nil && len(v) > 0 {
watchInfo, err := parseWatchInfo("fake", []byte(v))
@ -93,7 +93,7 @@ func TestChannelManager_StateTransfer(t *testing.T) {
p := "/tmp/milvus_ut/rdb_data"
t.Setenv("ROCKSMQ_PATH", p)
prefix := Params.DataCoordCfg.ChannelWatchSubPath
prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
var (
collectionID = UniqueID(9)
@ -376,7 +376,7 @@ func TestChannelManager(t *testing.T) {
metakv.Close()
}()
prefix := Params.DataCoordCfg.ChannelWatchSubPath
prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
t.Run("test AddNode with avalible node", func(t *testing.T) {
// Note: this test is based on the default registerPolicy
defer metakv.RemoveWithPrefix("")
@ -574,7 +574,7 @@ func TestChannelManager(t *testing.T) {
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)
assert.NoError(t, err)
@ -736,7 +736,7 @@ func TestChannelManager_Reload(t *testing.T) {
collectionID = UniqueID(2)
channelName = "channel-checkOldNodes"
)
prefix := Params.DataCoordCfg.ChannelWatchSubPath
prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
getWatchInfoWithState := func(state datapb.ChannelWatchState, collectionID UniqueID, channelName string) *datapb.ChannelWatchInfo {
return &datapb.ChannelWatchInfo{
@ -904,7 +904,7 @@ func TestChannelManager_BalanceBehaviour(t *testing.T) {
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) {
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.
func (c *ChannelStore) Reload() error {
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 {
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.
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.
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.

View File

@ -85,7 +85,7 @@ func TestClusterCreate(t *testing.T) {
}
info1Data, err := proto.Marshal(info1)
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)
sessionManager := NewSessionManager()

View File

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

View File

@ -23,6 +23,7 @@ import (
"time"
"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-proto/go-api/commonpb"
@ -124,7 +125,7 @@ func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
parallelCh: make(map[int64]chan struct{}),
allocator: newMockAllocator(),
}
Params.DataCoordCfg.CompactionCheckIntervalInSeconds = 1
Params.Save(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key, "1")
c.start()
err := c.execCompactionPlan(tt.args.signal, tt.args.plan)
assert.Equal(t, tt.err, err)
@ -154,7 +155,8 @@ func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
func Test_compactionPlanHandler_execWithParallels(t *testing.T) {
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{
plans: map[int64]*compactionTask{},
sessions: &SessionManager{

View File

@ -102,7 +102,7 @@ func newCompactionTrigger(
func (t *compactionTrigger) start() {
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)
go func() {
defer logutil.LogPanic()
@ -134,7 +134,7 @@ func (t *compactionTrigger) startGlobalCompactionLoop() {
defer t.wg.Done()
// If AutoCompaction disabled, global loop will not start
if !Params.DataCoordCfg.GetEnableAutoCompaction() {
if !Params.DataCoordCfg.EnableAutoCompaction.GetAsBool() {
return
}
@ -184,7 +184,7 @@ func (t *compactionTrigger) getCompactTime(ts Timestamp, collectionID UniqueID)
}
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)
if collectionTTL > 0 {
@ -216,7 +216,7 @@ func (t *compactionTrigger) triggerCompaction() error {
// triggerSingleCompaction triger a compaction bundled with collection-partiiton-channel-segment
func (t *compactionTrigger) triggerSingleCompaction(collectionID, partitionID, segmentID int64, channel string) error {
// If AutoCompaction diabled, flush request will not trigger compaction
if !Params.DataCoordCfg.GetEnableAutoCompaction() {
if !Params.DataCoordCfg.EnableAutoCompaction.GetAsBool() {
return nil
}
@ -515,7 +515,7 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, c
if segment.GetNumOfRows() < segment.GetMaxRowNum() {
var result []*SegmentInfo
free := segment.GetMaxRowNum() - segment.GetNumOfRows()
maxNum := Params.DataCoordCfg.MaxSegmentToMerge - 1
maxNum := Params.DataCoordCfg.MaxSegmentToMerge.GetAsInt() - 1
prioritizedCandidates, result, free = greedySelect(prioritizedCandidates, free, maxNum)
bucket = append(bucket, 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
// 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.
smallCandidates, result, _ = reverseGreedySelect(smallCandidates, free, Params.DataCoordCfg.MaxSegmentToMerge-1)
smallCandidates, result, _ = reverseGreedySelect(smallCandidates, free, Params.DataCoordCfg.MaxSegmentToMerge.GetAsInt()-1)
bucket = append(bucket, result...)
var size int64
@ -560,7 +560,7 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, c
targetRow += s.GetNumOfRows()
}
// 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)
log.Info("generate a plan for small candidates", zap.Any("plan", plan),
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 {
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 {
@ -653,7 +653,7 @@ func (t *compactionTrigger) fillOriginPlan(plan *datapb.CompactionPlan) error {
return err
}
plan.PlanID = id
plan.TimeoutInSeconds = Params.DataCoordCfg.CompactionTimeoutInSeconds
plan.TimeoutInSeconds = int32(Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt())
return nil
}
@ -676,7 +676,7 @@ func (t *compactionTrigger) ShouldDoSingleCompaction(segment *SegmentInfo, compa
totalLogNum += len(statsLogs.GetBinlogs())
}
// 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),
zap.Int("Delta logs", len(segment.GetDeltalogs())), zap.Int("Bin Logs", len(segment.GetBinlogs())), zap.Int("Stat logs", len(segment.GetStatslogs())))
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),
zap.Int("expired rows", totalExpiredRows), zap.Int64("expired log size", totalExpiredSize))
return true
@ -721,7 +721,7 @@ func (t *compactionTrigger) ShouldDoSingleCompaction(segment *SegmentInfo, compa
}
// 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),
zap.Int("deleted rows", totalDeletedRows), zap.Int64("delete log size", totalDeleteLogSize))
return true

View File

@ -24,6 +24,7 @@ import (
"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-proto/go-api/commonpb"
@ -84,11 +85,12 @@ func Test_compactionTrigger_force(t *testing.T) {
segRefer *SegmentReferenceManager
}
Params.Init()
Params.CommonCfg.RetentionDuration = 200
paramtable.Init()
paramtable.Get().Save(Params.CommonCfg.RetentionDuration.Key, "200")
defer paramtable.Get().Reset(Params.CommonCfg.RetentionDuration.Key)
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)
vecFieldID := int64(201)
@ -331,7 +333,7 @@ func Test_compactionTrigger_force(t *testing.T) {
},
},
StartTime: 0,
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds,
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(),
Type: datapb.CompactionType_MixCompaction,
Timetravel: timeTravel,
Channel: "ch1",
@ -766,7 +768,7 @@ func Test_compactionTrigger_force_maxSegmentLimit(t *testing.T) {
},
},
StartTime: 3,
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds,
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(),
Type: datapb.CompactionType_MixCompaction,
Timetravel: 200,
Channel: "ch1",
@ -870,7 +872,7 @@ func Test_compactionTrigger_noplan(t *testing.T) {
Binlogs: []*datapb.FieldBinlog{
{
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()
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
})
@ -1052,7 +1052,7 @@ func (m *meta) revertAlterMetaStoreAfterCompaction(oldSegments []*SegmentInfo, r
m.Lock()
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
})

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}
})

View File

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

View File

@ -461,7 +461,7 @@ func BgCheckWithMaxWatchDuration(kv kv.TxnKV) ChannelBGChecker {
}
startTime := time.Unix(watchInfo.StartTs, 0)
d := ts.Sub(startTime)
if d >= Params.DataCoordCfg.MaxWatchDuration {
if d >= Params.DataCoordCfg.MaxWatchDuration.GetAsDuration(time.Second) {
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}},
{1, "chan2", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Complete}}}),
[]*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}}}},
nil,
@ -412,7 +412,7 @@ func TestBgCheckWithMaxWatchDuration(t *testing.T) {
args{
getKv([]*watch{{1, "chan1", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Uncomplete}}}),
[]*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{},
nil,

View File

@ -41,7 +41,7 @@ func calBySchemaPolicy(schema *schemapb.CollectionSchema) (int, error) {
if sizePerRecord == 0 {
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
}
@ -57,7 +57,7 @@ func calBySchemaPolicyWithDiskIndex(schema *schemapb.CollectionSchema) (int, err
if sizePerRecord == 0 {
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
}

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,
},
}

View File

@ -184,9 +184,9 @@ func defaultAllocatePolicy() AllocatePolicy {
func defaultSegmentSealPolicy() []segmentSealPolicy {
return []segmentSealPolicy{
sealByLifetimePolicy(Params.DataCoordCfg.SegmentMaxLifetime),
getSegmentCapacityPolicy(Params.DataCoordCfg.SegmentSealProportion),
sealLongTimeIdlePolicy(Params.DataCoordCfg.SegmentMaxIdleTime, Params.DataCoordCfg.SegmentMinSizeFromIdleToSealed, Params.DataCoordCfg.SegmentMaxSize),
sealByLifetimePolicy(Params.DataCoordCfg.SegmentMaxLifetime.GetAsDuration(time.Second)),
getSegmentCapacityPolicy(Params.DataCoordCfg.SegmentSealProportion.GetAsFloat()),
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
}
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))
return expireTs, nil
}

View File

@ -194,7 +194,7 @@ func CreateServer(ctx context.Context, factory dependency.Factory, opts ...Optio
rootCoordClientCreator: defaultRootCoordCreatorFunc,
helper: defaultServerHelper(),
metricsCacheManager: metricsinfo.NewMetricsCacheManager(),
enableActiveStandBy: Params.DataCoordCfg.EnableActiveStandby,
enableActiveStandBy: Params.DataCoordCfg.EnableActiveStandby.GetAsBool(),
}
for _, opt := range opts {
@ -282,7 +282,7 @@ func (s *Server) Init() error {
return err
}
if Params.DataCoordCfg.EnableCompaction {
if Params.DataCoordCfg.EnableCompaction.GetAsBool() {
s.createCompactionHandler()
s.createCompactionTrigger()
}
@ -301,7 +301,7 @@ func (s *Server) Init() error {
// datanodes etcd watch, etcd alive check and flush completed status check
// 4. set server state to Healthy
func (s *Server) Start() error {
if Params.DataCoordCfg.EnableCompaction {
if Params.DataCoordCfg.EnableCompaction.GetAsBool() {
s.compactionHandler.start()
s.compactionTrigger.start()
}
@ -322,9 +322,6 @@ func (s *Server) Start() error {
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
// data from all DataNode.
// 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) {
s.garbageCollector = newGarbageCollector(s.meta, s.handler, s.segReferManager, s.indexCoord, GcOption{
cli: cli,
enabled: Params.DataCoordCfg.EnableGarbageCollection,
checkInterval: Params.DataCoordCfg.GCInterval,
missingTolerance: Params.DataCoordCfg.GCMissingTolerance,
dropTolerance: Params.DataCoordCfg.GCDropTolerance,
enabled: Params.DataCoordCfg.EnableGarbageCollection.GetAsBool(),
checkInterval: Params.DataCoordCfg.GCInterval.GetAsDuration(time.Second),
missingTolerance: Params.DataCoordCfg.GCMissingTolerance.GetAsDuration(time.Second),
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))
panic(err)
}
subName := fmt.Sprintf("%s-%d-datanodeTl", Params.CommonCfg.DataCoordSubName, paramtable.GetNodeID())
ttMsgStream.AsConsumer([]string{Params.CommonCfg.DataCoordTimeTick},
subName := fmt.Sprintf("%s-%d-datanodeTl", Params.CommonCfg.DataCoordSubName.GetValue(), paramtable.GetNodeID())
ttMsgStream.AsConsumer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()},
subName, mqwrapper.SubscriptionPositionLatest)
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))
go s.handleDataNodeTimetickMsgstream(ctx, ttMsgStream)
@ -827,7 +824,7 @@ func (s *Server) Stop() error {
s.stopServerLoop()
s.session.Revoke(time.Second)
if Params.DataCoordCfg.EnableCompaction {
if Params.DataCoordCfg.EnableCompaction.GetAsBool() {
s.stopCompactionTrigger()
s.stopCompactionHandler()
}

View File

@ -75,7 +75,7 @@ func TestGetSegmentInfoChannel(t *testing.T) {
resp, err := svr.GetSegmentInfoChannel(context.TODO())
assert.Nil(t, err)
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())
assert.Nil(t, err)
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) {
@ -1486,7 +1486,7 @@ func TestDataNodeTtChannel(t *testing.T) {
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.Nil(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick})
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close()
info := &NodeInfo{
Address: "localhost:7777",
@ -1553,7 +1553,7 @@ func TestDataNodeTtChannel(t *testing.T) {
})
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.Nil(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick})
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close()
info := &NodeInfo{
Address: "localhost:7777",
@ -1634,7 +1634,7 @@ func TestDataNodeTtChannel(t *testing.T) {
ttMsgStream, err := svr.factory.NewMsgStream(context.TODO())
assert.Nil(t, err)
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick})
ttMsgStream.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
defer ttMsgStream.Close()
node := &NodeInfo{
NodeID: 0,
@ -2600,7 +2600,8 @@ func TestGetRecoveryInfo(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) {
svr := &Server{}
svr.stateCode.Store(commonpb.StateCode_Healthy)
@ -2665,7 +2666,8 @@ func TestGetCompactionState(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) {
svr := &Server{allocator: &MockAllocator{}}
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 {
var err error
Params.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)
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 {
var err error
Params.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)
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 {
var err error
Params.Init()
Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int())
paramtable.Init()
paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))
factory := dependency.NewDefaultFactory(true)
etcdCli, err := etcd.GetEtcdClient(
@ -3714,12 +3714,11 @@ func Test_initServiceDiscovery(t *testing.T) {
func Test_newChunkManagerFactory(t *testing.T) {
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) {
os.Setenv("minio.address", "host:9000:bad")
defer os.Unsetenv("minio.address")
Params.Init()
paramtable.Get().Save(Params.MinioCfg.Address.Key, "host:9000:bad")
defer paramtable.Get().Reset(Params.MinioCfg.Address.Key)
storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli)
assert.Error(t, err)
@ -3727,22 +3726,19 @@ func Test_newChunkManagerFactory(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()
assert.NotNil(t, storageCli)
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) {
paramtable.Get().Save(Params.DataCoordCfg.EnableGarbageCollection.Key, "true")
defer paramtable.Get().Reset(Params.DataCoordCfg.EnableGarbageCollection.Key)
server := newTestServer2(t, nil)
Params.DataCoordCfg.EnableGarbageCollection = true
t.Run("ok", func(t *testing.T) {
storageCli, err := server.newChunkManagerFactory()
@ -3751,10 +3747,9 @@ func Test_initGarbageCollection(t *testing.T) {
server.initGarbageCollection(storageCli)
})
t.Run("err_minio_bad_address", func(t *testing.T) {
Params.CommonCfg.StorageType = "minio"
os.Setenv("minio.address", "host:9000:bad")
defer os.Unsetenv("minio.address")
Params.Init()
paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "minio")
paramtable.Get().Save(Params.MinioCfg.Address.Key, "host:9000:bad")
defer paramtable.Get().Reset(Params.MinioCfg.Address.Key)
storageCli, err := server.newChunkManagerFactory()
assert.Nil(t, storageCli)
assert.Error(t, err)
@ -3764,7 +3759,7 @@ func Test_initGarbageCollection(t *testing.T) {
func testDataCoordBase(t *testing.T, opts ...Option) *Server {
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)
etcdCli, err := etcd.GetEtcdClient(
@ -3810,16 +3805,15 @@ func testDataCoordBase(t *testing.T, opts ...Option) *Server {
}
func TestDataCoord_DisableActiveStandby(t *testing.T) {
Params.Init()
Params.DataCoordCfg.EnableActiveStandby = false
paramtable.Get().Save(Params.DataCoordCfg.EnableActiveStandby.Key, "false")
svr := testDataCoordBase(t)
defer closeTestServer(t, svr)
}
// make sure the main functions work well when EnableActiveStandby=true
func TestDataCoord_EnableActiveStandby(t *testing.T) {
Params.Init()
Params.DataCoordCfg.EnableActiveStandby = true
paramtable.Get().Save(Params.DataCoordCfg.EnableActiveStandby.Key, "true")
defer paramtable.Get().Reset(Params.DataCoordCfg.EnableActiveStandby.Key)
svr := testDataCoordBase(t)
defer closeTestServer(t, svr)
}

View File

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

View File

@ -88,14 +88,14 @@ func GetCompactTime(ctx context.Context, allocator allocator) (*compactTime, err
}
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)
// TODO, change to collection level
if Params.CommonCfg.EntityExpirationTTL > 0 {
ttexpired := pts.Add(-Params.CommonCfg.EntityExpirationTTL)
if Params.CommonCfg.EntityExpirationTTL.GetAsInt() > 0 {
ttexpired := pts.Add(-1 * Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second))
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
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 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/internal/common"
"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/stretchr/testify/suite"
)
@ -117,11 +118,11 @@ func (suite *UtilSuite) TestVerifyResponse() {
}
func (suite *UtilSuite) TestGetCompactTime() {
Params.Init()
Params.CommonCfg.RetentionDuration = 43200 // 5 days
paramtable.Get().Save(Params.CommonCfg.RetentionDuration.Key, "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)
tBefore := tFixed.Add(-time.Duration(Params.CommonCfg.RetentionDuration) * time.Second)
tBefore := tFixed.Add(-1 * Params.CommonCfg.RetentionDuration.GetAsDuration(time.Second))
type args struct {
allocator allocator
@ -192,5 +193,5 @@ func (suite *UtilSuite) TestGetCollectionTTL() {
ttl, err = getCollectionTTL(map[string]string{})
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 {
var shouldFlushSegments []UniqueID
if bm.delMemorySize < Params.DataNodeCfg.FlushDeleteBufferBytes {
if bm.delMemorySize < Params.DataNodeCfg.FlushDeleteBufferBytes.GetAsInt64() {
return shouldFlushSegments
}
mmUsage := bm.delMemorySize
@ -167,7 +167,7 @@ func (bm *DelBufferManager) ShouldFlushSegments() []UniqueID {
shouldFlushSegments = append(shouldFlushSegments, segMem.segmentID)
log.Debug("add segment for delete buf flush", zap.Int64("segmentID", segMem.segmentID))
mmUsage -= segMem.memorySize
if mmUsage < Params.DataNodeCfg.FlushDeleteBufferBytes {
if mmUsage < Params.DataNodeCfg.FlushDeleteBufferBytes.GetAsInt64() {
break
}
}
@ -357,7 +357,7 @@ func newBufferData(collSchema *schemapb.CollectionSchema) (*BufferData, error) {
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
return &BufferData{

View File

@ -20,6 +20,7 @@ import (
"container/heap"
"fmt"
"math"
"strconv"
"testing"
"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/schemapb"
"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 {
@ -52,8 +54,7 @@ func genTestCollectionSchema(dim int64, vectorType schemapb.DataType) *schemapb.
}
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 {
isValid bool
@ -92,7 +93,7 @@ func TestBufferData(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 {
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))
return 0, err
}
threshold := Params.DataCoordCfg.SegmentMaxSize * 1024 * 1024
threshold := Params.DataCoordCfg.SegmentMaxSize.GetAsFloat() * 1024 * 1024
return int64(threshold / float64(sizePerRecord)), nil
}

View File

@ -318,7 +318,7 @@ func (t *compactionTask) merge(
numRows = 0
numBinlogs = 0
currentTs := t.GetCurrentTime()
maxRowsPerBinlog = int(Params.DataNodeCfg.FlushInsertBufferSize / (int64(dim) * 4))
maxRowsPerBinlog = int(Params.DataNodeCfg.FlushInsertBufferSize.GetAsInt64() / (int64(dim) * 4))
currentRows := 0
downloadTimeCost := 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/internalpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
@ -273,7 +274,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
t.Run("Merge without expiration", func(t *testing.T) {
alloc := NewAllocatorFactory(1)
mockbIO := &binlogIO{cm, alloc}
Params.CommonCfg.EntityExpirationTTL = 0
paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0")
iData := genInsertDataWithExpiredTS()
var allPaths [][]string
@ -305,12 +306,12 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
t.Run("Merge without expiration2", func(t *testing.T) {
alloc := NewAllocatorFactory(1)
mockbIO := &binlogIO{cm, alloc}
Params.CommonCfg.EntityExpirationTTL = 0
paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0")
flushInsertBufferSize := Params.DataNodeCfg.FlushInsertBufferSize
defer func() {
Params.DataNodeCfg.FlushInsertBufferSize = flushInsertBufferSize
}()
Params.DataNodeCfg.FlushInsertBufferSize = 128
paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "128")
iData := genInsertDataWithExpiredTS()
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) {
alloc := NewAllocatorFactory(1)
mockbIO := &binlogIO{cm, alloc}
Params.CommonCfg.EntityExpirationTTL = 0
paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0")
iData := genInsertDataWithExpiredTS()
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) {
alloc := NewAllocatorFactory(1)
mockbIO := &binlogIO{cm, alloc}
Params.CommonCfg.EntityExpirationTTL = 0
paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0")
iData := genInsertDataWithExpiredTS()
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
@ -561,7 +562,7 @@ func TestCompactorInterfaceMethods(t *testing.T) {
Field2StatslogPaths: 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) {
invalidAlloc := NewAllocatorFactory(-1)

View File

@ -235,7 +235,7 @@ func (node *DataNode) initRateCollector() error {
// Init function does nothing now.
func (node *DataNode) Init() error {
log.Info("DataNode server initializing",
zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick),
zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick.GetValue()),
)
if err := node.initSession(); err != nil {
log.Error("DataNode server init session failed", zap.Error(err))
@ -260,7 +260,7 @@ func (node *DataNode) Init() error {
node.factory.Init(Params)
log.Info("DataNode server init succeeded",
zap.String("MsgChannelSubName", Params.CommonCfg.DataNodeSubName))
zap.String("MsgChannelSubName", Params.CommonCfg.DataNodeSubName.GetValue()))
return nil
}
@ -270,7 +270,7 @@ func (node *DataNode) StartWatchChannels(ctx context.Context) {
defer logutil.LogPanic()
// 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
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)
// after watch, first check all exists nodes first
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
func (node *DataNode) checkWatchedList() error {
// 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)
if err != nil {
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)
}
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))
// etcd error, retrying
@ -518,9 +518,6 @@ func (node *DataNode) Start() error {
// Start node watch node
go node.StartWatchChannels(node.ctx)
Params.DataNodeCfg.CreatedTime = time.Now()
Params.DataNodeCfg.UpdatedTime = time.Now()
node.UpdateStateCode(commonpb.StateCode_Healthy)
return nil
}
@ -1068,7 +1065,7 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest)
}
// 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,
node.chunkManager, importResult, reportFunc)
importWrapper.SetCallbackFunctions(assignSegmentFunc(node, req),

View File

@ -75,10 +75,9 @@ func TestMain(t *testing.M) {
os.Setenv("ROCKSMQ_PATH", path)
defer os.RemoveAll(path)
Params.DataNodeCfg.InitAlias("datanode-alias-1")
Params.Init()
// 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()
if err != nil {
@ -876,15 +875,15 @@ func TestWatchChannel(t *testing.T) {
// GOOSE TODO
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh)
path := fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), oldInvalidCh)
err = kv.Save(path, string([]byte{23}))
assert.NoError(t, err)
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{})
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{}{}
cnt := 0
for {
@ -923,7 +922,7 @@ func TestWatchChannel(t *testing.T) {
exist := node.flowgraphManager.exist(ch)
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)
//TODO there is not way to sync Release done, use sleep for now
time.Sleep(100 * time.Millisecond)
@ -935,15 +934,15 @@ func TestWatchChannel(t *testing.T) {
t.Run("Test release channel", func(t *testing.T) {
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
path := fmt.Sprintf("%s/%d/%s", Params.DataNodeCfg.ChannelWatchSubPath, paramtable.GetNodeID(), oldInvalidCh)
path := fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), oldInvalidCh)
err = kv.Save(path, string([]byte{23}))
assert.NoError(t, err)
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{})
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{}{}
cnt := 0
for {
@ -982,7 +981,7 @@ func TestWatchChannel(t *testing.T) {
exist := node.flowgraphManager.exist(ch)
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)
//TODO there is not way to sync Release done, use sleep for now
time.Sleep(100 * time.Millisecond)

View File

@ -121,7 +121,7 @@ type nodeConfig struct {
}
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

View File

@ -36,6 +36,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/paramtable"
)
var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service"
@ -217,7 +218,7 @@ func TestDataSyncService_Start(t *testing.T) {
allocFactory := NewAllocatorFactory(1)
factory := dependency.NewDefaultFactory(true)
Params.DataNodeCfg.FlushInsertBufferSize = 1
paramtable.Get().Save(Params.DataNodeCfg.FlushInsertBufferSize.Key, "1")
ufs := []*datapb.SegmentInfo{{
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) {
baseNode := BaseNode{}
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength)
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism)
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
deltaStream, err := msFactory.NewMsgStream(ctx)
if err != nil {
@ -360,7 +360,7 @@ func newDDNode(ctx context.Context, collID UniqueID, vChannelName string, droppe
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 {
return nil, err
}

View File

@ -33,6 +33,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/internal/util/paramtable"
"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
//and the sum of memory consumption in this case is 208
//so no segments will be flushed
Params.DataNodeCfg.FlushDeleteBufferBytes = 300
paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "300")
delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 0, len(mockFlushManager.flushedSegIDs))
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)
msg.deleteMessages = []*msgstream.DeleteMsg{}
msg.segmentsToSync = []UniqueID{}
Params.DataNodeCfg.FlushDeleteBufferBytes = 200
paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "200")
delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 1, len(mockFlushManager.flushedSegIDs))
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
//segment which is 48 in size to be flushed, so the remained del memory size
//will be 112
Params.DataNodeCfg.FlushDeleteBufferBytes = 150
paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "150")
delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 2, len(mockFlushManager.flushedSegIDs))
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
//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})
assert.Equal(t, 4, len(mockFlushManager.flushedSegIDs))
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
//is more than 20, so all five segments will be flushed and the remained
//del memory will be lowered to zero
Params.DataNodeCfg.FlushDeleteBufferBytes = 20
paramtable.Get().Save(Params.DataNodeCfg.FlushDeleteBufferBytes.Key, "20")
delNode.Operate([]flowgraph.Msg{fgMsg})
assert.Equal(t, 5, len(mockFlushManager.flushedSegIDs))
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) {
// subName should be unique, since pchannelName is shared among several collections
// 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)
if err != nil {
return nil, err

View File

@ -605,9 +605,9 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl
if err != nil {
return nil, err
}
wTt.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick})
wTt.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
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
mt := newMergedTimeTickerSender(func(ts Timestamp, segmentIDs []int64) error {

View File

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

View File

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

View File

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

View File

@ -7,12 +7,13 @@ import (
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/util/concurrency"
"github.com/milvus-io/milvus/internal/util/paramtable"
)
func Test_getOrCreateIOPool(t *testing.T) {
Params.InitOnce()
ioConcurrency := Params.DataNodeCfg.IOConcurrency
Params.DataNodeCfg.IOConcurrency = 64
paramtable.Get().Save(Params.DataNodeCfg.IOConcurrency.Key, "64")
defer func() { Params.DataNodeCfg.IOConcurrency = ioConcurrency }()
nP := 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
func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse {
prefix := "datanode."
matchedConfig := Params.DataNodeCfg.Base.GetByPattern(prefix + req.Pattern)
matchedConfig := Params.GetByPattern(prefix + req.Pattern)
configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig))
for key, value := range matchedConfig {
configList = append(configList,
@ -114,13 +114,13 @@ func (node *DataNode) getSystemInfoMetrics(ctx context.Context, req *milvuspb.Ge
Name: metricsinfo.ConstructComponentName(typeutil.DataNodeRole, paramtable.GetNodeID()),
HardwareInfos: hardwareMetrics,
SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.DataNodeCfg.CreatedTime.String(),
UpdatedTime: Params.DataNodeCfg.UpdatedTime.String(),
CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.DataNodeRole,
ID: node.session.ServerID,
},
SystemConfigurations: metricsinfo.DataNodeConfiguration{
FlushInsertBufferSize: Params.DataNodeCfg.FlushInsertBufferSize,
FlushInsertBufferSize: Params.DataNodeCfg.FlushInsertBufferSize.GetAsInt64(),
},
QuotaMetrics: quotaMetrics,
}

View File

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

View File

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

View File

@ -117,7 +117,7 @@ func (s *Server) registerHTTPServer() {
// (Embedded Milvus Only) Discard gin logs if logging is disabled.
// 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.
if !proxy.Params.ProxyCfg.GinLogging {
if !proxy.Params.ProxyCfg.GinLogging.GetAsBool() {
gin.DefaultWriter = 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.
// 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)
assert.Nil(t, err)
err = server.Stop()

View File

@ -19,9 +19,11 @@ package grpcquerynode
import (
"context"
"errors"
"os"
"testing"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert"
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) {
ctx := context.Background()
server, err := NewServer(ctx, nil)

View File

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

View File

@ -213,7 +213,7 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
return true
}
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),
zap.Int64("segID", meta.SegmentID), zap.Int64("num rows", meta.NumRows))
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)
var storageConfig *indexpb.StorageConfig
if Params.CommonCfg.StorageType == "local" {
if Params.CommonCfg.StorageType.GetValue() == "local" {
storageConfig = &indexpb.StorageConfig{
RootPath: Params.LocalStorageCfg.Path.GetValue(),
StorageType: Params.CommonCfg.StorageType,
StorageType: Params.CommonCfg.StorageType.GetValue(),
}
} else {
storageConfig = &indexpb.StorageConfig{
@ -289,11 +289,11 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
RootPath: Params.MinioCfg.RootPath.GetValue(),
UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
StorageType: Params.CommonCfg.StorageType,
StorageType: Params.CommonCfg.StorageType.GetValue(),
}
}
req := &indexpb.CreateJobRequest{
ClusterID: Params.CommonCfg.ClusterPrefix,
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
IndexFilePrefix: path.Join(ib.ic.chunkManager.RootPath(), common.SegmentIndexPath),
BuildID: buildID,
DataPaths: binLogs,
@ -390,7 +390,7 @@ func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
defer cancel()
response, err := client.QueryJobs(ctx1, &indexpb.QueryJobsRequest{
ClusterID: Params.CommonCfg.ClusterPrefix,
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
BuildIDs: []int64{buildID},
})
if err != nil {
@ -439,7 +439,7 @@ func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
defer cancel()
status, err := client.DropJobs(ctx1, &indexpb.DropJobsRequest{
ClusterID: Params.CommonCfg.ClusterPrefix,
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
BuildIDs: []UniqueID{buildID},
})
if err != nil {

View File

@ -24,6 +24,7 @@ import (
"time"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/util/paramtable"
"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) {
Params.CommonCfg.StorageType = "local"
paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "local")
ib.tasks[buildID] = indexTaskInit
ib.ic.dataCoordClient = NewDataCoordMock()
ib.meta = createMetaTable(&indexcoord.Catalog{

View File

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

View File

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

View File

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

View File

@ -91,7 +91,7 @@ func (nm *NodeManager) AddNode(nodeID UniqueID, address string) 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 {
log.Error("IndexCoord NodeManager", zap.Any("Add node err", err))
return err

View File

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

View File

@ -232,13 +232,13 @@ func getMockSystemInfoMetrics(
DiskUsage: hardware.GetDiskUsage(),
},
SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.IndexNodeCfg.CreatedTime.String(),
UpdatedTime: Params.IndexNodeCfg.UpdatedTime.String(),
CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.IndexNodeRole,
},
SystemConfigurations: metricsinfo.IndexNodeConfiguration{
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),
TaskSlots: int64(slots),
JobInfos: jobInfos,
EnableDisk: Params.IndexNodeCfg.EnableDisk,
EnableDisk: Params.IndexNodeCfg.EnableDisk.GetAsBool(),
}, nil
}

View File

@ -31,7 +31,7 @@ import (
//getComponentConfigurations returns the configurations of queryNode matching req.Pattern
func getComponentConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse {
prefix := "indexnode."
matchedConfig := Params.IndexNodeCfg.Base.GetByPattern(prefix + req.Pattern)
matchedConfig := Params.GetByPattern(prefix + req.Pattern)
configList := make([]*commonpb.KeyValuePair, 0, len(matchedConfig))
for key, value := range matchedConfig {
configList = append(configList,
@ -70,14 +70,14 @@ func getSystemInfoMetrics(
DiskUsage: hardware.GetDiskUsage(),
},
SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: Params.IndexNodeCfg.CreatedTime.String(),
UpdatedTime: Params.IndexNodeCfg.UpdatedTime.String(),
CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.IndexNodeRole,
ID: node.session.ServerID,
},
SystemConfigurations: metricsinfo.IndexNodeConfiguration{
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 {
// 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",
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")
}
@ -321,7 +321,7 @@ func (it *indexBuildTask) BuildDiskAnnIndex(ctx context.Context) error {
}
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 {
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{
ctx: ctx1,
cancel: cancel,
buildParallel: Params.IndexNodeCfg.BuildParallel,
buildParallel: Params.IndexNodeCfg.BuildParallel.GetAsInt(),
}
s.IndexBuildQueue = NewIndexBuildTaskQueue(s)

View File

@ -19,10 +19,11 @@ var (
func Connect(cfg *paramtable.MetaDBConfig) error {
// 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
if cfg.Base.Log.Level == "debug" {
if cfg.LogLevel.GetValue() == "debug" {
ormLogger = logger.Default.LogMode(logger.Info)
} else {
ormLogger = logger.Default
@ -33,21 +34,33 @@ func Connect(cfg *paramtable.MetaDBConfig) error {
CreateBatchSize: 100,
})
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
}
idb, err := db.DB()
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
}
idb.SetMaxIdleConns(cfg.MaxIdleConns)
idb.SetMaxOpenConns(cfg.MaxOpenConns)
idb.SetMaxIdleConns(cfg.MaxIdleConns.GetAsInt())
idb.SetMaxOpenConns(cfg.MaxOpenConns.GetAsInt())
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
}

View File

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

View File

@ -39,7 +39,7 @@ func TestAccessLogger_NotEnable(t *testing.T) {
defer closer.Close()
Params.Init()
Params.ProxyCfg.AccessLog.Enable = false
Params.Save(Params.ProxyCfg.AccessLog.Enable.Key, "false")
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -73,7 +73,7 @@ func TestAccessLogger_Basic(t *testing.T) {
Params.Init()
testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath
Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
defer os.RemoveAll(testPath)
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -107,7 +107,7 @@ func TestAccessLogger_Stdout(t *testing.T) {
defer closer.Close()
Params.Init()
Params.ProxyCfg.AccessLog.Filename = ""
Params.Save(Params.ProxyCfg.AccessLog.Filename.Key, "")
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -140,9 +140,10 @@ func TestAccessLogger_WithMinio(t *testing.T) {
Params.Init()
testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath
Params.ProxyCfg.AccessLog.MinioEnable = true
Params.ProxyCfg.AccessLog.RemotePath = "access_log/"
Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, testPath)
Params.Save(Params.ProxyCfg.AccessLog.MinioEnable.Key, "true")
Params.Save(Params.ProxyCfg.AccessLog.RemotePath.Key, "access_log/")
Params.Save(Params.ProxyCfg.AccessLog.MaxSize.Key, "1")
defer os.RemoveAll(testPath)
InitAccessLogger(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
@ -185,7 +186,7 @@ func TestAccessLogger_Error(t *testing.T) {
Params.Init()
testPath := "/tmp/accesstest"
Params.ProxyCfg.AccessLog.LocalPath = testPath
Params.Save(Params.ProxyCfg.AccessLog.LocalPath.Key, "testPath")
defer os.RemoveAll(testPath)
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/util/paramtable"
"go.uber.org/zap"
)
const megabyte = 1024 * 1024
@ -66,17 +67,19 @@ type RotateLogger struct {
func NewRotateLogger(logCfg *paramtable.AccessLogConfig, minioCfg *paramtable.MinioConfig) (*RotateLogger, error) {
logger := &RotateLogger{
localPath: logCfg.LocalPath,
fileName: logCfg.Filename,
rotatedTime: logCfg.RotatedTime,
maxSize: logCfg.MaxSize,
maxBackups: logCfg.MaxBackups,
localPath: logCfg.LocalPath.GetValue(),
fileName: logCfg.Filename.GetValue(),
rotatedTime: logCfg.RotatedTime.GetAsInt64(),
maxSize: logCfg.MaxSize.GetAsInt(),
maxBackups: logCfg.MaxBackups.GetAsInt(),
}
log.Info("Access log save to " + logger.dir())
if logCfg.MinioEnable {
if logCfg.MinioEnable.GetAsBool() {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
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 {
return nil, err
}

View File

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

View File

@ -29,7 +29,7 @@ func TestMinioHandler_ConnectError(t *testing.T) {
var params paramtable.ComponentParam
params.Init()
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.Address.Key, "")
defer os.RemoveAll(testPath)
@ -37,8 +37,8 @@ func TestMinioHandler_ConnectError(t *testing.T) {
_, err := NewMinioHandler(
context.Background(),
&params.MinioCfg,
params.ProxyCfg.AccessLog.RemotePath,
params.ProxyCfg.AccessLog.MaxBackups,
params.ProxyCfg.AccessLog.RemotePath.GetValue(),
params.ProxyCfg.AccessLog.MaxBackups.GetAsInt(),
)
assert.Error(t, err)
}

View File

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

View File

@ -9,6 +9,7 @@ import (
"github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/util/crypto"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert"
)
@ -46,7 +47,8 @@ func TestValidSourceID(t *testing.T) {
func TestAuthenticationInterceptor(t *testing.T) {
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
_, err := AuthenticationInterceptor(ctx)
assert.NotNil(t, err)

View File

@ -118,7 +118,7 @@ func (ticker *channelsTimeTickerImpl) tick() error {
} else {
if stat.minTs > current {
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 {
next = stat.maxTs
}

View File

@ -36,7 +36,7 @@ func (d defaultHook) Release() {}
var hoo hook.Hook
func initHook() error {
path := Params.ProxyCfg.SoPath
path := Params.ProxyCfg.SoPath.GetValue()
if path == "" {
hoo = defaultHook{}
return nil
@ -59,7 +59,7 @@ func initHook() error {
if !ok {
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 nil
@ -67,7 +67,7 @@ func initHook() error {
func UnaryServerHookInterceptor() grpc.UnaryServerInterceptor {
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{}
}
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"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert"
)
func TestInitHook(t *testing.T) {
Params.ProxyCfg.SoPath = ""
paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "")
initHook()
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()
assert.NotNil(t, err)
Params.ProxyCfg.SoPath = ""
paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "")
}
type mockHook struct {

View File

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

View File

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

View File

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

View File

@ -17,10 +17,12 @@
package proxy
import (
"fmt"
"math"
"testing"
"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/stretchr/testify/assert"
)
@ -28,7 +30,7 @@ import (
func TestMultiRateLimiter(t *testing.T) {
t.Run("test multiRateLimiter", func(t *testing.T) {
bak := Params.QuotaConfig.QuotaAndLimitsEnabled
Params.QuotaConfig.QuotaAndLimitsEnabled = true
paramtable.Get().Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, "true")
multiLimiter := NewMultiRateLimiter()
for _, rt := range internalpb.RateType_value {
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) {
multiLimiter := NewMultiRateLimiter()
bak := Params.QuotaConfig.QuotaAndLimitsEnabled
Params.QuotaConfig.QuotaAndLimitsEnabled = false
paramtable.Get().Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, "false")
for _, rt := range internalpb.RateType_value {
ok, r := multiLimiter.Limit(internalpb.RateType(rt), 1)
assert.False(t, ok)
@ -59,10 +61,10 @@ func TestMultiRateLimiter(t *testing.T) {
t.Run("test limit", func(t *testing.T) {
run := func(insertRate float64) {
bakInsertRate := Params.QuotaConfig.DMLMaxInsertRate
Params.QuotaConfig.DMLMaxInsertRate = insertRate
paramtable.Get().Save(Params.QuotaConfig.DMLMaxInsertRate.Key, fmt.Sprintf("%f", insertRate))
multiLimiter := NewMultiRateLimiter()
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)
assert.False(t, ok)
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) {
if !Params.CommonCfg.AuthorizationEnabled {
if !Params.CommonCfg.AuthorizationEnabled.GetAsBool() {
return ctx, nil
}
log.Debug("PrivilegeInterceptor", zap.String("type", reflect.TypeOf(req).String()))

View File

@ -5,6 +5,7 @@ import (
"testing"
"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/milvuspb"
@ -20,7 +21,7 @@ func TestUnaryServerInterceptor(t *testing.T) {
func TestPrivilegeInterceptor(t *testing.T) {
ctx := context.Background()
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{
DbName: "db_test",
CollectionName: "col1",
@ -29,7 +30,7 @@ func TestPrivilegeInterceptor(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{})
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))
syncTimeTickInterval := Params.ProxyCfg.TimeTickInterval / 2
syncTimeTickInterval := Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond) / 2
log.Debug("create channels time ticker",
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 metrics cache manager", zap.String("role", typeutil.ProxyRole))
@ -269,7 +269,7 @@ func (node *Proxy) sendChannelsTimeTickLoop() {
go func() {
defer node.wg.Done()
timer := time.NewTicker(Params.ProxyCfg.TimeTickInterval)
timer := time.NewTicker(Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond))
for {
select {
@ -365,10 +365,6 @@ func (node *Proxy) Start() error {
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()))
node.UpdateStateCode(commonpb.StateCode_Healthy)

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -11,7 +11,6 @@ import (
"time"
"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/paramtable"
"github.com/stretchr/testify/assert"
@ -1897,13 +1896,17 @@ func TestTaskSearch_parseSearchParams_AutoIndexEnable(t *testing.T) {
oldIndexType := Params.AutoIndexConfig.IndexType
oldIndexParams := Params.AutoIndexConfig.IndexParams
oldSearchParamYamStr := Params.AutoIndexConfig.SearchParamsYamlStr
oldParser := Params.AutoIndexConfig.Parser
//parseSearchParams
Params.AutoIndexConfig.Enable = true
Params.AutoIndexConfig.IndexType = indexparamcheck.IndexHNSW
Params.AutoIndexConfig.IndexParams = make(map[string]string)
paramtable.Get().Save(Params.AutoIndexConfig.Enable.Key, "true")
paramtable.Get().Save(Params.AutoIndexConfig.IndexType.Key, indexparamcheck.IndexHNSW)
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.IndexTypeKey: indexparamcheck.IndexHNSW,
"M": 8,
@ -1911,7 +1914,8 @@ func TestTaskSearch_parseSearchParams_AutoIndexEnable(t *testing.T) {
}
buildParamsJSONValue, err := json.Marshal(buildParams)
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)
jsonStr := `
@ -1941,8 +1945,8 @@ func TestTaskSearch_parseSearchParams_AutoIndexEnable(t *testing.T) {
]
}
}`
Params.AutoIndexConfig.Parser = autoindex.NewParser()
Params.AutoIndexConfig.Parser.InitFromJSONStr(jsonStr)
paramtable.Get().Save(Params.AutoIndexConfig.SearchParamsYamlStr.Key, jsonStr)
defer paramtable.Get().Reset(Params.AutoIndexConfig.SearchParamsYamlStr.Key)
normalKVPairs := []*commonpb.KeyValuePair{
{
@ -2085,7 +2089,6 @@ func TestTaskSearch_parseSearchParams_AutoIndexEnable(t *testing.T) {
Params.AutoIndexConfig.IndexType = oldIndexType
Params.AutoIndexConfig.IndexParams = oldIndexParams
Params.AutoIndexConfig.SearchParamsYamlStr = oldSearchParamYamStr
Params.AutoIndexConfig.Parser = oldParser
}

View File

@ -570,7 +570,7 @@ func TestCreateCollectionTask(t *testing.T) {
assert.Error(t, err)
task.Schema = marshaledSchema
task.ShardsNum = Params.ProxyCfg.MaxShardNum + 1
task.ShardsNum = Params.ProxyCfg.MaxShardNum.GetAsInt32() + 1
err = task.PreExecute(ctx)
assert.Error(t, err)
task.ShardsNum = shardsNum
@ -582,7 +582,7 @@ func TestCreateCollectionTask(t *testing.T) {
Name: collectionName,
Description: "",
AutoID: false,
Fields: make([]*schemapb.FieldSchema, Params.ProxyCfg.MaxFieldNum+1),
Fields: make([]*schemapb.FieldSchema, Params.ProxyCfg.MaxFieldNum.GetAsInt32()+1),
}
marshaledSchemaWithTooManyFields, err := proto.Marshal(schemaWithTooManyFields)
assert.NoError(t, err)
@ -602,7 +602,7 @@ func TestCreateCollectionTask(t *testing.T) {
assert.Error(t, err)
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)
}
tooLongNameSchema, err := proto.Marshal(schema)
@ -701,7 +701,7 @@ func TestCreateCollectionTask(t *testing.T) {
schema.Fields[idx].TypeParams = []*commonpb.KeyValuePair{
{
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{
{
Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension) + 1),
Value: strconv.Itoa(Params.ProxyCfg.MaxDimension.GetAsInt() + 1),
},
}
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)
if int64(len(entity)) > Params.ProxyCfg.MaxNameLength {
msg := invalidMsg + fmt.Sprintf("The length of a collection %s must be less than ", entityType) +
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
if len(entity) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + fmt.Sprintf("The length of a collection %s must be less than ", entityType) + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
return errors.New(msg)
}
@ -139,9 +138,8 @@ func validatePartitionTag(partitionTag string, strictCheck bool) error {
return errors.New(msg)
}
if int64(len(partitionTag)) > Params.ProxyCfg.MaxNameLength {
msg := invalidMsg + "The length of a partition name must be less than " +
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
if len(partitionTag) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + "The length of a partition name must be less than " + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
return errors.New(msg)
}
@ -173,9 +171,8 @@ func validateFieldName(fieldName string) error {
}
invalidMsg := "Invalid field name: " + fieldName + ". "
if int64(len(fieldName)) > Params.ProxyCfg.MaxNameLength {
msg := invalidMsg + "The length of a field name must be less than " +
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
if len(fieldName) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + "The length of a field name must be less than " + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
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")
}
if dim <= 0 || dim > Params.ProxyCfg.MaxDimension {
return fmt.Errorf("invalid dimension: %d. should be in range 1 ~ %d", 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.GetAsInt())
}
if field.DataType == schemapb.DataType_BinaryVector && dim%8 != 0 {
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 + ". "
if int64(len(username)) > Params.ProxyCfg.MaxUsernameLength {
msg := invalidMsg + "The length of username must be less than " +
strconv.FormatInt(Params.ProxyCfg.MaxUsernameLength, 10) + " characters."
if len(username) > Params.ProxyCfg.MaxUsernameLength.GetAsInt() {
msg := invalidMsg + "The length of username must be less than " + Params.ProxyCfg.MaxUsernameLength.GetValue() + " characters."
return errors.New(msg)
}
@ -590,9 +586,9 @@ func ValidateUsername(username string) error {
}
func ValidatePassword(password string) error {
if int64(len(password)) < Params.ProxyCfg.MinPasswordLength || int64(len(password)) > Params.ProxyCfg.MaxPasswordLength {
msg := "The length of password must be great than " + strconv.FormatInt(Params.ProxyCfg.MinPasswordLength, 10) +
" and less than " + strconv.FormatInt(Params.ProxyCfg.MaxPasswordLength, 10) + " characters."
if len(password) < Params.ProxyCfg.MinPasswordLength.GetAsInt() || len(password) > Params.ProxyCfg.MaxPasswordLength.GetAsInt() {
msg := "The length of password must be great than " + Params.ProxyCfg.MinPasswordLength.GetValue() +
" and less than " + Params.ProxyCfg.MaxPasswordLength.GetValue() + " characters."
return errors.New(msg)
}
return nil
@ -600,10 +596,10 @@ func ValidatePassword(password string) error {
func validateTravelTimestamp(travelTs, tMax typeutil.Timestamp) error {
durationSeconds := tsoutil.CalculateDuration(tMax, travelTs) / 1000
if durationSeconds > Params.CommonCfg.RetentionDuration {
if durationSeconds > Params.CommonCfg.RetentionDuration.GetAsInt64() {
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 nil
@ -618,7 +614,7 @@ func parseGuaranteeTs(ts, tMax typeutil.Timestamp) typeutil.Timestamp {
case strongTS:
ts = tMax
case boundedTS:
ratio := time.Duration(-Params.CommonCfg.GracefulTime)
ratio := time.Duration(-Params.CommonCfg.GracefulTime.GetAsInt64())
ts = tsoutil.AddPhysicalDurationOnTs(tMax, ratio*time.Millisecond)
}
return ts
@ -632,9 +628,8 @@ func validateName(entity string, nameType string) error {
}
invalidMsg := fmt.Sprintf("invalid %s: %s. ", nameType, entity)
if int64(len(entity)) > Params.ProxyCfg.MaxNameLength {
msg := invalidMsg + fmt.Sprintf("the length of %s must be less than ", nameType) +
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
if len(entity) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + fmt.Sprintf("the length of %s must be less than ", nameType) + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
return errors.New(msg)
}
@ -813,9 +808,8 @@ func validateIndexName(indexName string) error {
return nil
}
invalidMsg := "Invalid index name: " + indexName + ". "
if int64(len(indexName)) > Params.ProxyCfg.MaxNameLength {
msg := invalidMsg + "The length of a index name must be less than " +
strconv.FormatInt(Params.ProxyCfg.MaxNameLength, 10) + " characters."
if len(indexName) > Params.ProxyCfg.MaxNameLength.GetAsInt() {
msg := invalidMsg + "The length of a index name must be less than " + Params.ProxyCfg.MaxNameLength.GetValue() + " characters."
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/util"
"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/typeutil"
)
@ -133,7 +134,7 @@ func TestValidateDimension(t *testing.T) {
fieldSchema.TypeParams = []*commonpb.KeyValuePair{
{
Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension)),
Value: Params.ProxyCfg.MaxDimension.GetValue(),
},
}
assert.Nil(t, validateDimension(fieldSchema))
@ -149,7 +150,7 @@ func TestValidateDimension(t *testing.T) {
fieldSchema.TypeParams = []*commonpb.KeyValuePair{
{
Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension + 1)),
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension.GetAsInt32() + 1)),
},
}
assert.NotNil(t, validateDimension(fieldSchema))
@ -165,7 +166,7 @@ func TestValidateDimension(t *testing.T) {
fieldSchema.TypeParams = []*commonpb.KeyValuePair{
{
Key: "dim",
Value: strconv.Itoa(int(Params.ProxyCfg.MaxDimension)),
Value: strconv.Itoa(Params.ProxyCfg.MaxDimension.GetAsInt()),
},
}
assert.Nil(t, validateDimension(fieldSchema))
@ -784,18 +785,19 @@ func TestValidateTravelTimestamp(t *testing.T) {
travelTs := tsoutil.GetCurrentTime()
tests := []struct {
description string
defaultRD int64
defaultRD string
nowTs typeutil.Timestamp
isValid bool
}{
{"one second", 100, tsoutil.AddPhysicalDurationOnTs(travelTs, time.Second), true},
{"retention duration", 100, tsoutil.AddPhysicalDurationOnTs(travelTs, 100*time.Second), true},
{"retention duration+1", 100, tsoutil.AddPhysicalDurationOnTs(travelTs, 101*time.Second), false},
{"one second", "100", tsoutil.AddPhysicalDurationOnTs(travelTs, time.Second), true},
{"retention duration", "100", tsoutil.AddPhysicalDurationOnTs(travelTs, 100*time.Second), true},
{"retention duration+1", "100", tsoutil.AddPhysicalDurationOnTs(travelTs, 101*time.Second), false},
}
for _, test := range tests {
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)
if test.isValid {
assert.NoError(t, err)

View File

@ -18,6 +18,7 @@ package checkers
import (
"context"
"time"
"github.com/milvus-io/milvus/internal/querycoordv2/balance"
. "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)
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 {
if t.Priority() == task.TaskPriorityHigh {
return task.TaskPriorityHigh
@ -53,7 +54,7 @@ func (b *BalanceChecker) Check(ctx context.Context) []task.Task {
}, 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...)
return ret
}

View File

@ -18,6 +18,7 @@ package checkers
import (
"context"
"time"
"github.com/milvus-io/milvus/internal/log"
"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",
// zap.Any("channels", channels),
// 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 {
ret := make([]task.Task, 0, len(channels))
for _, ch := range channels {
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 {
log.Warn("Create channel reduce task failed",
zap.Int64("collection", ch.GetCollectionID()),

View File

@ -80,7 +80,7 @@ func NewCheckerController(
func (controller *CheckerController) Start(ctx context.Context) {
go func() {
ticker := time.NewTicker(Params.QueryCoordCfg.CheckInterval)
ticker := time.NewTicker(Params.QueryCoordCfg.CheckInterval.GetAsDuration(time.Millisecond))
defer ticker.Stop()
for {
select {
@ -98,7 +98,7 @@ func (controller *CheckerController) Start(ctx context.Context) {
case <-controller.checkCh:
ticker.Stop()
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 (
"context"
"time"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
@ -269,7 +270,7 @@ func (c *SegmentChecker) createSegmentLoadTasks(ctx context.Context, segments []
for i := range plans {
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 {
@ -278,7 +279,7 @@ func (c *SegmentChecker) createSegmentReduceTasks(ctx context.Context, segments
action := task.NewSegmentActionWithScope(s.Node, task.ActionTypeReduce, s.GetInsertChannel(), s.GetID(), scope)
task, err := task.NewSegmentTask(
ctx,
Params.QueryCoordCfg.SegmentTaskTimeout,
Params.QueryCoordCfg.SegmentTaskTimeout.GetAsDuration(time.Millisecond),
c.ID(),
s.GetCollectionID(),
replicaID,

View File

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

View File

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

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