mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
enhance:Limit the number of concurrent vector index builds per worker (#46773)
issue: #46772 --------- Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>
This commit is contained in:
parent
68bd064228
commit
0c200ff781
65
internal/datanode/index/pool.go
Normal file
65
internal/datanode/index/pool.go
Normal file
@ -0,0 +1,65 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package index
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/config"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/conc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
)
|
||||
|
||||
var (
|
||||
vecIndexBuildPool *conc.Pool[any]
|
||||
vecIndexBuildPoolInitOnce sync.Once
|
||||
)
|
||||
|
||||
func initVecIndexBuildPool() {
|
||||
pt := paramtable.Get()
|
||||
initPoolSize := pt.DataNodeCfg.MaxVecIndexBuildConcurrency.GetAsInt()
|
||||
vecIndexBuildPool = conc.NewPool[any](
|
||||
initPoolSize,
|
||||
)
|
||||
|
||||
watchKey := pt.DataNodeCfg.MaxVecIndexBuildConcurrency.Key
|
||||
pt.Watch(watchKey, config.NewHandler(watchKey, resizeVecIndexBuildPool))
|
||||
log.Info("init vector index building pool done", zap.Int("size", initPoolSize))
|
||||
}
|
||||
|
||||
func resizeVecIndexBuildPool(evt *config.Event) {
|
||||
if evt.HasUpdated {
|
||||
newSize := paramtable.Get().DataNodeCfg.MaxVecIndexBuildConcurrency.GetAsInt()
|
||||
log := log.Ctx(context.Background()).With(zap.Int("newSize", newSize))
|
||||
|
||||
err := GetVecIndexBuildPool().Resize(newSize)
|
||||
if err != nil {
|
||||
log.Warn("failed to resize pool", zap.Error(err))
|
||||
return
|
||||
}
|
||||
log.Info("vector index building pool resize successfully")
|
||||
}
|
||||
}
|
||||
|
||||
func GetVecIndexBuildPool() *conc.Pool[any] {
|
||||
vecIndexBuildPoolInitOnce.Do(initVecIndexBuildPool)
|
||||
return vecIndexBuildPool
|
||||
}
|
||||
64
internal/datanode/index/pool_test.go
Normal file
64
internal/datanode/index/pool_test.go
Normal file
@ -0,0 +1,64 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package index
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/config"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
)
|
||||
|
||||
func TestResizePools(t *testing.T) {
|
||||
paramtable.Get().Init(paramtable.NewBaseTable(paramtable.SkipRemote(true)))
|
||||
pt := paramtable.Get()
|
||||
|
||||
defer func() {
|
||||
_ = pt.Reset(pt.DataNodeCfg.MaxVecIndexBuildConcurrency.Key)
|
||||
}()
|
||||
|
||||
t.Run("GetVecIndexBuildPool", func(t *testing.T) {
|
||||
expectedCap := pt.DataNodeCfg.MaxVecIndexBuildConcurrency.GetAsInt()
|
||||
assert.Equal(t, expectedCap, GetVecIndexBuildPool().Cap())
|
||||
resizeVecIndexBuildPool(&config.Event{
|
||||
HasUpdated: true,
|
||||
})
|
||||
assert.Equal(t, expectedCap, GetVecIndexBuildPool().Cap())
|
||||
|
||||
_ = pt.Save(pt.DataNodeCfg.MaxVecIndexBuildConcurrency.Key, fmt.Sprintf("%d", expectedCap*2))
|
||||
expectedCap = pt.DataNodeCfg.MaxVecIndexBuildConcurrency.GetAsInt()
|
||||
resizeVecIndexBuildPool(&config.Event{
|
||||
HasUpdated: true,
|
||||
})
|
||||
assert.Equal(t, expectedCap, GetVecIndexBuildPool().Cap())
|
||||
|
||||
_ = pt.Save(pt.DataNodeCfg.MaxVecIndexBuildConcurrency.Key, "0")
|
||||
resizeVecIndexBuildPool(&config.Event{
|
||||
HasUpdated: true,
|
||||
})
|
||||
assert.Equal(t, expectedCap, GetVecIndexBuildPool().Cap(), "pool shall not be resized when newSize is 0")
|
||||
|
||||
_ = pt.Save(pt.DataNodeCfg.MaxVecIndexBuildConcurrency.Key, "invalid")
|
||||
resizeVecIndexBuildPool(&config.Event{
|
||||
HasUpdated: true,
|
||||
})
|
||||
assert.Equal(t, expectedCap, GetVecIndexBuildPool().Cap())
|
||||
})
|
||||
}
|
||||
@ -21,7 +21,6 @@ import (
|
||||
"context"
|
||||
"runtime/debug"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/atomic"
|
||||
@ -227,7 +226,7 @@ func getStateFromError(err error) indexpb.JobState {
|
||||
return indexpb.JobState_JobStateRetry
|
||||
}
|
||||
|
||||
func (sched *TaskScheduler) processTask(t Task, q TaskQueue) {
|
||||
func (sched *TaskScheduler) processTask(t Task) {
|
||||
wrap := func(fn func(ctx context.Context) error) error {
|
||||
select {
|
||||
case <-t.Ctx().Done():
|
||||
@ -264,17 +263,16 @@ func (sched *TaskScheduler) indexBuildLoop() {
|
||||
return
|
||||
case <-sched.TaskQueue.utChan():
|
||||
t := sched.TaskQueue.PopUnissuedTask()
|
||||
for {
|
||||
totalSlot := CalculateNodeSlots()
|
||||
availableSlot := totalSlot - sched.TaskQueue.GetActiveSlot()
|
||||
if availableSlot >= t.GetSlot() || totalSlot == availableSlot {
|
||||
go func(t Task) {
|
||||
sched.processTask(t, sched.TaskQueue)
|
||||
}(t)
|
||||
break
|
||||
go func(t Task) {
|
||||
if t.IsVectorIndex() {
|
||||
GetVecIndexBuildPool().Submit(func() (any, error) {
|
||||
sched.processTask(t)
|
||||
return nil, nil
|
||||
})
|
||||
} else {
|
||||
sched.processTask(t)
|
||||
}
|
||||
time.Sleep(time.Millisecond * 50)
|
||||
}
|
||||
}(t)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -152,6 +152,10 @@ func (t *fakeTask) GetState() indexpb.JobState {
|
||||
return t.retstate
|
||||
}
|
||||
|
||||
func (t *fakeTask) IsVectorIndex() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
var (
|
||||
idLock sync.Mutex
|
||||
id = 0
|
||||
|
||||
@ -46,4 +46,5 @@ type Task interface {
|
||||
PostExecute(context.Context) error
|
||||
Reset()
|
||||
GetSlot() int64
|
||||
IsVectorIndex() bool
|
||||
}
|
||||
|
||||
@ -75,6 +75,10 @@ func (at *analyzeTask) GetSlot() int64 {
|
||||
return at.req.GetTaskSlot()
|
||||
}
|
||||
|
||||
func (at *analyzeTask) IsVectorIndex() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
func (at *analyzeTask) PreExecute(ctx context.Context) error {
|
||||
at.queueDur = at.tr.RecordSpan()
|
||||
log := log.Ctx(ctx).With(zap.String("clusterID", at.req.GetClusterID()),
|
||||
|
||||
@ -143,6 +143,11 @@ func (it *indexBuildTask) GetSlot() int64 {
|
||||
return it.req.GetTaskSlot()
|
||||
}
|
||||
|
||||
func (it *indexBuildTask) IsVectorIndex() bool {
|
||||
indexType := GetIndexType(it.req.GetIndexParams())
|
||||
return vecindexmgr.GetVecIndexMgrInstance().IsVecIndex(indexType)
|
||||
}
|
||||
|
||||
func (it *indexBuildTask) PreExecute(ctx context.Context) error {
|
||||
it.queueDur = it.tr.RecordSpan()
|
||||
log.Ctx(ctx).Info("Begin to prepare indexBuildTask", zap.Int64("buildID", it.req.GetBuildID()),
|
||||
|
||||
@ -136,6 +136,10 @@ func (st *statsTask) GetSlot() int64 {
|
||||
return st.req.GetTaskSlot()
|
||||
}
|
||||
|
||||
func (st *statsTask) IsVectorIndex() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
func (st *statsTask) PreExecute(ctx context.Context) error {
|
||||
ctx, span := otel.Tracer(typeutil.IndexNodeRole).Start(ctx, fmt.Sprintf("Stats-PreExecute-%s-%d", st.req.GetClusterID(), st.req.GetTaskID()))
|
||||
defer span.End()
|
||||
|
||||
@ -96,3 +96,12 @@ func CalculateNodeSlots() int64 {
|
||||
}
|
||||
return totalSlot
|
||||
}
|
||||
|
||||
func GetIndexType(indexParams []*commonpb.KeyValuePair) string {
|
||||
for _, param := range indexParams {
|
||||
if param.Key == common.IndexTypeKey {
|
||||
return param.Value
|
||||
}
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
@ -6028,7 +6028,8 @@ type dataNodeConfig struct {
|
||||
DeltalogFormat ParamItem `refreshable:"false"`
|
||||
|
||||
// index services config
|
||||
BuildParallel ParamItem `refreshable:"false"`
|
||||
BuildParallel ParamItem `refreshable:"false"`
|
||||
MaxVecIndexBuildConcurrency ParamItem `refreshable:"true"`
|
||||
|
||||
WorkerSlotUnit ParamItem `refreshable:"true"`
|
||||
StandaloneSlotRatio ParamItem `refreshable:"false"`
|
||||
@ -6469,6 +6470,14 @@ if this parameter <= 0, will set it as 10`,
|
||||
}
|
||||
p.BuildParallel.Init(base.mgr)
|
||||
|
||||
p.MaxVecIndexBuildConcurrency = ParamItem{
|
||||
Key: "dataNode.index.maxVecIndexBuildConcurrency",
|
||||
Version: "2.6.9",
|
||||
DefaultValue: "4",
|
||||
Export: false,
|
||||
}
|
||||
p.MaxVecIndexBuildConcurrency.Init(base.mgr)
|
||||
|
||||
p.WorkerSlotUnit = ParamItem{
|
||||
Key: "dataNode.workerSlotUnit",
|
||||
Version: "2.5.7",
|
||||
|
||||
@ -669,6 +669,8 @@ func TestComponentParam(t *testing.T) {
|
||||
// compaction
|
||||
assert.Equal(t, 10, Params.MaxCompactionConcurrency.GetAsInt())
|
||||
|
||||
assert.Equal(t, 4, Params.MaxVecIndexBuildConcurrency.GetAsInt())
|
||||
|
||||
// clustering compaction
|
||||
params.Save("datanode.clusteringCompaction.memoryBufferRatio", "0.1")
|
||||
assert.Equal(t, 0.1, Params.ClusteringCompactionMemoryBufferRatio.GetAsFloat())
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user