mirror of https://github.com/milvus-io/milvus.git
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
parent
80a2a49681
commit
89b810a4db
|
@ -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,
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -79,13 +79,11 @@ func (c *MilvusConfig) init(base *paramtable.BaseTable) {
|
|||
c.EtcdCfg = ¶mtable.EtcdConfig{}
|
||||
c.MysqlCfg = ¶mtable.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())
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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})
|
||||
}
|
||||
|
|
|
@ -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("")
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
|
|
@ -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
|
||||
})
|
||||
|
||||
|
|
|
@ -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}
|
||||
})
|
||||
|
||||
|
|
|
@ -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(),
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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,
|
||||
},
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
@ -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,
|
||||
},
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()))
|
||||
})
|
||||
|
|
|
@ -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(),
|
||||
},
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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(),
|
||||
},
|
||||
}
|
||||
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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(¶ms.ProxyCfg.AccessLog, ¶ms.MinioCfg)
|
||||
|
|
|
@ -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(),
|
||||
¶ms.MinioCfg,
|
||||
params.ProxyCfg.AccessLog.RemotePath,
|
||||
params.ProxyCfg.AccessLog.MaxBackups,
|
||||
params.ProxyCfg.AccessLog.RemotePath.GetValue(),
|
||||
params.ProxyCfg.AccessLog.MaxBackups.GetAsInt(),
|
||||
)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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(),
|
||||
},
|
||||
},
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()))
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue