mirror of https://github.com/milvus-io/milvus.git
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
parent
36d3fd41e1
commit
7ce876a072
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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())
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
|
Loading…
Reference in New Issue