Skip to content

Commit

Permalink
fix: Sync dropped segment for dropped partition (milvus-io#33331)
Browse files Browse the repository at this point in the history
See also: milvus-io#33330

---------

Signed-off-by: yangxuan <[email protected]>
  • Loading branch information
XuanYang-cn authored Jun 6, 2024
1 parent fd191dd commit 68c9e7d
Show file tree
Hide file tree
Showing 13 changed files with 177 additions and 5 deletions.
4 changes: 4 additions & 0 deletions internal/datanode/flow_graph_write_node.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,10 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
wNode.wbManager.DropChannel(wNode.channelName)
}

if len(fgMsg.dropPartitions) > 0 {
wNode.wbManager.DropPartitions(wNode.channelName, fgMsg.dropPartitions)
}

// send delete msg to DeleteNode
return []Msg{&res}
}
Expand Down
7 changes: 7 additions & 0 deletions internal/datanode/metacache/actions.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,13 @@ func WithPartitionID(partitionID int64) SegmentFilter {
})
}

func WithPartitionIDs(partitionIDs []int64) SegmentFilter {
return SegmentFilterFunc(func(info *SegmentInfo) bool {
idSet := typeutil.NewSet(partitionIDs...)
return idSet.Contain(info.partitionID)
})
}

func WithStartPosNotRecorded() SegmentFilter {
return SegmentFilterFunc(func(info *SegmentInfo) bool {
return !info.startPosRecorded
Expand Down
14 changes: 9 additions & 5 deletions internal/datanode/syncmgr/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,10 @@ func (t *SyncTask) Run() (err error) {
var has bool
t.segment, has = t.metacache.GetSegmentByID(t.segmentID)
if !has {
if t.isDrop {
log.Info("segment dropped, discard sync task")
return nil
}
log.Warn("failed to sync data, segment not found in metacache")
err := merr.WrapErrSegmentNotFound(t.segmentID)
return err
Expand Down Expand Up @@ -198,15 +202,15 @@ func (t *SyncTask) Run() (err error) {
}

actions := []metacache.SegmentAction{metacache.FinishSyncing(t.batchSize)}
switch {
case t.isDrop:
actions = append(actions, metacache.UpdateState(commonpb.SegmentState_Dropped))
case t.isFlush:
if t.isFlush {
actions = append(actions, metacache.UpdateState(commonpb.SegmentState_Flushed))
}

t.metacache.UpdateSegments(metacache.MergeSegmentAction(actions...), metacache.WithSegmentIDs(t.segment.SegmentID()))

if t.isDrop {
t.metacache.RemoveSegments(metacache.WithSegmentIDs(t.segment.SegmentID()))
}

log.Info("task done", zap.Float64("flushedSize", totalSize))

if !t.isFlush {
Expand Down
1 change: 1 addition & 0 deletions internal/datanode/syncmgr/task_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,6 +244,7 @@ func (s *SyncTaskSuite) TestRunNormal() {
})

s.Run("with_delta_data", func() {
s.metacache.EXPECT().RemoveSegments(mock.Anything, mock.Anything).Return(nil).Once()
task := s.getSuiteSyncTask()
task.WithTimeRange(50, 100)
task.WithMetaWriter(BrokerMetaWriter(s.broker, 1))
Expand Down
14 changes: 14 additions & 0 deletions internal/datanode/writebuffer/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ type BufferManager interface {
RemoveChannel(channel string)
// DropChannel remove write buffer and perform drop.
DropChannel(channel string)
DropPartitions(channel string, partitionIDs []int64)
// BufferData put data into channel write buffer.
BufferData(channel string, insertMsgs []*msgstream.InsertMsg, deleteMsgs []*msgstream.DeleteMsg, startPos, endPos *msgpb.MsgPosition) error
// GetCheckpoint returns checkpoint for provided channel.
Expand Down Expand Up @@ -259,3 +260,16 @@ func (m *bufferManager) DropChannel(channel string) {

buf.Close(true)
}

func (m *bufferManager) DropPartitions(channel string, partitionIDs []int64) {
m.mut.RLock()
buf, ok := m.buffers[channel]
m.mut.RUnlock()

if !ok {
log.Warn("failed to drop partition, channel not maintained in manager", zap.String("channel", channel), zap.Int64s("partitionIDs", partitionIDs))
return
}

buf.DropPartitions(partitionIDs)
}
21 changes: 21 additions & 0 deletions internal/datanode/writebuffer/manager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,27 @@ func (s *ManagerSuite) TestRemoveChannel() {
})
}

func (s *ManagerSuite) TestDropPartitions() {
manager := s.manager

s.Run("drop_not_exist", func() {
s.NotPanics(func() {
manager.DropPartitions("not_exist_channel", nil)
})
})

s.Run("drop_partitions", func() {
wb := NewMockWriteBuffer(s.T())
wb.EXPECT().DropPartitions(mock.Anything).Return()

manager.mut.Lock()
manager.buffers[s.channelName] = wb
manager.mut.Unlock()

manager.DropPartitions(s.channelName, []int64{1})
})
}

func (s *ManagerSuite) TestMemoryCheck() {
manager := s.manager
param := paramtable.Get()
Expand Down
34 changes: 34 additions & 0 deletions internal/datanode/writebuffer/mock_mananger.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

33 changes: 33 additions & 0 deletions internal/datanode/writebuffer/mock_write_buffer.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions internal/datanode/writebuffer/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ func defaultWBOption(metacache metacache.MetaCache) *writeBufferOption {
GetSyncStaleBufferPolicy(paramtable.Get().DataNodeCfg.SyncPeriod.GetAsDuration(time.Second)),
GetCompactedSegmentsPolicy(metacache),
GetSealedSegmentsPolicy(metacache),
GetDroppedSegmentPolicy(metacache),
},
}
}
Expand Down
8 changes: 8 additions & 0 deletions internal/datanode/writebuffer/sync_policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,14 @@ func wrapSelectSegmentFuncPolicy(fn SelectSegmentFunc, reason string) SelectSegm
}
}

func GetDroppedSegmentPolicy(meta metacache.MetaCache) SyncPolicy {
return wrapSelectSegmentFuncPolicy(
func(buffers []*segmentBuffer, _ typeutil.Timestamp) []int64 {
ids := meta.GetSegmentIDsBy(metacache.WithSegmentState(commonpb.SegmentState_Dropped))
return ids
}, "segment dropped")
}

func GetFullBufferPolicy() SyncPolicy {
return wrapSelectSegmentFuncPolicy(
func(buffers []*segmentBuffer, _ typeutil.Timestamp) []int64 {
Expand Down
9 changes: 9 additions & 0 deletions internal/datanode/writebuffer/sync_policy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,15 @@ func (s *SyncPolicySuite) TestSyncStalePolicy() {
s.Equal(0, len(ids), "")
}

func (s *SyncPolicySuite) TestSyncDroppedPolicy() {
metacache := metacache.NewMockMetaCache(s.T())
policy := GetDroppedSegmentPolicy(metacache)
ids := []int64{1, 2, 3}
metacache.EXPECT().GetSegmentIDsBy(mock.Anything).Return(ids)
result := policy.SelectSegments([]*segmentBuffer{}, tsoutil.ComposeTSByTime(time.Now(), 0))
s.ElementsMatch(ids, result)
}

func (s *SyncPolicySuite) TestSealedSegmentsPolicy() {
metacache := metacache.NewMockMetaCache(s.T())
policy := GetSealedSegmentsPolicy(metacache)
Expand Down
21 changes: 21 additions & 0 deletions internal/datanode/writebuffer/write_buffer.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@ type WriteBuffer interface {
GetFlushTimestamp() uint64
// SealSegments is the method to perform `Sync` operation with provided options.
SealSegments(ctx context.Context, segmentIDs []int64) error
// DropPartitions mark segments as Dropped of the partition
DropPartitions(partitionIDs []int64)
// GetCheckpoint returns current channel checkpoint.
// If there are any non-empty segment buffer, returns the earliest buffer start position.
// Otherwise, returns latest buffered checkpoint.
Expand Down Expand Up @@ -222,6 +224,13 @@ func (wb *writeBufferBase) SealSegments(ctx context.Context, segmentIDs []int64)
return wb.sealSegments(ctx, segmentIDs)
}

func (wb *writeBufferBase) DropPartitions(partitionIDs []int64) {
wb.mut.RLock()
defer wb.mut.RUnlock()

wb.dropPartitions(partitionIDs)
}

func (wb *writeBufferBase) SetFlushTimestamp(flushTs uint64) {
wb.flushTimestamp.Store(flushTs)
}
Expand Down Expand Up @@ -328,6 +337,14 @@ func (wb *writeBufferBase) sealSegments(_ context.Context, segmentIDs []int64) e
return nil
}

func (wb *writeBufferBase) dropPartitions(partitionIDs []int64) {
// mark segment dropped if partition was dropped
segIDs := wb.metaCache.GetSegmentIDsBy(metacache.WithPartitionIDs(partitionIDs))
wb.metaCache.UpdateSegments(metacache.UpdateState(commonpb.SegmentState_Dropped),
metacache.WithSegmentIDs(segIDs...),
)
}

func (wb *writeBufferBase) syncSegments(ctx context.Context, segmentIDs []int64) []*conc.Future[struct{}] {
log := log.Ctx(ctx)
result := make([]*conc.Future[struct{}], 0, len(segmentIDs))
Expand Down Expand Up @@ -590,6 +607,10 @@ func (wb *writeBufferBase) getSyncTask(ctx context.Context, segmentID int64) (sy
pack.WithFlush()
}

if segmentInfo.State() == commonpb.SegmentState_Dropped {
pack.WithDrop()
}

metrics.DataNodeFlowGraphBufferDataSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), fmt.Sprint(wb.collectionID)).Sub(totalMemSize)

return wb.serializer.EncodeBuffer(ctx, pack)
Expand Down
15 changes: 15 additions & 0 deletions internal/datanode/writebuffer/write_buffer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -365,6 +365,21 @@ func (s *WriteBufferSuite) TestEvictBuffer() {
})
}

func (s *WriteBufferSuite) TestDropPartitions() {
wb, err := newWriteBufferBase(s.channelName, s.metacache, s.storageCache, s.syncMgr, &writeBufferOption{
pkStatsFactory: func(vchannel *datapb.SegmentInfo) *metacache.BloomFilterSet {
return metacache.NewBloomFilterSet()
},
})
s.Require().NoError(err)

segIDs := []int64{1, 2, 3}
s.metacache.EXPECT().GetSegmentIDsBy(mock.Anything).Return(segIDs).Once()
s.metacache.EXPECT().UpdateSegments(mock.AnythingOfType("metacache.SegmentAction"), metacache.WithSegmentIDs(segIDs...)).Return().Once()

wb.dropPartitions([]int64{100, 101})
}

func TestWriteBufferBase(t *testing.T) {
suite.Run(t, new(WriteBufferSuite))
}

0 comments on commit 68c9e7d

Please sign in to comment.