fix: Decoupling importing segment from flush process (#30402)

This pr decoups importing segment from flush process by:
1. Exclude the importing segment from the flush policy, this approch
avoids notifying the datanode to flush the importing segment, which may
not exist.
2. When RootCoord call Flush, DataCoord directly set the importing
segment state to `Flushed`.

issue: https://github.com/milvus-io/milvus/issues/30359

---------

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
pull/30486/head
yihao.dai 2024-02-03 13:01:12 +08:00 committed by GitHub
parent 36d3fd41e1
commit 7ce876a072
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 175 additions and 36 deletions

View File

@ -183,7 +183,13 @@ type flushPolicy func(segment *SegmentInfo, t Timestamp) bool
const flushInterval = 2 * time.Second
func flushPolicyL1(segment *SegmentInfo, t Timestamp) bool {
return segment.GetState() == commonpb.SegmentState_Sealed && segment.Level != datapb.SegmentLevel_L0 &&
return segment.GetState() == commonpb.SegmentState_Sealed &&
segment.Level != datapb.SegmentLevel_L0 &&
time.Since(segment.lastFlushTime) >= flushInterval &&
(segment.GetLastExpireTime() <= t && segment.currRows != 0 || (segment.IsImporting))
segment.GetLastExpireTime() <= t &&
segment.currRows != 0 &&
// Decoupling the importing segment from the flush process,
// This check avoids notifying the datanode to flush the
// importing segment which may not exist.
!segment.GetIsImporting()
}

View File

@ -23,6 +23,7 @@ import (
"time"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.opentelemetry.io/otel"
"go.uber.org/zap"
@ -75,9 +76,11 @@ type Manager interface {
allocSegmentForImport(ctx context.Context, collectionID, partitionID UniqueID, channelName string, requestRows int64, taskID int64) (*Allocation, error)
// DropSegment drops the segment from manager.
DropSegment(ctx context.Context, segmentID UniqueID)
// FlushImportSegments set importing segment state to Flushed.
FlushImportSegments(ctx context.Context, collectionID UniqueID, segmentIDs []UniqueID) error
// SealAllSegments seals all segments of collection with collectionID and return sealed segments.
// If segIDs is not empty, also seals segments in segIDs.
SealAllSegments(ctx context.Context, collectionID UniqueID, segIDs []UniqueID, isImporting bool) ([]UniqueID, error)
SealAllSegments(ctx context.Context, collectionID UniqueID, segIDs []UniqueID) ([]UniqueID, error)
// GetFlushableSegments returns flushable segment ids
GetFlushableSegments(ctx context.Context, channel string, ts Timestamp) ([]UniqueID, error)
// ExpireAllocations notifies segment status to expire old allocations
@ -456,10 +459,42 @@ func (s *SegmentManager) DropSegment(ctx context.Context, segmentID UniqueID) {
}
}
// FlushImportSegments set importing segment state to Flushed.
func (s *SegmentManager) FlushImportSegments(ctx context.Context, collectionID UniqueID, segmentIDs []UniqueID) error {
_, sp := otel.Tracer(typeutil.DataCoordRole).Start(ctx, "Flush-Import-Segments")
defer sp.End()
s.mu.Lock()
defer s.mu.Unlock()
candidates := lo.Filter(segmentIDs, func(segmentID UniqueID, _ int) bool {
info := s.meta.GetHealthySegment(segmentID)
if info == nil {
log.Warn("failed to get seg info from meta", zap.Int64("segmentID", segmentID))
return false
}
if info.CollectionID != collectionID {
return false
}
return info.State == commonpb.SegmentState_Importing
})
// We set the importing segment state directly to 'Flushed' rather than
// 'Sealed' because all data has been imported, and there is no data
// in the datanode flowgraph that needs to be synced.
for _, id := range candidates {
if err := s.meta.SetState(id, commonpb.SegmentState_Flushed); err != nil {
return err
}
}
return nil
}
// SealAllSegments seals all segments of collection with collectionID and return sealed segments
func (s *SegmentManager) SealAllSegments(ctx context.Context, collectionID UniqueID, segIDs []UniqueID, isImport bool) ([]UniqueID, error) {
func (s *SegmentManager) SealAllSegments(ctx context.Context, collectionID UniqueID, segIDs []UniqueID) ([]UniqueID, error) {
_, sp := otel.Tracer(typeutil.DataCoordRole).Start(ctx, "Seal-Segments")
defer sp.End()
s.mu.Lock()
defer s.mu.Unlock()
var ret []UniqueID
@ -481,8 +516,8 @@ func (s *SegmentManager) SealAllSegments(ctx context.Context, collectionID Uniqu
ret = append(ret, id)
continue
}
// segment can be sealed only if it is growing or if it's importing
if (!isImport && info.State != commonpb.SegmentState_Growing) || (isImport && info.State != commonpb.SegmentState_Importing) {
// segment can be sealed only if it is growing.
if info.State != commonpb.SegmentState_Growing {
continue
}
if err := s.meta.SetState(id, commonpb.SegmentState_Sealed); err != nil {

View File

@ -344,7 +344,7 @@ func TestSaveSegmentsToMeta(t *testing.T) {
allocations, err := segmentManager.AllocSegment(context.Background(), collID, 0, "c1", 1000)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
_, err = segmentManager.SealAllSegments(context.Background(), collID, nil, false)
_, err = segmentManager.SealAllSegments(context.Background(), collID, nil)
assert.NoError(t, err)
segment := meta.GetHealthySegment(allocations[0].SegmentID)
assert.NotNil(t, segment)
@ -366,7 +366,7 @@ func TestSaveSegmentsToMetaWithSpecificSegments(t *testing.T) {
allocations, err := segmentManager.AllocSegment(context.Background(), collID, 0, "c1", 1000)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
_, err = segmentManager.SealAllSegments(context.Background(), collID, []int64{allocations[0].SegmentID}, false)
_, err = segmentManager.SealAllSegments(context.Background(), collID, []int64{allocations[0].SegmentID})
assert.NoError(t, err)
segment := meta.GetHealthySegment(allocations[0].SegmentID)
assert.NotNil(t, segment)
@ -507,7 +507,7 @@ func TestGetFlushableSegments(t *testing.T) {
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
ids, err := segmentManager.SealAllSegments(context.TODO(), collID, nil, false)
ids, err := segmentManager.SealAllSegments(context.TODO(), collID, nil)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(ids))
assert.EqualValues(t, allocations[0].SegmentID, ids[0])
@ -889,3 +889,41 @@ func TestSegmentManager_DropSegmentsOfChannel(t *testing.T) {
})
}
}
func TestSegmentManager_FlushImportSegments(t *testing.T) {
alloc := NewNMockAllocator(t)
alloc.EXPECT().allocID(mock.Anything).Return(0, nil)
alloc.EXPECT().allocTimestamp(mock.Anything).Return(1000, nil)
mm, err := newMemoryMeta()
assert.NoError(t, err)
schema := newTestSchema()
assert.NoError(t, err)
mm.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(mm, alloc)
allocation, err := segmentManager.allocSegmentForImport(context.TODO(), collID, 1, "c1", 2, 3)
assert.NoError(t, err)
segmentID := allocation.SegmentID
segment := mm.GetSegment(segmentID)
assert.Equal(t, commonpb.SegmentState_Importing, segment.GetState())
// normal
err = segmentManager.FlushImportSegments(context.TODO(), collID, []UniqueID{segmentID})
assert.NoError(t, err)
segment = mm.GetSegment(segmentID)
assert.Equal(t, commonpb.SegmentState_Flushed, segment.GetState())
// no segment
err = segmentManager.FlushImportSegments(context.TODO(), collID, []UniqueID{6})
assert.NoError(t, err)
// collection not match
mm.AddCollection(&collectionInfo{ID: 6, Schema: schema})
allocation, err = segmentManager.allocSegmentForImport(context.TODO(), 6, 1, "c1", 2, 3)
assert.NoError(t, err)
err = segmentManager.FlushImportSegments(context.TODO(), collID, []UniqueID{allocation.SegmentID})
assert.NoError(t, err)
segment = mm.GetSegment(allocation.SegmentID)
assert.Equal(t, commonpb.SegmentState_Importing, segment.GetState())
}

View File

@ -1008,8 +1008,13 @@ func (s *spySegmentManager) allocSegmentForImport(ctx context.Context, collectio
func (s *spySegmentManager) DropSegment(ctx context.Context, segmentID UniqueID) {
}
// FlushImportSegments set importing segment state to Flushed.
func (s *spySegmentManager) FlushImportSegments(ctx context.Context, collectionID UniqueID, segmentIDs []UniqueID) error {
panic("not implemented")
}
// SealAllSegments seals all segments of collection with collectionID and return sealed segments
func (s *spySegmentManager) SealAllSegments(ctx context.Context, collectionID UniqueID, segIDs []UniqueID, isImport bool) ([]UniqueID, error) {
func (s *spySegmentManager) SealAllSegments(ctx context.Context, collectionID UniqueID, segIDs []UniqueID) ([]UniqueID, error) {
panic("not implemented") // TODO: Implement
}

View File

@ -63,6 +63,27 @@ func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetSt
}, nil
}
func (s *Server) flushForImport(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error) {
err := s.segmentManager.FlushImportSegments(ctx, req.GetCollectionID(), req.GetSegmentIDs())
if err != nil {
return &datapb.FlushResponse{
Status: merr.Status(err),
}, nil
}
// To expedite the process of index building.
for _, segmentID := range req.GetSegmentIDs() {
select {
case s.buildIndexCh <- segmentID:
default:
}
}
log.Info("flush for import done", zap.Int64("collectionID", req.GetCollectionID()),
zap.Int64s("segmentIDs", req.GetSegmentIDs()))
return &datapb.FlushResponse{
Status: merr.Success(),
}, nil
}
// Flush notify segment to flush
// this api only guarantees all the segments requested is sealed
// these segments will be flushed only after the Flush policy is fulfilled
@ -81,6 +102,10 @@ func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.F
}, nil
}
if req.GetIsImport() {
return s.flushForImport(ctx, req)
}
// generate a timestamp timeOfSeal, all data before timeOfSeal is guaranteed to be sealed or flushed
ts, err := s.allocator.allocTimestamp(ctx)
if err != nil {
@ -91,7 +116,7 @@ func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.F
}
timeOfSeal, _ := tsoutil.ParseTS(ts)
sealedSegmentIDs, err := s.segmentManager.SealAllSegments(ctx, req.GetCollectionID(), req.GetSegmentIDs(), req.GetIsImport())
sealedSegmentIDs, err := s.segmentManager.SealAllSegments(ctx, req.GetCollectionID(), req.GetSegmentIDs())
if err != nil {
return &datapb.FlushResponse{
Status: merr.Status(errors.Wrapf(err, "failed to flush collection %d",

View File

@ -5,6 +5,7 @@ import (
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
@ -383,6 +384,36 @@ func (s *ServerSuite) TestSaveBinlogPath_NormalCase() {
s.EqualValues(segment.NumOfRows, 10)
}
func (s *ServerSuite) TestFlushForImport() {
schema := newTestSchema()
s.testServer.meta.AddCollection(&collectionInfo{ID: 0, Schema: schema, Partitions: []int64{}})
// normal
allocation, err := s.testServer.segmentManager.allocSegmentForImport(
context.TODO(), 0, 1, "ch-1", 1, 1)
s.NoError(err)
segmentID := allocation.SegmentID
req := &datapb.FlushRequest{
CollectionID: 0,
SegmentIDs: []UniqueID{segmentID},
}
resp, err := s.testServer.flushForImport(context.TODO(), req)
s.NoError(err)
s.EqualValues(int32(0), resp.GetStatus().GetCode())
// failed
allocation, err = s.testServer.segmentManager.allocSegmentForImport(
context.TODO(), 0, 1, "ch-1", 1, 1)
s.NoError(err)
catalog := mocks.NewDataCoordCatalog(s.T())
catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything).Return(errors.New("mock err"))
s.testServer.meta.catalog = catalog
req.SegmentIDs = []UniqueID{allocation.SegmentID}
resp, err = s.testServer.flushForImport(context.TODO(), req)
s.NoError(err)
s.NotEqual(int32(0), resp.GetStatus().GetCode())
}
func (s *ServerSuite) TestFlush_NormalCase() {
req := &datapb.FlushRequest{
Base: &commonpb.MsgBase{
@ -437,11 +468,11 @@ func (s *ServerSuite) TestFlush_BulkLoadSegment() {
}
s.mockChMgr.EXPECT().GetNodeChannelsByCollectionID(mock.Anything).Return(map[int64][]string{
1: {"channel-1"},
}).Twice()
})
mockCluster := NewMockCluster(s.T())
mockCluster.EXPECT().FlushChannels(mock.Anything, mock.Anything, mock.Anything, mock.Anything).
Return(nil).Twice()
Return(nil)
mockCluster.EXPECT().Close().Maybe()
s.testServer.cluster = mockCluster
@ -472,18 +503,21 @@ func (s *ServerSuite) TestFlush_BulkLoadSegment() {
},
DbID: 0,
CollectionID: 0,
SegmentIDs: []int64{segID},
IsImport: true,
}
resp, err = s.testServer.Flush(context.TODO(), req)
s.NoError(err)
s.EqualValues(commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
s.EqualValues(1, len(resp.SegmentIDs))
segment := s.testServer.meta.GetSegment(segID)
s.Equal(commonpb.SegmentState_Flushed, segment.GetState())
err = s.testServer.meta.UnsetIsImporting(segID)
s.NoError(err)
ids, err = s.testServer.segmentManager.GetFlushableSegments(context.TODO(), "channel-1", expireTs)
s.NoError(err)
s.EqualValues(1, len(ids))
s.EqualValues(segID, ids[0])
s.EqualValues(0, len(ids))
}
func (s *ServerSuite) TestFlush_ClosedServer() {

View File

@ -57,7 +57,7 @@ func NewSyncTask(ctx context.Context, task *ImportTask, segmentID, partitionID i
}, func(info *datapb.SegmentInfo) *metacache.BloomFilterSet {
bfs := metacache.NewBloomFilterSet()
return bfs
}, metacache.UpdateImporting(true))
})
}
var serializer syncmgr.Serializer

View File

@ -53,12 +53,6 @@ func WithStartPosNotRecorded() SegmentFilter {
}
}
func WithImporting() SegmentFilter {
return func(info *SegmentInfo) bool {
return info.importing
}
}
func WithLevel(level datapb.SegmentLevel) SegmentFilter {
return func(info *SegmentInfo) bool {
return info.level == level
@ -115,12 +109,6 @@ func CompactTo(compactTo int64) SegmentAction {
}
}
func UpdateImporting(importing bool) SegmentAction {
return func(info *SegmentInfo) {
info.importing = importing
}
}
func StartSyncing(batchSize int64) SegmentAction {
return func(info *SegmentInfo) {
info.syncingRows += batchSize

View File

@ -41,7 +41,6 @@ type SegmentInfo struct {
syncingRows int64
bfs *BloomFilterSet
compactTo int64
importing bool
level datapb.SegmentLevel
syncingTasks int32
}
@ -107,7 +106,6 @@ func (s *SegmentInfo) Clone() *SegmentInfo {
bfs: s.bfs,
compactTo: s.compactTo,
level: s.level,
importing: s.importing,
syncingTasks: s.syncingTasks,
}
}

View File

@ -609,7 +609,7 @@ func (node *DataNode) AddImportSegment(ctx context.Context, req *datapb.AddImpor
}, func(info *datapb.SegmentInfo) *metacache.BloomFilterSet {
bfs := metacache.NewBloomFilterSet(pks...)
return bfs
}, metacache.UpdateImporting(true))
})
}
return &datapb.AddImportSegmentResponse{

View File

@ -293,10 +293,6 @@ func (wb *writeBufferBase) sealSegments(ctx context.Context, segmentIDs []int64)
wb.metaCache.UpdateSegments(metacache.UpdateState(commonpb.SegmentState_Sealed),
metacache.WithSegmentIDs(segmentIDs...),
metacache.WithSegmentState(commonpb.SegmentState_Growing))
// mark segment flushing if segment was importing
wb.metaCache.UpdateSegments(metacache.UpdateState(commonpb.SegmentState_Sealed),
metacache.WithSegmentIDs(segmentIDs...),
metacache.WithImporting())
return nil
}

View File

@ -151,6 +151,20 @@ func (s *BulkInsertSuite) TestBulkInsert() {
}
}
// flush
flushResp, err := c.Proxy.Flush(ctx, &milvuspb.FlushRequest{
DbName: dbName,
CollectionNames: []string{collectionName},
})
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.Require().Empty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
health2, err := c.DataCoord.CheckHealth(ctx, &milvuspb.CheckHealthRequest{})
s.NoError(err)
log.Info("dataCoord health", zap.Any("health2", health2))