Skip to content

Commit

Permalink
fix: Block warmup submit if pool full in sync mode (milvus-io#38690)
Browse files Browse the repository at this point in the history
milvus-io#38692

---------

Signed-off-by: sunby <[email protected]>
  • Loading branch information
sunby authored Jan 2, 2025
1 parent d3a5282 commit aa0a87e
Show file tree
Hide file tree
Showing 12 changed files with 106 additions and 10 deletions.
1 change: 1 addition & 0 deletions internal/core/src/common/ChunkTarget.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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_);
Expand Down
1 change: 1 addition & 0 deletions internal/querynodev2/delegator/delegator_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
1 change: 1 addition & 0 deletions internal/querynodev2/segments/manager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion internal/querynodev2/segments/pool.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 2 additions & 0 deletions internal/querynodev2/segments/retrieve_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down Expand Up @@ -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)

Expand Down
2 changes: 2 additions & 0 deletions internal/querynodev2/segments/search_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down Expand Up @@ -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)

Expand Down
67 changes: 62 additions & 5 deletions internal/querynodev2/segments/segment.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"context"
"fmt"
"strings"
"sync"
"time"
"unsafe"

Expand Down Expand Up @@ -279,13 +280,15 @@ type LocalSegment struct {
lastDeltaTimestamp *atomic.Uint64
fields *typeutil.ConcurrentMap[int64, *FieldInfo]
fieldIndexes *typeutil.ConcurrentMap[int64, *IndexedFieldInfo]
warmupDispatcher *AsyncWarmupDispatcher
}

func NewSegment(ctx context.Context,
collection *Collection,
segmentType SegmentType,
version int64,
loadInfo *querypb.SegmentLoadInfo,
warmupDispatcher *AsyncWarmupDispatcher,
) (Segment, error) {
log := log.Ctx(ctx)
/*
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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.
Expand All @@ -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
}
Expand Down Expand 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()
}
}
}
7 changes: 7 additions & 0 deletions internal/querynodev2/segments/segment_loader.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@ type resourceEstimateFactor struct {
}

func NewLoader(
ctx context.Context,
manager *Manager,
cm storage.ChunkManager,
) *segmentLoader {
Expand All @@ -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
Expand Down Expand Up @@ -212,6 +216,8 @@ type segmentLoader struct {
loadingSegments *typeutil.ConcurrentMap[int64, *loadResult]
committedResource LoadResource
committedResourceNotifier *syncutil.VersionedNotifier

warmupDispatcher *AsyncWarmupDispatcher
}

var _ Loader = (*segmentLoader)(nil)
Expand Down Expand Up @@ -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",
Expand Down
6 changes: 3 additions & 3 deletions internal/querynodev2/segments/segment_loader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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")
Expand Down Expand Up @@ -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
Expand Down
24 changes: 24 additions & 0 deletions internal/querynodev2/segments/segment_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -91,6 +94,7 @@ func (suite *SegmentSuite) SetupTest() {
},
},
},
nil,
)
suite.Require().NoError(err)

Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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)
}
2 changes: 1 addition & 1 deletion internal/querynodev2/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions internal/querynodev2/server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down

0 comments on commit aa0a87e

Please sign in to comment.