mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
issue: #46540 Empty timetick is just used to sync up the time clock between different component in milvus. So empty timetick can be ignored if we achieve the lsn/mvcc semantic for timetick. Currently, some components need the empty timetick to trigger some operation, such as flush/tsafe. So we only slow down the empty time tick for 5 seconds. <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: with LSN/MVCC semantics consumers only need (a) the first timetick that advances the latest-required-MVCC to unblock MVCC-dependent waits and (b) occasional periodic timeticks (~≤5s) for clock synchronization—therefore frequent non-persisted empty timeticks can be suppressed without breaking MVCC correctness. - Logic removed/simplified: per-message dispatch/consumption of frequent non-persisted empty timeticks is suppressed — an MVCC-aware filter emptyTimeTickSlowdowner (internal/util/pipeline/consuming_slowdown.go) short-circuits frequent empty timeticks in the stream pipeline (internal/util/pipeline/stream_pipeline.go), and the WAL flusher rate-limits non-persisted timetick dispatch to one emission per ~5s (internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go); the delegator exposes GetLatestRequiredMVCCTimeTick to drive the filter (internal/querynodev2/delegator/delegator.go). - Why this does NOT introduce data loss or regressions: the slowdowner always refreshes latestRequiredMVCCTimeTick via GetLatestRequiredMVCCTimeTick and (1) never filters timeticks < latestRequiredMVCCTimeTick (so existing tsafe/flush waits stay unblocked) and (2) always lets the first timetick ≥ latestRequiredMVCCTimeTick pass to notify pending MVCC waits; separately, WAL flusher suppression applies only to non-persisted timeticks and still emits when the 5s threshold elapses, preserving periodic clock-sync messages used by flush/tsafe. - Enhancement summary (where it takes effect): adds GetLatestRequiredMVCCTimeTick on ShardDelegator and LastestMVCCTimeTickGetter, wires emptyTimeTickSlowdowner into NewPipelineWithStream (internal/util/pipeline), and adds WAL flusher rate-limiting + metrics (internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go, pkg/metrics) to reduce CPU/dispatch overhead while keeping MVCC correctness and periodic synchronization. <!-- end of auto-generated comment: release notes by coderabbit.ai --> --------- Signed-off-by: chyezh <chyezh@outlook.com>
217 lines
7.4 KiB
Go
217 lines
7.4 KiB
Go
// Licensed to the LF AI & Data foundation under one
|
|
// or more contributor license agreements. See the NOTICE file
|
|
// distributed with this work for additional information
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
// to you under the Apache License, Version 2.0 (the
|
|
// "License"); you may not use this file except in compliance
|
|
// with the License. You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package flusherimpl
|
|
|
|
import (
|
|
"context"
|
|
"testing"
|
|
|
|
"github.com/cockroachdb/errors"
|
|
"github.com/stretchr/testify/assert"
|
|
"github.com/stretchr/testify/mock"
|
|
|
|
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
|
|
"github.com/milvus-io/milvus/pkg/v2/mocks/streaming/util/mock_message"
|
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
|
)
|
|
|
|
func TestFlushMsgHandler_HandleFlush(t *testing.T) {
|
|
vchannel := "ch-0"
|
|
|
|
// test failed
|
|
wbMgr := writebuffer.NewMockBufferManager(t)
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
|
|
|
msg := message.NewFlushMessageBuilderV2().
|
|
WithBroadcast([]string{vchannel}).
|
|
WithHeader(&message.FlushMessageHeader{
|
|
CollectionId: 0,
|
|
SegmentId: 1,
|
|
}).
|
|
WithBody(&message.FlushMessageBody{}).
|
|
MustBuildBroadcast().
|
|
WithBroadcastID(1).
|
|
SplitIntoMutableMessage()[0]
|
|
|
|
handler := newMsgHandler(wbMgr)
|
|
msgID := mock_message.NewMockMessageID(t)
|
|
im, err := message.AsImmutableFlushMessageV2(msg.IntoImmutableMessage(msgID))
|
|
assert.NoError(t, err)
|
|
err = handler.HandleFlush(im)
|
|
assert.Error(t, err)
|
|
|
|
// test normal
|
|
wbMgr = writebuffer.NewMockBufferManager(t)
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
|
|
|
handler = newMsgHandler(wbMgr)
|
|
err = handler.HandleFlush(im)
|
|
assert.NoError(t, err)
|
|
}
|
|
|
|
func TestFlushMsgHandler_HandleManualFlush(t *testing.T) {
|
|
vchannel := "ch-0"
|
|
|
|
// test failed
|
|
wbMgr := writebuffer.NewMockBufferManager(t)
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
|
|
|
msg := message.NewManualFlushMessageBuilderV2().
|
|
WithBroadcast([]string{vchannel}).
|
|
WithHeader(&message.ManualFlushMessageHeader{
|
|
CollectionId: 0,
|
|
FlushTs: 1000,
|
|
}).
|
|
WithBody(&message.ManualFlushMessageBody{}).
|
|
MustBuildBroadcast().
|
|
WithBroadcastID(1).
|
|
SplitIntoMutableMessage()[0]
|
|
|
|
handler := newMsgHandler(wbMgr)
|
|
msgID := mock_message.NewMockMessageID(t)
|
|
im, err := message.AsImmutableManualFlushMessageV2(msg.WithTimeTick(1000).IntoImmutableMessage(msgID))
|
|
assert.NoError(t, err)
|
|
err = handler.HandleManualFlush(im)
|
|
assert.Error(t, err)
|
|
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Unset()
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
|
err = handler.HandleManualFlush(im)
|
|
assert.Error(t, err)
|
|
|
|
// test normal
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Unset()
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
|
|
|
err = handler.HandleManualFlush(im)
|
|
assert.NoError(t, err)
|
|
}
|
|
|
|
func TestFlushMsgHandler_HandleFlushAll(t *testing.T) {
|
|
vchannel := "ch-0"
|
|
|
|
// test failed
|
|
wbMgr := writebuffer.NewMockBufferManager(t)
|
|
wbMgr.EXPECT().SealAllSegments(mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
|
|
|
msg := message.NewFlushAllMessageBuilderV2().
|
|
WithBroadcast([]string{vchannel}).
|
|
WithHeader(&message.FlushAllMessageHeader{}).
|
|
WithBody(&message.FlushAllMessageBody{}).
|
|
WithProperties(map[string]string{
|
|
"_tt": "1",
|
|
}).
|
|
MustBuildBroadcast().
|
|
WithBroadcastID(1).
|
|
SplitIntoMutableMessage()[0]
|
|
|
|
handler := newMsgHandler(wbMgr)
|
|
msgID := mock_message.NewMockMessageID(t)
|
|
im, err := message.AsImmutableFlushAllMessageV2(msg.IntoImmutableMessage(msgID))
|
|
assert.NoError(t, err)
|
|
err = handler.HandleFlushAll(vchannel, im)
|
|
assert.Error(t, err)
|
|
|
|
wbMgr.EXPECT().SealAllSegments(mock.Anything, mock.Anything).Unset()
|
|
wbMgr.EXPECT().SealAllSegments(mock.Anything, mock.Anything).Return(nil)
|
|
err = handler.HandleFlushAll(vchannel, im)
|
|
assert.Error(t, err)
|
|
|
|
// test normal
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Unset()
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
|
|
|
err = handler.HandleFlushAll(vchannel, im)
|
|
assert.NoError(t, err)
|
|
}
|
|
|
|
func TestFlushMsgHandler_HandlSchemaChange(t *testing.T) {
|
|
vchannel := "ch-0"
|
|
|
|
// test failed
|
|
wbMgr := writebuffer.NewMockBufferManager(t)
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
|
|
|
msg := message.NewSchemaChangeMessageBuilderV2().
|
|
WithBroadcast([]string{vchannel}).
|
|
WithHeader(&message.SchemaChangeMessageHeader{
|
|
CollectionId: 0,
|
|
FlushedSegmentIds: []int64{1},
|
|
}).
|
|
WithBody(&message.SchemaChangeMessageBody{}).
|
|
MustBuildBroadcast().
|
|
WithBroadcastID(1).
|
|
SplitIntoMutableMessage()[0]
|
|
|
|
handler := newMsgHandler(wbMgr)
|
|
msgID := mock_message.NewMockMessageID(t)
|
|
im := message.MustAsImmutableSchemaChangeMessageV2(msg.IntoImmutableMessage(msgID))
|
|
err := handler.HandleSchemaChange(context.Background(), im)
|
|
assert.Error(t, err)
|
|
|
|
// test normal
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Unset()
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
|
|
|
err = handler.HandleSchemaChange(context.Background(), im)
|
|
assert.NoError(t, err)
|
|
}
|
|
|
|
func TestFlushMsgHandler_HandleTruncateCollection(t *testing.T) {
|
|
vchannel := "ch-0"
|
|
|
|
// test failed - SealSegments error
|
|
wbMgr := writebuffer.NewMockBufferManager(t)
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
|
|
|
msg := message.NewTruncateCollectionMessageBuilderV2().
|
|
WithBroadcast([]string{vchannel}).
|
|
WithHeader(&message.TruncateCollectionMessageHeader{
|
|
CollectionId: 0,
|
|
SegmentIds: []int64{1, 2},
|
|
}).
|
|
WithBody(&message.TruncateCollectionMessageBody{}).
|
|
MustBuildBroadcast().
|
|
WithBroadcastID(1).
|
|
SplitIntoMutableMessage()[0]
|
|
msg.WithTimeTick(1000)
|
|
|
|
handler := newMsgHandler(wbMgr)
|
|
msgID := mock_message.NewMockMessageID(t)
|
|
im, err := message.AsImmutableTruncateCollectionMessageV2(msg.IntoImmutableMessage(msgID))
|
|
assert.NoError(t, err)
|
|
err = handler.HandleTruncateCollection(im)
|
|
assert.Error(t, err)
|
|
|
|
// test failed - SealSegments success but FlushChannel error
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Unset()
|
|
wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
|
|
|
err = handler.HandleTruncateCollection(im)
|
|
assert.Error(t, err)
|
|
|
|
// test normal
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Unset()
|
|
wbMgr.EXPECT().FlushChannel(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
|
|
|
err = handler.HandleTruncateCollection(im)
|
|
assert.NoError(t, err)
|
|
}
|