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:
|
walWriteAheadBuffer:
|
||||||
capacity: 64m # The capacity of write ahead buffer of each wal, 64M by default
|
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
|
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:
|
logging:
|
||||||
# The threshold of slow log, 1s by default.
|
# The threshold of slow log, 1s by default.
|
||||||
# If the wal implementation is woodpecker, the minimum threshold is 3s
|
# 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"))
|
resp.Status = merr.Status(merr.WrapErrChannelNotAvailable(req.GetVchannel(), "start position is nil"))
|
||||||
return resp, 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",
|
log.Info("datacoord get channel recovery info",
|
||||||
zap.String("channel", channelInfo.GetChannelName()),
|
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 dropped segments", len(channelInfo.GetDroppedSegmentIds())),
|
||||||
zap.Int("# of indexed segments", len(channelInfo.GetIndexedSegmentIds())),
|
zap.Int("# of indexed segments", len(channelInfo.GetIndexedSegmentIds())),
|
||||||
zap.Int("# of l0 segments", len(channelInfo.GetLevelZeroSegmentIds())),
|
zap.Int("# of l0 segments", len(channelInfo.GetLevelZeroSegmentIds())),
|
||||||
|
zap.Int("# of segments not created by streaming", len(segmentsNotCreatedByStreaming)),
|
||||||
)
|
)
|
||||||
|
|
||||||
resp.Info = channelInfo
|
resp.Info = channelInfo
|
||||||
resp.Schema = collection.Schema
|
resp.Schema = collection.Schema
|
||||||
|
resp.SegmentsNotCreatedByStreaming = segmentsNotCreatedByStreaming
|
||||||
return resp, nil
|
return resp, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -1557,6 +1557,16 @@ func TestGetChannelRecoveryInfo(t *testing.T) {
|
|||||||
handler := NewNMockHandler(t)
|
handler := NewNMockHandler(t)
|
||||||
handler.EXPECT().GetDataVChanPositions(mock.Anything, mock.Anything).Return(channelInfo)
|
handler.EXPECT().GetDataVChanPositions(mock.Anything, mock.Anything).Return(channelInfo)
|
||||||
s.handler = handler
|
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)
|
assert.NoError(t, err)
|
||||||
resp, err = s.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{
|
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.
|
// buildDataSyncServiceWithRetry builds the data sync service with retry.
|
||||||
func (impl *flusherComponents) buildDataSyncServiceWithRetry(ctx context.Context, recoverInfo *datapb.GetChannelRecoveryInfoResponse) (*dataSyncServiceWrapper, error) {
|
func (impl *flusherComponents) buildDataSyncServiceWithRetry(ctx context.Context, recoverInfo *datapb.GetChannelRecoveryInfoResponse) (*dataSyncServiceWrapper, error) {
|
||||||
// Flush all the growing segment that is not created by streaming.
|
// Flush all the growing segment that is not created by streaming.
|
||||||
segmentIDs := make([]int64, 0, len(recoverInfo.GetInfo().UnflushedSegments))
|
for _, segment := range recoverInfo.SegmentsNotCreatedByStreaming {
|
||||||
for _, segment := range recoverInfo.GetInfo().UnflushedSegments {
|
logger := impl.logger.With(
|
||||||
if segment.IsCreatedByStreaming {
|
zap.Int64("collectionID", segment.CollectionId),
|
||||||
continue
|
zap.String("vchannel", recoverInfo.GetInfo().GetChannelName()),
|
||||||
}
|
zap.Int64("partitionID", segment.PartitionId),
|
||||||
|
zap.Int64("segmentID", segment.SegmentId),
|
||||||
|
)
|
||||||
msg := message.NewFlushMessageBuilderV2().
|
msg := message.NewFlushMessageBuilderV2().
|
||||||
WithVChannel(recoverInfo.GetInfo().GetChannelName()).
|
WithVChannel(recoverInfo.GetInfo().GetChannelName()).
|
||||||
WithHeader(&message.FlushMessageHeader{
|
WithHeader(&message.FlushMessageHeader{
|
||||||
CollectionId: recoverInfo.GetInfo().GetCollectionID(),
|
CollectionId: segment.CollectionId,
|
||||||
PartitionId: segment.PartitionID,
|
PartitionId: segment.PartitionId,
|
||||||
SegmentId: segment.ID,
|
SegmentId: segment.SegmentId,
|
||||||
}).
|
}).
|
||||||
WithBody(&message.FlushMessageBody{}).MustBuildMutable()
|
WithBody(&message.FlushMessageBody{}).MustBuildMutable()
|
||||||
if err := retry.Do(ctx, func() error {
|
if err := retry.Do(ctx, func() error {
|
||||||
appendResult, err := impl.wal.Append(ctx, msg)
|
appendResult, err := impl.wal.Append(ctx, msg)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
impl.logger.Warn(
|
logger.Warn("fail to append flush message for segments that not created by streaming service into wal", zap.Error(err))
|
||||||
"fail to append flush message for segments that not created by streaming service into wal",
|
|
||||||
zap.String("vchannel", recoverInfo.GetInfo().GetChannelName()),
|
|
||||||
zap.Error(err))
|
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
impl.logger.Info(
|
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))
|
||||||
"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),
|
|
||||||
)
|
|
||||||
return nil
|
return nil
|
||||||
}, retry.AttemptAlways()); err != nil {
|
}, retry.AttemptAlways()); err != nil {
|
||||||
return nil, err
|
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/message"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/options"
|
"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/streaming/walimpls"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -178,9 +179,14 @@ func (s *catchupScanner) createReaderWithBackoff(ctx context.Context, deliverPol
|
|||||||
})
|
})
|
||||||
backoffTimer.EnableBackoff()
|
backoffTimer.EnableBackoff()
|
||||||
for {
|
for {
|
||||||
|
bufSize := paramtable.Get().StreamingCfg.WALReadAheadBufferLength.GetAsInt()
|
||||||
|
if bufSize < 0 {
|
||||||
|
bufSize = 0
|
||||||
|
}
|
||||||
innerScanner, err := s.innerWAL.Read(ctx, walimpls.ReadOption{
|
innerScanner, err := s.innerWAL.Read(ctx, walimpls.ReadOption{
|
||||||
Name: s.scannerName,
|
Name: s.scannerName,
|
||||||
DeliverPolicy: deliverPolicy,
|
DeliverPolicy: deliverPolicy,
|
||||||
|
ReadAheadBufferSize: bufSize,
|
||||||
})
|
})
|
||||||
if err == nil {
|
if err == nil {
|
||||||
return innerScanner, nil
|
return innerScanner, nil
|
||||||
|
|||||||
@ -127,8 +127,8 @@ func (c *client) Subscribe(options ConsumerOptions) (Consumer, error) {
|
|||||||
|
|
||||||
func (c *client) consume(consumer *consumer) {
|
func (c *client) consume(consumer *consumer) {
|
||||||
defer func() {
|
defer func() {
|
||||||
close(consumer.stopCh)
|
|
||||||
c.wg.Done()
|
c.wg.Done()
|
||||||
|
consumer.wg.Done()
|
||||||
}()
|
}()
|
||||||
|
|
||||||
if err := c.blockUntilInitDone(consumer); err != nil {
|
if err := c.blockUntilInitDone(consumer); err != nil {
|
||||||
@ -161,6 +161,9 @@ func (c *client) consume(consumer *consumer) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
select {
|
select {
|
||||||
|
case <-consumer.ctx.Done():
|
||||||
|
log.Info("Consumer is closed, consumer goroutine exit")
|
||||||
|
return
|
||||||
case <-c.closeCh:
|
case <-c.closeCh:
|
||||||
log.Info("Client is closed, consumer goroutine exit")
|
log.Info("Client is closed, consumer goroutine exit")
|
||||||
return
|
return
|
||||||
|
|||||||
@ -12,6 +12,7 @@
|
|||||||
package client
|
package client
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"context"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
@ -28,7 +29,10 @@ type consumer struct {
|
|||||||
|
|
||||||
startOnce sync.Once
|
startOnce sync.Once
|
||||||
|
|
||||||
stopCh chan struct{}
|
ctx context.Context
|
||||||
|
cancel context.CancelFunc
|
||||||
|
wg sync.WaitGroup
|
||||||
|
|
||||||
msgMutex chan struct{}
|
msgMutex chan struct{}
|
||||||
initCh chan struct{}
|
initCh chan struct{}
|
||||||
messageCh chan common.Message
|
messageCh chan common.Message
|
||||||
@ -54,12 +58,15 @@ func newConsumer(c *client, options ConsumerOptions) (*consumer, error) {
|
|||||||
// only used for
|
// only used for
|
||||||
initCh := make(chan struct{}, 1)
|
initCh := make(chan struct{}, 1)
|
||||||
initCh <- struct{}{}
|
initCh <- struct{}{}
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
return &consumer{
|
return &consumer{
|
||||||
topic: options.Topic,
|
topic: options.Topic,
|
||||||
client: c,
|
client: c,
|
||||||
consumerName: options.SubscriptionName,
|
consumerName: options.SubscriptionName,
|
||||||
options: options,
|
options: options,
|
||||||
stopCh: make(chan struct{}),
|
ctx: ctx,
|
||||||
|
cancel: cancel,
|
||||||
|
wg: sync.WaitGroup{},
|
||||||
msgMutex: make(chan struct{}, 1),
|
msgMutex: make(chan struct{}, 1),
|
||||||
initCh: initCh,
|
initCh: initCh,
|
||||||
messageCh: messageCh,
|
messageCh: messageCh,
|
||||||
@ -85,11 +92,15 @@ func getExistedConsumer(c *client, options ConsumerOptions, msgMutex chan struct
|
|||||||
messageCh = make(chan common.Message, 1)
|
messageCh = make(chan common.Message, 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
return &consumer{
|
return &consumer{
|
||||||
topic: options.Topic,
|
topic: options.Topic,
|
||||||
client: c,
|
client: c,
|
||||||
consumerName: options.SubscriptionName,
|
consumerName: options.SubscriptionName,
|
||||||
options: options,
|
options: options,
|
||||||
|
ctx: ctx,
|
||||||
|
cancel: cancel,
|
||||||
|
wg: sync.WaitGroup{},
|
||||||
msgMutex: msgMutex,
|
msgMutex: msgMutex,
|
||||||
messageCh: messageCh,
|
messageCh: messageCh,
|
||||||
}, nil
|
}, nil
|
||||||
@ -114,6 +125,7 @@ func (c *consumer) MsgMutex() chan struct{} {
|
|||||||
func (c *consumer) Chan() <-chan common.Message {
|
func (c *consumer) Chan() <-chan common.Message {
|
||||||
c.startOnce.Do(func() {
|
c.startOnce.Do(func() {
|
||||||
c.client.wg.Add(1)
|
c.client.wg.Add(1)
|
||||||
|
c.wg.Add(1)
|
||||||
go c.client.consume(c)
|
go c.client.consume(c)
|
||||||
})
|
})
|
||||||
return c.messageCh
|
return c.messageCh
|
||||||
@ -141,7 +153,8 @@ func (c *consumer) Close() {
|
|||||||
close(c.msgMutex)
|
close(c.msgMutex)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
<-c.stopCh
|
c.cancel()
|
||||||
|
c.wg.Wait()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *consumer) GetLatestMsgID() (int64, error) {
|
func (c *consumer) GetLatestMsgID() (int64, error) {
|
||||||
|
|||||||
@ -278,9 +278,9 @@ message VchannelInfo {
|
|||||||
int64 collectionID = 1;
|
int64 collectionID = 1;
|
||||||
string channelName = 2;
|
string channelName = 2;
|
||||||
msg.MsgPosition seek_position = 3;
|
msg.MsgPosition seek_position = 3;
|
||||||
repeated SegmentInfo unflushedSegments = 4; // deprecated, keep it for compatibility
|
repeated SegmentInfo unflushedSegments = 4 [deprecated = true]; // deprecated, keep it for compatibility
|
||||||
repeated SegmentInfo flushedSegments = 5; // deprecated, keep it for compatibility
|
repeated SegmentInfo flushedSegments = 5 [deprecated = true]; // deprecated, keep it for compatibility
|
||||||
repeated SegmentInfo dropped_segments = 6; // deprecated, keep it for compatibility
|
repeated SegmentInfo dropped_segments = 6 [deprecated = true]; // deprecated, keep it for compatibility
|
||||||
repeated int64 unflushedSegmentIds = 7;
|
repeated int64 unflushedSegmentIds = 7;
|
||||||
repeated int64 flushedSegmentIds = 8;
|
repeated int64 flushedSegmentIds = 8;
|
||||||
repeated int64 dropped_segmentIds = 9;
|
repeated int64 dropped_segmentIds = 9;
|
||||||
@ -517,6 +517,13 @@ message GetChannelRecoveryInfoResponse {
|
|||||||
common.Status status = 1;
|
common.Status status = 1;
|
||||||
VchannelInfo info = 2;
|
VchannelInfo info = 2;
|
||||||
schema.CollectionSchema schema = 3;
|
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 {
|
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()
|
wg.Wait()
|
||||||
|
f.testReadWithFastClose(ctx, w)
|
||||||
|
|
||||||
f.assertSortedMessageList(read1)
|
f.assertSortedMessageList(read1)
|
||||||
f.assertSortedMessageList(read2)
|
f.assertSortedMessageList(read2)
|
||||||
@ -309,6 +310,25 @@ func (f *testOneWALImplsFramework) testAppend(ctx context.Context, w WALImpls) (
|
|||||||
return ids, nil
|
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) {
|
func (f *testOneWALImplsFramework) testRead(ctx context.Context, w ROWALImpls, name string) ([]message.ImmutableMessage, error) {
|
||||||
s, err := w.Read(ctx, ReadOption{
|
s, err := w.Read(ctx, ReadOption{
|
||||||
Name: name,
|
Name: name,
|
||||||
|
|||||||
@ -5570,6 +5570,9 @@ type streamingConfig struct {
|
|||||||
WALWriteAheadBufferCapacity ParamItem `refreshable:"true"`
|
WALWriteAheadBufferCapacity ParamItem `refreshable:"true"`
|
||||||
WALWriteAheadBufferKeepalive ParamItem `refreshable:"true"`
|
WALWriteAheadBufferKeepalive ParamItem `refreshable:"true"`
|
||||||
|
|
||||||
|
// read ahead buffer size
|
||||||
|
WALReadAheadBufferLength ParamItem `refreshable:"true"`
|
||||||
|
|
||||||
// logging
|
// logging
|
||||||
LoggingAppendSlowThreshold ParamItem `refreshable:"true"`
|
LoggingAppendSlowThreshold ParamItem `refreshable:"true"`
|
||||||
// memory usage control
|
// 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.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{
|
p.LoggingAppendSlowThreshold = ParamItem{
|
||||||
Key: "streaming.logging.appendSlowThreshold",
|
Key: "streaming.logging.appendSlowThreshold",
|
||||||
Version: "2.6.0",
|
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, 10*time.Second, params.StreamingCfg.TxnDefaultKeepaliveTimeout.GetAsDurationByParse())
|
||||||
assert.Equal(t, 30*time.Second, params.StreamingCfg.WALWriteAheadBufferKeepalive.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, 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, 1*time.Second, params.StreamingCfg.LoggingAppendSlowThreshold.GetAsDurationByParse())
|
||||||
assert.Equal(t, 3*time.Second, params.StreamingCfg.WALRecoveryGracefulCloseTimeout.GetAsDurationByParse())
|
assert.Equal(t, 3*time.Second, params.StreamingCfg.WALRecoveryGracefulCloseTimeout.GetAsDurationByParse())
|
||||||
assert.Equal(t, 100, params.StreamingCfg.WALRecoveryMaxDirtyMessage.GetAsInt())
|
assert.Equal(t, 100, params.StreamingCfg.WALRecoveryMaxDirtyMessage.GetAsInt())
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user