fix: growing segment from old arch is not flushed after upgrading (#42164)

issue: #42162

- enhance: add read ahead buffer size issue #42129
- fix: rocksmq consumer's close operation may get stucked
- fix: growing segment from old arch is not flushed after upgrading

---------

Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
Zhen Ye 2025-05-29 23:00:28 +08:00 committed by GitHub
parent eabb62e3ab
commit b94cee2413
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 2665 additions and 2472 deletions

View File

@ -1225,6 +1225,11 @@ streaming:
walWriteAheadBuffer:
capacity: 64m # The capacity of write ahead buffer of each wal, 64M by default
keepalive: 30s # The keepalive duration for entries in write ahead buffer of each wal, 30s by default
walReadAheadBuffer:
# The buffer length (pending message count) of read ahead buffer of each wal scanner can be used, 128 by default.
# Higher one will increase the throughput of wal message handling, but introduce higher memory utilization.
# Use the underlying wal default value if 0 is given.
length: 128
logging:
# The threshold of slow log, 1s by default.
# If the wal implementation is woodpecker, the minimum threshold is 3s

View File

@ -1030,6 +1030,17 @@ func (s *Server) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChan
resp.Status = merr.Status(merr.WrapErrChannelNotAvailable(req.GetVchannel(), "start position is nil"))
return resp, nil
}
segmentsNotCreatedByStreaming := make([]*datapb.SegmentNotCreatedByStreaming, 0)
for _, segmentID := range channelInfo.GetUnflushedSegmentIds() {
segment := s.meta.GetSegment(ctx, segmentID)
if segment != nil && !segment.IsCreatedByStreaming {
segmentsNotCreatedByStreaming = append(segmentsNotCreatedByStreaming, &datapb.SegmentNotCreatedByStreaming{
CollectionId: segment.CollectionID,
PartitionId: segment.PartitionID,
SegmentId: segmentID,
})
}
}
log.Info("datacoord get channel recovery info",
zap.String("channel", channelInfo.GetChannelName()),
@ -1038,10 +1049,12 @@ func (s *Server) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChan
zap.Int("# of dropped segments", len(channelInfo.GetDroppedSegmentIds())),
zap.Int("# of indexed segments", len(channelInfo.GetIndexedSegmentIds())),
zap.Int("# of l0 segments", len(channelInfo.GetLevelZeroSegmentIds())),
zap.Int("# of segments not created by streaming", len(segmentsNotCreatedByStreaming)),
)
resp.Info = channelInfo
resp.Schema = collection.Schema
resp.SegmentsNotCreatedByStreaming = segmentsNotCreatedByStreaming
return resp, nil
}

View File

@ -1557,6 +1557,16 @@ func TestGetChannelRecoveryInfo(t *testing.T) {
handler := NewNMockHandler(t)
handler.EXPECT().GetDataVChanPositions(mock.Anything, mock.Anything).Return(channelInfo)
s.handler = handler
s.meta = &meta{
segments: NewSegmentsInfo(),
}
s.meta.segments.segments[1] = NewSegmentInfo(&datapb.SegmentInfo{
ID: 1,
CollectionID: 0,
PartitionID: 0,
State: commonpb.SegmentState_Growing,
IsCreatedByStreaming: false,
})
assert.NoError(t, err)
resp, err = s.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{

View File

@ -199,35 +199,28 @@ func (impl *flusherComponents) recover(ctx context.Context, recoverInfos map[str
// buildDataSyncServiceWithRetry builds the data sync service with retry.
func (impl *flusherComponents) buildDataSyncServiceWithRetry(ctx context.Context, recoverInfo *datapb.GetChannelRecoveryInfoResponse) (*dataSyncServiceWrapper, error) {
// Flush all the growing segment that is not created by streaming.
segmentIDs := make([]int64, 0, len(recoverInfo.GetInfo().UnflushedSegments))
for _, segment := range recoverInfo.GetInfo().UnflushedSegments {
if segment.IsCreatedByStreaming {
continue
}
for _, segment := range recoverInfo.SegmentsNotCreatedByStreaming {
logger := impl.logger.With(
zap.Int64("collectionID", segment.CollectionId),
zap.String("vchannel", recoverInfo.GetInfo().GetChannelName()),
zap.Int64("partitionID", segment.PartitionId),
zap.Int64("segmentID", segment.SegmentId),
)
msg := message.NewFlushMessageBuilderV2().
WithVChannel(recoverInfo.GetInfo().GetChannelName()).
WithHeader(&message.FlushMessageHeader{
CollectionId: recoverInfo.GetInfo().GetCollectionID(),
PartitionId: segment.PartitionID,
SegmentId: segment.ID,
CollectionId: segment.CollectionId,
PartitionId: segment.PartitionId,
SegmentId: segment.SegmentId,
}).
WithBody(&message.FlushMessageBody{}).MustBuildMutable()
if err := retry.Do(ctx, func() error {
appendResult, err := impl.wal.Append(ctx, msg)
if err != nil {
impl.logger.Warn(
"fail to append flush message for segments that not created by streaming service into wal",
zap.String("vchannel", recoverInfo.GetInfo().GetChannelName()),
zap.Error(err))
logger.Warn("fail to append flush message for segments that not created by streaming service into wal", zap.Error(err))
return err
}
impl.logger.Info(
"append flush message for segments that not created by streaming service into wal",
zap.String("vchannel", recoverInfo.GetInfo().GetChannelName()),
zap.Int64s("segmentIDs", segmentIDs),
zap.Stringer("msgID", appendResult.MessageID),
zap.Uint64("timeTick", appendResult.TimeTick),
)
impl.logger.Info("append flush message for segments that not created by streaming service into wal", zap.Stringer("msgID", appendResult.MessageID), zap.Uint64("timeTick", appendResult.TimeTick))
return nil
}, retry.AttemptAlways()); err != nil {
return nil, err

View File

@ -14,6 +14,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/options"
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
@ -178,9 +179,14 @@ func (s *catchupScanner) createReaderWithBackoff(ctx context.Context, deliverPol
})
backoffTimer.EnableBackoff()
for {
bufSize := paramtable.Get().StreamingCfg.WALReadAheadBufferLength.GetAsInt()
if bufSize < 0 {
bufSize = 0
}
innerScanner, err := s.innerWAL.Read(ctx, walimpls.ReadOption{
Name: s.scannerName,
DeliverPolicy: deliverPolicy,
Name: s.scannerName,
DeliverPolicy: deliverPolicy,
ReadAheadBufferSize: bufSize,
})
if err == nil {
return innerScanner, nil

View File

@ -127,8 +127,8 @@ func (c *client) Subscribe(options ConsumerOptions) (Consumer, error) {
func (c *client) consume(consumer *consumer) {
defer func() {
close(consumer.stopCh)
c.wg.Done()
consumer.wg.Done()
}()
if err := c.blockUntilInitDone(consumer); err != nil {
@ -161,6 +161,9 @@ func (c *client) consume(consumer *consumer) {
}
select {
case <-consumer.ctx.Done():
log.Info("Consumer is closed, consumer goroutine exit")
return
case <-c.closeCh:
log.Info("Client is closed, consumer goroutine exit")
return

View File

@ -12,6 +12,7 @@
package client
import (
"context"
"sync"
"go.uber.org/zap"
@ -28,7 +29,10 @@ type consumer struct {
startOnce sync.Once
stopCh chan struct{}
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
msgMutex chan struct{}
initCh chan struct{}
messageCh chan common.Message
@ -54,12 +58,15 @@ func newConsumer(c *client, options ConsumerOptions) (*consumer, error) {
// only used for
initCh := make(chan struct{}, 1)
initCh <- struct{}{}
ctx, cancel := context.WithCancel(context.Background())
return &consumer{
topic: options.Topic,
client: c,
consumerName: options.SubscriptionName,
options: options,
stopCh: make(chan struct{}),
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
msgMutex: make(chan struct{}, 1),
initCh: initCh,
messageCh: messageCh,
@ -85,11 +92,15 @@ func getExistedConsumer(c *client, options ConsumerOptions, msgMutex chan struct
messageCh = make(chan common.Message, 1)
}
ctx, cancel := context.WithCancel(context.Background())
return &consumer{
topic: options.Topic,
client: c,
consumerName: options.SubscriptionName,
options: options,
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
msgMutex: msgMutex,
messageCh: messageCh,
}, nil
@ -114,6 +125,7 @@ func (c *consumer) MsgMutex() chan struct{} {
func (c *consumer) Chan() <-chan common.Message {
c.startOnce.Do(func() {
c.client.wg.Add(1)
c.wg.Add(1)
go c.client.consume(c)
})
return c.messageCh
@ -141,7 +153,8 @@ func (c *consumer) Close() {
close(c.msgMutex)
return
}
<-c.stopCh
c.cancel()
c.wg.Wait()
}
func (c *consumer) GetLatestMsgID() (int64, error) {

View File

@ -278,9 +278,9 @@ message VchannelInfo {
int64 collectionID = 1;
string channelName = 2;
msg.MsgPosition seek_position = 3;
repeated SegmentInfo unflushedSegments = 4; // deprecated, keep it for compatibility
repeated SegmentInfo flushedSegments = 5; // deprecated, keep it for compatibility
repeated SegmentInfo dropped_segments = 6; // deprecated, keep it for compatibility
repeated SegmentInfo unflushedSegments = 4 [deprecated = true]; // deprecated, keep it for compatibility
repeated SegmentInfo flushedSegments = 5 [deprecated = true]; // deprecated, keep it for compatibility
repeated SegmentInfo dropped_segments = 6 [deprecated = true]; // deprecated, keep it for compatibility
repeated int64 unflushedSegmentIds = 7;
repeated int64 flushedSegmentIds = 8;
repeated int64 dropped_segmentIds = 9;
@ -517,6 +517,13 @@ message GetChannelRecoveryInfoResponse {
common.Status status = 1;
VchannelInfo info = 2;
schema.CollectionSchema schema = 3;
repeated SegmentNotCreatedByStreaming segments_not_created_by_streaming = 4; // Should be flushed by streaming service when upgrading.
}
message SegmentNotCreatedByStreaming {
int64 collection_id = 1;
int64 partition_id = 2;
int64 segment_id = 3;
}
message GetSegmentsByStatesRequest {

File diff suppressed because it is too large Load Diff

View File

@ -186,6 +186,7 @@ func (f *testOneWALImplsFramework) testReadAndWrite(ctx context.Context, w WALIm
}()
wg.Wait()
f.testReadWithFastClose(ctx, w)
f.assertSortedMessageList(read1)
f.assertSortedMessageList(read2)
@ -309,6 +310,25 @@ func (f *testOneWALImplsFramework) testAppend(ctx context.Context, w WALImpls) (
return ids, nil
}
func (f *testOneWALImplsFramework) testReadWithFastClose(ctx context.Context, w WALImpls) {
wg := sync.WaitGroup{}
wg.Add(10)
for i := 0; i < 10; i++ {
name := fmt.Sprintf("scanner-fast-close-%d", i)
go func() {
defer wg.Done()
s, err := w.Read(ctx, ReadOption{
Name: name,
DeliverPolicy: options.DeliverPolicyAll(),
ReadAheadBufferSize: 128,
})
assert.NoError(f.t, err)
s.Close()
}()
}
wg.Wait()
}
func (f *testOneWALImplsFramework) testRead(ctx context.Context, w ROWALImpls, name string) ([]message.ImmutableMessage, error) {
s, err := w.Read(ctx, ReadOption{
Name: name,

View File

@ -5570,6 +5570,9 @@ type streamingConfig struct {
WALWriteAheadBufferCapacity ParamItem `refreshable:"true"`
WALWriteAheadBufferKeepalive ParamItem `refreshable:"true"`
// read ahead buffer size
WALReadAheadBufferLength ParamItem `refreshable:"true"`
// logging
LoggingAppendSlowThreshold ParamItem `refreshable:"true"`
// memory usage control
@ -5711,6 +5714,17 @@ it also determine the depth of depth first search method that is used to find th
}
p.WALWriteAheadBufferKeepalive.Init(base.mgr)
p.WALReadAheadBufferLength = ParamItem{
Key: "streaming.walReadAheadBuffer.length",
Version: "2.6.0",
Doc: `The buffer length (pending message count) of read ahead buffer of each wal scanner can be used, 128 by default.
Higher one will increase the throughput of wal message handling, but introduce higher memory utilization.
Use the underlying wal default value if 0 is given.`,
DefaultValue: "128",
Export: true,
}
p.WALReadAheadBufferLength.Init(base.mgr)
p.LoggingAppendSlowThreshold = ParamItem{
Key: "streaming.logging.appendSlowThreshold",
Version: "2.6.0",

View File

@ -632,6 +632,7 @@ func TestComponentParam(t *testing.T) {
assert.Equal(t, 10*time.Second, params.StreamingCfg.TxnDefaultKeepaliveTimeout.GetAsDurationByParse())
assert.Equal(t, 30*time.Second, params.StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse())
assert.Equal(t, int64(64*1024*1024), params.StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize())
assert.Equal(t, 128, params.StreamingCfg.WALReadAheadBufferLength.GetAsInt())
assert.Equal(t, 1*time.Second, params.StreamingCfg.LoggingAppendSlowThreshold.GetAsDurationByParse())
assert.Equal(t, 3*time.Second, params.StreamingCfg.WALRecoveryGracefulCloseTimeout.GetAsDurationByParse())
assert.Equal(t, 100, params.StreamingCfg.WALRecoveryMaxDirtyMessage.GetAsInt())