mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
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:
parent
eabb62e3ab
commit
b94cee2413
@ -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
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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{
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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
@ -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,
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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())
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user