mirror of https://github.com/milvus-io/milvus.git
fix: Block warmup submit if pool full in sync mode (#38690)
https://github.com/milvus-io/milvus/issues/38692 --------- Signed-off-by: sunby <sunbingyi1992@gmail.com>pull/38940/head
parent
d3a5282eaa
commit
aa0a87eda7
|
@ -102,6 +102,7 @@ MmapChunkTarget::get() {
|
|||
write(padding, padding_size);
|
||||
|
||||
flush();
|
||||
file_.FFlush();
|
||||
|
||||
auto m = mmap(
|
||||
nullptr, size_, PROT_READ, MAP_SHARED, file_.Descriptor(), offset_);
|
||||
|
|
|
@ -116,6 +116,7 @@ func (sd *shardDelegator) ProcessInsert(insertRecords map[int64]*InsertData) {
|
|||
DeltaPosition: insertData.StartPosition,
|
||||
Level: datapb.SegmentLevel_L1,
|
||||
},
|
||||
nil,
|
||||
)
|
||||
if err != nil {
|
||||
log.Error("failed to create new segment",
|
||||
|
|
|
@ -66,6 +66,7 @@ func (s *ManagerSuite) SetupTest() {
|
|||
InsertChannel: s.channels[i],
|
||||
Level: s.levels[i],
|
||||
},
|
||||
nil,
|
||||
)
|
||||
s.Require().NoError(err)
|
||||
s.segments = append(s.segments, segment)
|
||||
|
|
|
@ -142,7 +142,7 @@ func initWarmupPool() {
|
|||
runtime.LockOSThread()
|
||||
C.SetThreadName(cgoTagWarmup)
|
||||
}), // lock os thread for cgo thread disposal
|
||||
conc.WithNonBlocking(true), // make warming up non blocking
|
||||
conc.WithNonBlocking(false),
|
||||
)
|
||||
|
||||
warmupPool.Store(pool)
|
||||
|
|
|
@ -98,6 +98,7 @@ func (suite *RetrieveSuite) SetupTest() {
|
|||
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
|
||||
Level: datapb.SegmentLevel_Legacy,
|
||||
},
|
||||
nil,
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
|
@ -126,6 +127,7 @@ func (suite *RetrieveSuite) SetupTest() {
|
|||
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
|
||||
Level: datapb.SegmentLevel_Legacy,
|
||||
},
|
||||
nil,
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
|
|
|
@ -88,6 +88,7 @@ func (suite *SearchSuite) SetupTest() {
|
|||
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
|
||||
Level: datapb.SegmentLevel_Legacy,
|
||||
},
|
||||
nil,
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
|
@ -116,6 +117,7 @@ func (suite *SearchSuite) SetupTest() {
|
|||
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
|
||||
Level: datapb.SegmentLevel_Legacy,
|
||||
},
|
||||
nil,
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
|
|
|
@ -30,6 +30,7 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
"unsafe"
|
||||
|
||||
|
@ -279,6 +280,7 @@ type LocalSegment struct {
|
|||
lastDeltaTimestamp *atomic.Uint64
|
||||
fields *typeutil.ConcurrentMap[int64, *FieldInfo]
|
||||
fieldIndexes *typeutil.ConcurrentMap[int64, *IndexedFieldInfo]
|
||||
warmupDispatcher *AsyncWarmupDispatcher
|
||||
}
|
||||
|
||||
func NewSegment(ctx context.Context,
|
||||
|
@ -286,6 +288,7 @@ func NewSegment(ctx context.Context,
|
|||
segmentType SegmentType,
|
||||
version int64,
|
||||
loadInfo *querypb.SegmentLoadInfo,
|
||||
warmupDispatcher *AsyncWarmupDispatcher,
|
||||
) (Segment, error) {
|
||||
log := log.Ctx(ctx)
|
||||
/*
|
||||
|
@ -345,9 +348,10 @@ func NewSegment(ctx context.Context,
|
|||
fields: typeutil.NewConcurrentMap[int64, *FieldInfo](),
|
||||
fieldIndexes: typeutil.NewConcurrentMap[int64, *IndexedFieldInfo](),
|
||||
|
||||
memSize: atomic.NewInt64(-1),
|
||||
rowNum: atomic.NewInt64(-1),
|
||||
insertCount: atomic.NewInt64(0),
|
||||
memSize: atomic.NewInt64(-1),
|
||||
rowNum: atomic.NewInt64(-1),
|
||||
insertCount: atomic.NewInt64(0),
|
||||
warmupDispatcher: warmupDispatcher,
|
||||
}
|
||||
|
||||
if err := segment.initializeSegment(); err != nil {
|
||||
|
@ -1157,7 +1161,7 @@ func (s *LocalSegment) WarmupChunkCache(ctx context.Context, fieldID int64, mmap
|
|||
return nil, nil
|
||||
}).Await()
|
||||
case "async":
|
||||
GetWarmupPool().Submit(func() (any, error) {
|
||||
task := func() (any, error) {
|
||||
// bad implemtation, warmup is async at another goroutine and hold the rlock.
|
||||
// the state transition of segment in segment loader will blocked.
|
||||
// add a waiter to avoid it.
|
||||
|
@ -1176,7 +1180,8 @@ func (s *LocalSegment) WarmupChunkCache(ctx context.Context, fieldID int64, mmap
|
|||
}
|
||||
log.Info("warming up chunk cache asynchronously done")
|
||||
return nil, nil
|
||||
})
|
||||
}
|
||||
s.warmupDispatcher.AddTask(task)
|
||||
default:
|
||||
// no warming up
|
||||
}
|
||||
|
@ -1347,3 +1352,55 @@ func (s *LocalSegment) indexNeedLoadRawData(schema *schemapb.CollectionSchema, i
|
|||
}
|
||||
return !typeutil.IsVectorType(fieldSchema.DataType) && s.HasRawData(indexInfo.IndexInfo.FieldID), nil
|
||||
}
|
||||
|
||||
type (
|
||||
WarmupTask = func() (any, error)
|
||||
AsyncWarmupDispatcher struct {
|
||||
mu sync.RWMutex
|
||||
tasks []WarmupTask
|
||||
notify chan struct{}
|
||||
}
|
||||
)
|
||||
|
||||
func NewWarmupDispatcher() *AsyncWarmupDispatcher {
|
||||
return &AsyncWarmupDispatcher{
|
||||
notify: make(chan struct{}, 1),
|
||||
}
|
||||
}
|
||||
|
||||
func (d *AsyncWarmupDispatcher) AddTask(task func() (any, error)) {
|
||||
d.mu.Lock()
|
||||
d.tasks = append(d.tasks, task)
|
||||
d.mu.Unlock()
|
||||
select {
|
||||
case d.notify <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
func (d *AsyncWarmupDispatcher) Run(ctx context.Context) {
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-d.notify:
|
||||
d.mu.RLock()
|
||||
tasks := make([]WarmupTask, len(d.tasks))
|
||||
copy(tasks, d.tasks)
|
||||
d.mu.RUnlock()
|
||||
|
||||
for _, task := range tasks {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
default:
|
||||
GetDynamicPool().Submit(task)
|
||||
}
|
||||
}
|
||||
|
||||
d.mu.Lock()
|
||||
d.tasks = d.tasks[len(tasks):]
|
||||
d.mu.Unlock()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -147,6 +147,7 @@ type resourceEstimateFactor struct {
|
|||
}
|
||||
|
||||
func NewLoader(
|
||||
ctx context.Context,
|
||||
manager *Manager,
|
||||
cm storage.ChunkManager,
|
||||
) *segmentLoader {
|
||||
|
@ -167,11 +168,14 @@ func NewLoader(
|
|||
|
||||
log.Info("SegmentLoader created", zap.Int("ioPoolSize", ioPoolSize))
|
||||
|
||||
warmupDispatcher := NewWarmupDispatcher()
|
||||
go warmupDispatcher.Run(ctx)
|
||||
loader := &segmentLoader{
|
||||
manager: manager,
|
||||
cm: cm,
|
||||
loadingSegments: typeutil.NewConcurrentMap[int64, *loadResult](),
|
||||
committedResourceNotifier: syncutil.NewVersionedNotifier(),
|
||||
warmupDispatcher: warmupDispatcher,
|
||||
}
|
||||
|
||||
return loader
|
||||
|
@ -212,6 +216,8 @@ type segmentLoader struct {
|
|||
loadingSegments *typeutil.ConcurrentMap[int64, *loadResult]
|
||||
committedResource LoadResource
|
||||
committedResourceNotifier *syncutil.VersionedNotifier
|
||||
|
||||
warmupDispatcher *AsyncWarmupDispatcher
|
||||
}
|
||||
|
||||
var _ Loader = (*segmentLoader)(nil)
|
||||
|
@ -294,6 +300,7 @@ func (loader *segmentLoader) Load(ctx context.Context,
|
|||
segmentType,
|
||||
version,
|
||||
loadInfo,
|
||||
loader.warmupDispatcher,
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("load segment failed when create new segment",
|
||||
|
|
|
@ -81,7 +81,7 @@ func (suite *SegmentLoaderSuite) SetupTest() {
|
|||
|
||||
// Dependencies
|
||||
suite.manager = NewManager()
|
||||
suite.loader = NewLoader(suite.manager, suite.chunkManager)
|
||||
suite.loader = NewLoader(ctx, suite.manager, suite.chunkManager)
|
||||
initcore.InitRemoteChunkManager(paramtable.Get())
|
||||
|
||||
// Data
|
||||
|
@ -98,7 +98,7 @@ func (suite *SegmentLoaderSuite) SetupTest() {
|
|||
func (suite *SegmentLoaderSuite) SetupBM25() {
|
||||
// Dependencies
|
||||
suite.manager = NewManager()
|
||||
suite.loader = NewLoader(suite.manager, suite.chunkManager)
|
||||
suite.loader = NewLoader(context.Background(), suite.manager, suite.chunkManager)
|
||||
initcore.InitRemoteChunkManager(paramtable.Get())
|
||||
|
||||
suite.schema = mock_segcore.GenTestBM25CollectionSchema("test")
|
||||
|
@ -798,7 +798,7 @@ func (suite *SegmentLoaderDetailSuite) SetupTest() {
|
|||
ctx := context.Background()
|
||||
chunkManagerFactory := storage.NewTestChunkManagerFactory(paramtable.Get(), suite.rootPath)
|
||||
suite.chunkManager, _ = chunkManagerFactory.NewPersistentStorageChunkManager(ctx)
|
||||
suite.loader = NewLoader(suite.manager, suite.chunkManager)
|
||||
suite.loader = NewLoader(ctx, suite.manager, suite.chunkManager)
|
||||
initcore.InitRemoteChunkManager(paramtable.Get())
|
||||
|
||||
// Data
|
||||
|
|
|
@ -5,8 +5,11 @@ import (
|
|||
"fmt"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"go.uber.org/atomic"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/mocks/util/mock_segcore"
|
||||
|
@ -91,6 +94,7 @@ func (suite *SegmentSuite) SetupTest() {
|
|||
},
|
||||
},
|
||||
},
|
||||
nil,
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
|
@ -122,6 +126,7 @@ func (suite *SegmentSuite) SetupTest() {
|
|||
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", suite.collectionID),
|
||||
Level: datapb.SegmentLevel_Legacy,
|
||||
},
|
||||
nil,
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
|
@ -222,3 +227,22 @@ func (suite *SegmentSuite) TestSegmentReleased() {
|
|||
func TestSegment(t *testing.T) {
|
||||
suite.Run(t, new(SegmentSuite))
|
||||
}
|
||||
|
||||
func TestWarmupDispatcher(t *testing.T) {
|
||||
d := NewWarmupDispatcher()
|
||||
ctx := context.Background()
|
||||
go d.Run(ctx)
|
||||
|
||||
completed := atomic.NewInt64(0)
|
||||
taskCnt := 10000
|
||||
for i := 0; i < taskCnt; i++ {
|
||||
d.AddTask(func() (any, error) {
|
||||
completed.Inc()
|
||||
return nil, nil
|
||||
})
|
||||
}
|
||||
|
||||
assert.Eventually(t, func() bool {
|
||||
return completed.Load() == int64(taskCnt)
|
||||
}, 10*time.Second, time.Second)
|
||||
}
|
||||
|
|
|
@ -369,7 +369,7 @@ func (node *QueryNode) Init() error {
|
|||
node.subscribingChannels = typeutil.NewConcurrentSet[string]()
|
||||
node.unsubscribingChannels = typeutil.NewConcurrentSet[string]()
|
||||
node.manager = segments.NewManager()
|
||||
node.loader = segments.NewLoader(node.manager, node.chunkManager)
|
||||
node.loader = segments.NewLoader(node.ctx, node.manager, node.chunkManager)
|
||||
node.manager.SetLoader(node.loader)
|
||||
node.dispClient = msgdispatcher.NewClient(node.factory, typeutil.QueryNodeRole, node.GetNodeID())
|
||||
// init pipeline manager
|
||||
|
|
|
@ -236,6 +236,7 @@ func (suite *QueryNodeSuite) TestStop() {
|
|||
Level: datapb.SegmentLevel_Legacy,
|
||||
InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", 1),
|
||||
},
|
||||
nil,
|
||||
)
|
||||
suite.NoError(err)
|
||||
suite.node.manager.Segment.Put(context.Background(), segments.SegmentTypeSealed, segment)
|
||||
|
|
Loading…
Reference in New Issue