mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 09:08:43 +08:00
fix: wrong execution order of DDL/DCL on secondary (#44886)
issue: #44697, #44696 - The DDL executing order of secondary keep same with order of control channel timetick now. - filtering the control channel operation on shard manager of streamingnode to avoid wrong vchannel of create segment. - fix that the immutable txn message lost replicate header. --------- Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
parent
b497dd0b45
commit
2aa48bf4ca
@ -118,7 +118,7 @@ func createReplicateCreateCollectionMessages() []message.ReplicateMutableMessage
|
||||
immutableMsg := msg.WithLastConfirmedUseMessageID().WithTimeTick(1).IntoImmutableMessage(pulsar2.NewPulsarID(
|
||||
pulsar.NewMessageID(1, 2, 3, 4),
|
||||
))
|
||||
replicateMsgs = append(replicateMsgs, message.NewReplicateMessage("primary", immutableMsg.IntoImmutableMessageProto()))
|
||||
replicateMsgs = append(replicateMsgs, message.MustNewReplicateMessage("primary", immutableMsg.IntoImmutableMessageProto()))
|
||||
}
|
||||
return replicateMsgs
|
||||
}
|
||||
|
||||
@ -121,7 +121,7 @@ func TestReplicateCreateCollection(t *testing.T) {
|
||||
immutableMsg := msg.WithLastConfirmedUseMessageID().WithTimeTick(1).IntoImmutableMessage(pulsar2.NewPulsarID(
|
||||
pulsar.NewMessageID(1, 2, 3, 4),
|
||||
))
|
||||
_, err := streaming.WAL().Replicate().Append(context.Background(), message.NewReplicateMessage("primary", immutableMsg.IntoImmutableMessageProto()))
|
||||
_, err := streaming.WAL().Replicate().Append(context.Background(), message.MustNewReplicateMessage("primary", immutableMsg.IntoImmutableMessageProto()))
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
@ -114,7 +114,10 @@ func (p *ReplicateStreamServer) handleReplicateMessage(req *milvuspb.ReplicateRe
|
||||
p.wg.Add(1)
|
||||
defer p.wg.Done()
|
||||
reqMsg := req.ReplicateMessage.GetMessage()
|
||||
msg := message.NewReplicateMessage(req.ReplicateMessage.SourceClusterId, reqMsg)
|
||||
msg, err := message.NewReplicateMessage(req.ReplicateMessage.SourceClusterId, reqMsg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
sourceTs := msg.ReplicateHeader().TimeTick
|
||||
log.Debug("recv replicate message from client",
|
||||
zap.String("messageID", reqMsg.GetId().GetId()),
|
||||
@ -123,7 +126,7 @@ func (p *ReplicateStreamServer) handleReplicateMessage(req *milvuspb.ReplicateRe
|
||||
)
|
||||
|
||||
// Append message to wal.
|
||||
_, err := streaming.WAL().Replicate().Append(p.streamServer.Context(), msg)
|
||||
_, err = streaming.WAL().Replicate().Append(p.streamServer.Context(), msg)
|
||||
if err == nil {
|
||||
p.sendReplicateResult(sourceTs, msg)
|
||||
return nil
|
||||
|
||||
@ -5,6 +5,7 @@ import (
|
||||
"sync"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
||||
@ -14,6 +15,7 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/replicateutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
|
||||
@ -406,7 +408,10 @@ func (cm *ChannelManager) UpdateReplicateConfiguration(ctx context.Context, resu
|
||||
return nil
|
||||
}
|
||||
|
||||
newIncomingCDCTasks := cm.getNewIncomingTask(config, result.Results)
|
||||
appendResults := lo.MapKeys(result.Results, func(_ *message.AppendResult, key string) string {
|
||||
return funcutil.ToPhysicalChannel(key)
|
||||
})
|
||||
newIncomingCDCTasks := cm.getNewIncomingTask(config, appendResults)
|
||||
if err := resource.Resource().StreamingCatalog().SaveReplicateConfiguration(ctx,
|
||||
&streamingpb.ReplicateConfigurationMeta{ReplicateConfiguration: config.GetReplicateConfiguration()},
|
||||
newIncomingCDCTasks); err != nil {
|
||||
|
||||
@ -10,7 +10,6 @@ import (
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
|
||||
)
|
||||
@ -52,7 +51,7 @@ type ackCallbackScheduler struct {
|
||||
// Initialize initializes the ack scheduler with a list of broadcast tasks.
|
||||
func (s *ackCallbackScheduler) Initialize(tasks []*broadcastTask, tombstoneIDs []uint64, bm *broadcastTaskManager) {
|
||||
// when initializing, the tasks in recovery info may be out of order, so we need to sort them by the broadcastID.
|
||||
sortByBroadcastID(tasks)
|
||||
sortByControlChannelTimeTick(tasks)
|
||||
s.tombstoneScheduler.Initialize(bm, tombstoneIDs)
|
||||
s.pendingAckedTasks = tasks
|
||||
go s.background()
|
||||
@ -99,8 +98,6 @@ func (s *ackCallbackScheduler) background() {
|
||||
// addBroadcastTask adds a broadcast task into the pending acked tasks.
|
||||
func (s *ackCallbackScheduler) addBroadcastTask(task *broadcastTask) error {
|
||||
s.pendingAckedTasks = append(s.pendingAckedTasks, task)
|
||||
sortByBroadcastID(s.pendingAckedTasks) // It's a redundant operation,
|
||||
// once at runtime, the tasks are coming with the order of the broadcastID if they have the conflict resource-key.
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -108,12 +105,6 @@ func (s *ackCallbackScheduler) addBroadcastTask(task *broadcastTask) error {
|
||||
func (s *ackCallbackScheduler) triggerAckCallback() {
|
||||
pendingTasks := make([]*broadcastTask, 0, len(s.pendingAckedTasks))
|
||||
for _, task := range s.pendingAckedTasks {
|
||||
if task.State() != streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING &&
|
||||
task.State() != streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_WAIT_ACK &&
|
||||
task.State() != streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_REPLICATED {
|
||||
s.Logger().Info("task cannot be acked, skip the ack callback", zap.Uint64("broadcastID", task.Header().BroadcastID))
|
||||
continue
|
||||
}
|
||||
g, err := s.rkLocker.FastLock(task.Header().ResourceKeys.Collect()...)
|
||||
if err != nil {
|
||||
s.Logger().Warn("lock is occupied, delay the ack callback", zap.Uint64("broadcastID", task.Header().BroadcastID), zap.Error(err))
|
||||
@ -128,16 +119,21 @@ func (s *ackCallbackScheduler) triggerAckCallback() {
|
||||
|
||||
// doAckCallback executes the ack callback.
|
||||
func (s *ackCallbackScheduler) doAckCallback(bt *broadcastTask, g *lockGuards) (err error) {
|
||||
logger := s.Logger().With(zap.Uint64("broadcastID", bt.Header().BroadcastID))
|
||||
defer func() {
|
||||
g.Unlock()
|
||||
s.triggerChan <- struct{}{}
|
||||
if err == nil {
|
||||
s.Logger().Info("execute ack callback done", zap.Uint64("broadcastID", bt.Header().BroadcastID))
|
||||
logger.Info("execute ack callback done")
|
||||
} else {
|
||||
s.Logger().Warn("execute ack callback failed", zap.Uint64("broadcastID", bt.Header().BroadcastID), zap.Error(err))
|
||||
logger.Warn("execute ack callback failed", zap.Error(err))
|
||||
}
|
||||
}()
|
||||
s.Logger().Info("start to execute ack callback", zap.Uint64("broadcastID", bt.Header().BroadcastID))
|
||||
logger.Info("start to execute ack callback")
|
||||
if err := bt.BlockUntilAllAck(s.notifier.Context()); err != nil {
|
||||
return err
|
||||
}
|
||||
logger.Debug("all vchannels are acked")
|
||||
|
||||
msg, result := bt.BroadcastResult()
|
||||
makeMap := make(map[string]*message.AppendResult, len(result))
|
||||
@ -148,11 +144,11 @@ func (s *ackCallbackScheduler) doAckCallback(bt *broadcastTask, g *lockGuards) (
|
||||
TimeTick: result.TimeTick,
|
||||
}
|
||||
}
|
||||
|
||||
// call the ack callback until done.
|
||||
if err := s.callMessageAckCallbackUntilDone(s.notifier.Context(), msg, makeMap); err != nil {
|
||||
return err
|
||||
}
|
||||
logger.Debug("ack callback done")
|
||||
if err := bt.MarkAckCallbackDone(s.notifier.Context()); err != nil {
|
||||
// The catalog is reliable to write, so we can mark the ack callback done without retrying.
|
||||
return err
|
||||
@ -187,8 +183,9 @@ func (s *ackCallbackScheduler) callMessageAckCallbackUntilDone(ctx context.Conte
|
||||
}
|
||||
}
|
||||
|
||||
func sortByBroadcastID(tasks []*broadcastTask) {
|
||||
// sortByControlChannelTimeTick sorts the tasks by the time tick of the control channel.
|
||||
func sortByControlChannelTimeTick(tasks []*broadcastTask) {
|
||||
sort.Slice(tasks, func(i, j int) bool {
|
||||
return tasks[i].Header().BroadcastID < tasks[j].Header().BroadcastID
|
||||
return tasks[i].ControlChannelTimeTick() < tasks[j].ControlChannelTimeTick()
|
||||
})
|
||||
}
|
||||
|
||||
@ -67,7 +67,7 @@ func newBroadcastTaskManager(protos []*streamingpb.BroadcastTask) *broadcastTask
|
||||
case streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_REPLICATED:
|
||||
// The task is recovered from the remote cluster, so it doesn't hold the resource lock.
|
||||
// but the task execution order should be protected by the order of broadcastID (by ackCallbackScheduler)
|
||||
if isAllDone(task.task) {
|
||||
if task.isControlChannelAcked() || isAllDone(task.task) {
|
||||
pendingAckCallbackTasks = append(pendingAckCallbackTasks, task)
|
||||
}
|
||||
case streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_TOMBSTONE:
|
||||
|
||||
@ -56,7 +56,7 @@ func (b *broadcasterScheduler) AddTask(ctx context.Context, task *pendingBroadca
|
||||
// Wait both request context and the background task context.
|
||||
ctx, _ = contextutil.MergeContext(ctx, b.backgroundTaskNotifier.Context())
|
||||
// wait for all the vchannels acked.
|
||||
result, err := task.BlockUntilAllAck(ctx)
|
||||
result, err := task.BlockUntilDone(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@ -13,6 +13,7 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
)
|
||||
|
||||
// newBroadcastTaskFromProto creates a new broadcast task from the proto.
|
||||
@ -26,11 +27,15 @@ func newBroadcastTaskFromProto(proto *streamingpb.BroadcastTask, metrics *broadc
|
||||
dirty: true, // the task is recovered from the recovery info, so it's persisted.
|
||||
metrics: m,
|
||||
ackCallbackScheduler: ackCallbackScheduler,
|
||||
done: make(chan struct{}),
|
||||
allAcked: make(chan struct{}),
|
||||
}
|
||||
if proto.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_TOMBSTONE {
|
||||
if isAllDone(bt.task) {
|
||||
close(bt.allAcked)
|
||||
}
|
||||
if proto.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_TOMBSTONE {
|
||||
close(bt.done)
|
||||
}
|
||||
return bt
|
||||
}
|
||||
|
||||
@ -51,11 +56,9 @@ func newBroadcastTaskFromBroadcastMessage(msg message.BroadcastMutableMessage, m
|
||||
dirty: false,
|
||||
metrics: m,
|
||||
ackCallbackScheduler: ackCallbackScheduler,
|
||||
done: make(chan struct{}),
|
||||
allAcked: make(chan struct{}),
|
||||
}
|
||||
if bt.task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_TOMBSTONE {
|
||||
close(bt.allAcked)
|
||||
}
|
||||
return bt
|
||||
}
|
||||
|
||||
@ -72,14 +75,16 @@ func newBroadcastTaskFromImmutableMessage(msg message.ImmutableMessage, metrics
|
||||
// broadcastTask is the state of the broadcast task.
|
||||
type broadcastTask struct {
|
||||
log.Binder
|
||||
mu sync.Mutex
|
||||
msg message.BroadcastMutableMessage
|
||||
task *streamingpb.BroadcastTask
|
||||
dirty bool // a flag to indicate that the task has been modified and needs to be saved into the recovery info.
|
||||
metrics *taskMetricsGuard
|
||||
allAcked chan struct{}
|
||||
guards *lockGuards
|
||||
ackCallbackScheduler *ackCallbackScheduler
|
||||
mu sync.Mutex
|
||||
msg message.BroadcastMutableMessage
|
||||
task *streamingpb.BroadcastTask
|
||||
dirty bool // a flag to indicate that the task has been modified and needs to be saved into the recovery info.
|
||||
metrics *taskMetricsGuard
|
||||
done chan struct{}
|
||||
allAcked chan struct{}
|
||||
guards *lockGuards
|
||||
ackCallbackScheduler *ackCallbackScheduler
|
||||
joinAckCallbackScheduled bool // a flag to indicate that the join ack callback is scheduled.
|
||||
}
|
||||
|
||||
// SetLogger sets the logger of the broadcast task.
|
||||
@ -133,6 +138,16 @@ func (b *broadcastTask) Header() *message.BroadcastHeader {
|
||||
return b.msg.BroadcastHeader()
|
||||
}
|
||||
|
||||
// ControlChannelTimeTick returns the time tick of the control channel.
|
||||
func (b *broadcastTask) ControlChannelTimeTick() uint64 {
|
||||
for idx, vc := range b.Header().VChannels {
|
||||
if funcutil.IsControlChannel(vc) {
|
||||
return b.task.AckedCheckpoints[idx].TimeTick
|
||||
}
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
// State returns the State of the broadcast task.
|
||||
func (b *broadcastTask) State() streamingpb.BroadcastTaskState {
|
||||
b.mu.Lock()
|
||||
@ -214,26 +229,48 @@ func (b *broadcastTask) Ack(ctx context.Context, msgs ...message.ImmutableMessag
|
||||
|
||||
// ack acknowledges the message at the specified vchannel.
|
||||
func (b *broadcastTask) ack(ctx context.Context, msgs ...message.ImmutableMessage) (err error) {
|
||||
b.copyAndSetAckedCheckpoints(msgs...)
|
||||
isControlChannelAcked := b.copyAndSetAckedCheckpoints(msgs...)
|
||||
if !b.dirty {
|
||||
return nil
|
||||
}
|
||||
if err := b.saveTaskIfDirty(ctx, b.Logger()); err != nil {
|
||||
return err
|
||||
}
|
||||
if isAllDone(b.task) {
|
||||
|
||||
allDone := isAllDone(b.task)
|
||||
if (isControlChannelAcked || allDone) && !b.joinAckCallbackScheduled {
|
||||
// after 2.6.5, the control channel is always broadcasted, it's used to determine the order of the ack callback operations.
|
||||
// so if the control channel is acked, it should be added to the ack callback scheduler.
|
||||
//
|
||||
// allDone is for the compatibility only for the operation before 2.6.5, the control channel is not broadcasted,
|
||||
b.ackCallbackScheduler.AddTask(b)
|
||||
b.joinAckCallbackScheduled = true
|
||||
}
|
||||
if allDone {
|
||||
close(b.allAcked)
|
||||
b.metrics.ObserveAckAll()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// BlockUntilAllAck blocks until all the vchannels are acked.
|
||||
func (b *broadcastTask) BlockUntilAllAck(ctx context.Context) (*types.BroadcastAppendResult, error) {
|
||||
// hasControlChannel checks if the control channel is broadcasted.
|
||||
// for the operation since 2.6.5, the control channel is always broadcasted.
|
||||
// so it's just a dummy function for compatibility.
|
||||
func (b *broadcastTask) isControlChannelAcked() bool {
|
||||
for idx, vc := range b.Header().VChannels {
|
||||
if funcutil.IsControlChannel(vc) && b.task.AckedCheckpoints[idx] != nil {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// BlockUntilDone blocks until the broadcast task is done.
|
||||
func (b *broadcastTask) BlockUntilDone(ctx context.Context) (*types.BroadcastAppendResult, error) {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
case <-b.allAcked:
|
||||
case <-b.done:
|
||||
_, result := b.BroadcastResult()
|
||||
return &types.BroadcastAppendResult{
|
||||
BroadcastID: b.Header().BroadcastID,
|
||||
@ -242,8 +279,18 @@ func (b *broadcastTask) BlockUntilAllAck(ctx context.Context) (*types.BroadcastA
|
||||
}
|
||||
}
|
||||
|
||||
// BlockUntilAllAck blocks until all the vchannels are acked.
|
||||
func (b *broadcastTask) BlockUntilAllAck(ctx context.Context) error {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-b.allAcked:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// copyAndSetAckedCheckpoints copies the task and set the acked checkpoints.
|
||||
func (b *broadcastTask) copyAndSetAckedCheckpoints(msgs ...message.ImmutableMessage) {
|
||||
func (b *broadcastTask) copyAndSetAckedCheckpoints(msgs ...message.ImmutableMessage) (isControlChannelAcked bool) {
|
||||
task := proto.Clone(b.task).(*streamingpb.BroadcastTask)
|
||||
for _, msg := range msgs {
|
||||
vchannel := msg.VChannel()
|
||||
@ -269,9 +316,13 @@ func (b *broadcastTask) copyAndSetAckedCheckpoints(msgs ...message.ImmutableMess
|
||||
LastConfirmedMessageId: msg.LastConfirmedMessageID().IntoProto(),
|
||||
TimeTick: msg.TimeTick(),
|
||||
}
|
||||
if funcutil.IsControlChannel(vchannel) {
|
||||
isControlChannelAcked = true
|
||||
}
|
||||
}
|
||||
// update current task state.
|
||||
b.task = task
|
||||
return
|
||||
}
|
||||
|
||||
// findIdxOfVChannel finds the index of the vchannel in the broadcast task.
|
||||
@ -339,7 +390,7 @@ func (b *broadcastTask) MarkAckCallbackDone(ctx context.Context) error {
|
||||
defer b.mu.Unlock()
|
||||
if b.task.State != streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_TOMBSTONE {
|
||||
b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_TOMBSTONE
|
||||
close(b.allAcked)
|
||||
close(b.done)
|
||||
b.dirty = true
|
||||
}
|
||||
|
||||
|
||||
@ -10,6 +10,7 @@ import (
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/balance"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/channel"
|
||||
@ -20,6 +21,7 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/replicateutil"
|
||||
)
|
||||
@ -89,6 +91,8 @@ func (s *assignmentServiceImpl) UpdateReplicateConfiguration(ctx context.Context
|
||||
}
|
||||
return nil, err
|
||||
}
|
||||
defer broadcaster.Close()
|
||||
|
||||
msg, err := s.validateReplicateConfiguration(ctx, config)
|
||||
if err != nil {
|
||||
if errors.Is(err, errReplicateConfigurationSame) {
|
||||
@ -141,9 +145,17 @@ func (s *assignmentServiceImpl) validateReplicateConfiguration(ctx context.Conte
|
||||
return nil, errReplicateConfigurationSame
|
||||
}
|
||||
|
||||
controlChannel := streaming.WAL().ControlChannel()
|
||||
pchannels := lo.MapToSlice(latestAssignment.PChannelView.Channels, func(_ channel.ChannelID, channel *channel.PChannelMeta) string {
|
||||
return channel.Name()
|
||||
})
|
||||
broadcastPChannels := lo.Map(pchannels, func(pchannel string, _ int) string {
|
||||
if funcutil.IsOnPhysicalChannel(controlChannel, pchannel) {
|
||||
// return control channel if the control channel is on the pchannel.
|
||||
return controlChannel
|
||||
}
|
||||
return pchannel
|
||||
})
|
||||
|
||||
// validate the configuration itself
|
||||
currentClusterID := paramtable.Get().CommonCfg.ClusterPrefix.GetValue()
|
||||
@ -164,7 +176,7 @@ func (s *assignmentServiceImpl) validateReplicateConfiguration(ctx context.Conte
|
||||
ReplicateConfiguration: config,
|
||||
}).
|
||||
WithBody(&message.AlterReplicateConfigMessageBody{}).
|
||||
WithBroadcast(pchannels).
|
||||
WithBroadcast(broadcastPChannels).
|
||||
MustBuildBroadcast()
|
||||
return b, nil
|
||||
}
|
||||
|
||||
@ -10,6 +10,8 @@ import (
|
||||
"github.com/stretchr/testify/mock"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||
"github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming"
|
||||
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_balancer"
|
||||
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_broadcaster"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer"
|
||||
@ -29,6 +31,10 @@ import (
|
||||
func TestAssignmentService(t *testing.T) {
|
||||
resource.InitForTest()
|
||||
|
||||
mw := mock_streaming.NewMockWALAccesser(t)
|
||||
mw.EXPECT().ControlChannel().Return("by-dev-1_vcchan")
|
||||
streaming.SetWALForTest(mw)
|
||||
|
||||
broadcast.ResetBroadcaster()
|
||||
// Set up the balancer
|
||||
b := mock_balancer.NewMockBalancer(t)
|
||||
@ -44,6 +50,7 @@ func TestAssignmentService(t *testing.T) {
|
||||
mb := mock_broadcaster.NewMockBroadcaster(t)
|
||||
fb.Set(mb)
|
||||
mba.EXPECT().Broadcast(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Maybe()
|
||||
mba.EXPECT().Close().Return().Maybe()
|
||||
mb.EXPECT().WithResourceKeys(mock.Anything, mock.Anything).Return(mba, nil).Maybe()
|
||||
mb.EXPECT().Ack(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
mb.EXPECT().LegacyAck(mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
@ -420,7 +420,7 @@ func newReplicateMessage(clusterID string, sourceClusterID string, timetick ...u
|
||||
WithLastConfirmed(walimplstest.NewTestMessageID(1)).
|
||||
IntoImmutableMessage(walimplstest.NewTestMessageID(1))
|
||||
|
||||
replicateMsg := message.NewReplicateMessage(
|
||||
replicateMsg := message.MustNewReplicateMessage(
|
||||
sourceClusterID,
|
||||
msg.IntoImmutableMessageProto(),
|
||||
)
|
||||
@ -480,7 +480,7 @@ func newReplicateTxnMessage(clusterID string, sourceClusterID string, timetick .
|
||||
immutables := newImmutableTxnMessage(sourceClusterID, timetick...)
|
||||
replicateMsgs := []message.MutableMessage{}
|
||||
for _, immutable := range immutables {
|
||||
replicateMsg := message.NewReplicateMessage(
|
||||
replicateMsg := message.MustNewReplicateMessage(
|
||||
sourceClusterID,
|
||||
immutable.IntoImmutableMessageProto(),
|
||||
)
|
||||
|
||||
@ -43,7 +43,7 @@ func (s *replicateTxnHelper) BeginDone(txn *message.TxnContext) {
|
||||
|
||||
func (s *replicateTxnHelper) AddNewMessage(txn *message.TxnContext, replicateHeader *message.ReplicateHeader) error {
|
||||
if s.currentTxn == nil {
|
||||
return status.NewReplicateViolation("add new txn message without new txn, incoming: %d", s.currentTxn.TxnID, txn.TxnID)
|
||||
return status.NewReplicateViolation("add new txn message without new txn, incoming: %d", txn.TxnID)
|
||||
}
|
||||
if s.currentTxn.TxnID != txn.TxnID {
|
||||
return status.NewReplicateViolation("add new txn message with different txn, current: %d, incoming: %d", s.currentTxn.TxnID, txn.TxnID)
|
||||
|
||||
@ -16,6 +16,7 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
)
|
||||
|
||||
const interceptorName = "shard"
|
||||
@ -52,8 +53,10 @@ func (impl *shardInterceptor) Name() string {
|
||||
// DoAppend assigns segment for every partition in the message.
|
||||
func (impl *shardInterceptor) DoAppend(ctx context.Context, msg message.MutableMessage, appendOp interceptors.Append) (msgID message.MessageID, err error) {
|
||||
op, ok := impl.ops[msg.MessageType()]
|
||||
if ok {
|
||||
if ok && !funcutil.IsControlChannel(msg.VChannel()) {
|
||||
// If the message type is registered in the interceptor, use the registered operation.
|
||||
// control channel message is only used to determine the DDL/DCL order,
|
||||
// perform no effect on the shard manager, so skip it.
|
||||
return op(ctx, msg, appendOp)
|
||||
}
|
||||
return appendOp(ctx, msg)
|
||||
|
||||
@ -229,7 +229,7 @@ func TestManagerFromReplcateMessage(t *testing.T) {
|
||||
Keepalive: 10 * time.Millisecond,
|
||||
}).
|
||||
IntoImmutableMessage(walimplstest.NewTestMessageID(1))
|
||||
replicateMsg := message.NewReplicateMessage("test2", immutableMsg.IntoImmutableMessageProto()).WithTimeTick(2)
|
||||
replicateMsg := message.MustNewReplicateMessage("test2", immutableMsg.IntoImmutableMessageProto()).WithTimeTick(2)
|
||||
|
||||
session, err := manager.BeginNewTxn(context.Background(), message.MustAsMutableBeginTxnMessageV2(replicateMsg))
|
||||
assert.NoError(t, err)
|
||||
|
||||
@ -150,7 +150,9 @@ func (r *recoveryStorageImpl) ObserveMessage(ctx context.Context, msg message.Im
|
||||
return err
|
||||
}
|
||||
}
|
||||
if funcutil.IsControlChannel(msg.VChannel()) {
|
||||
if funcutil.IsControlChannel(msg.VChannel()) && msg.MessageType() != message.MessageTypeAlterReplicateConfig {
|
||||
// message on control channel except AlterReplicateConfig message is just used to determine the DDL/DCL order,
|
||||
// will not affect the recovery storage, so skip it.
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
@ -30,7 +30,7 @@ func TestUpdateCheckpoint(t *testing.T) {
|
||||
assert.Nil(t, rs.checkpoint.ReplicateCheckpoint.MessageID)
|
||||
assert.Zero(t, rs.checkpoint.ReplicateCheckpoint.TimeTick)
|
||||
|
||||
replicateMsg := message.NewReplicateMessage("test3", message.NewCreateDatabaseMessageBuilderV2().
|
||||
replicateMsg := message.MustNewReplicateMessage("test3", message.NewCreateDatabaseMessageBuilderV2().
|
||||
WithHeader(&message.CreateDatabaseMessageHeader{}).
|
||||
WithBody(&message.CreateDatabaseMessageBody{}).
|
||||
WithVChannel("test3-rootcoord-dml_0").
|
||||
|
||||
@ -75,12 +75,14 @@ func (b *TxnBuffer) handleBeginTxn(msg message.ImmutableMessage) {
|
||||
return
|
||||
}
|
||||
if _, ok := b.builders[beginMsg.TxnContext().TxnID]; ok {
|
||||
// Because the wal on secondary node may replicate the same txn message, so we need to reset the txn from buffer to avoid
|
||||
// the txn body repeated.
|
||||
b.logger.Warn(
|
||||
"txn id already exist, so ignore the repeated begin txn message",
|
||||
"txn id already exist, rollback the txn from buffer",
|
||||
zap.Int64("txnID", int64(beginMsg.TxnContext().TxnID)),
|
||||
zap.Any("messageID", beginMsg.MessageID()),
|
||||
)
|
||||
return
|
||||
b.rollbackTxn(beginMsg.TxnContext().TxnID)
|
||||
}
|
||||
b.builders[beginMsg.TxnContext().TxnID] = message.NewImmutableTxnMessageBuilder(beginMsg)
|
||||
b.bytes += beginMsg.EstimateSize()
|
||||
@ -145,9 +147,13 @@ func (b *TxnBuffer) handleRollbackTxn(msg message.ImmutableMessage) {
|
||||
zap.Int64("txnID", int64(rollbackMsg.TxnContext().TxnID)),
|
||||
zap.Any("messageID", rollbackMsg.MessageID()),
|
||||
)
|
||||
if builder, ok := b.builders[rollbackMsg.TxnContext().TxnID]; ok {
|
||||
b.rollbackTxn(rollbackMsg.TxnContext().TxnID)
|
||||
}
|
||||
|
||||
func (b *TxnBuffer) rollbackTxn(txnID message.TxnID) {
|
||||
if builder, ok := b.builders[txnID]; ok {
|
||||
// just drop the txn from buffer.
|
||||
delete(b.builders, rollbackMsg.TxnContext().TxnID)
|
||||
delete(b.builders, txnID)
|
||||
b.bytes -= builder.EstimateSize()
|
||||
b.metrics.ObserveTxn(message.TxnStateRollbacked)
|
||||
}
|
||||
|
||||
@ -950,6 +950,54 @@ func (_c *MockMutableMessage_WithOldVersion_Call) RunAndReturn(run func() messag
|
||||
return _c
|
||||
}
|
||||
|
||||
// WithReplicateHeader provides a mock function with given fields: rh
|
||||
func (_m *MockMutableMessage) WithReplicateHeader(rh *message.ReplicateHeader) message.MutableMessage {
|
||||
ret := _m.Called(rh)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for WithReplicateHeader")
|
||||
}
|
||||
|
||||
var r0 message.MutableMessage
|
||||
if rf, ok := ret.Get(0).(func(*message.ReplicateHeader) message.MutableMessage); ok {
|
||||
r0 = rf(rh)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(message.MutableMessage)
|
||||
}
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockMutableMessage_WithReplicateHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'WithReplicateHeader'
|
||||
type MockMutableMessage_WithReplicateHeader_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// WithReplicateHeader is a helper method to define mock.On call
|
||||
// - rh *message.ReplicateHeader
|
||||
func (_e *MockMutableMessage_Expecter) WithReplicateHeader(rh interface{}) *MockMutableMessage_WithReplicateHeader_Call {
|
||||
return &MockMutableMessage_WithReplicateHeader_Call{Call: _e.mock.On("WithReplicateHeader", rh)}
|
||||
}
|
||||
|
||||
func (_c *MockMutableMessage_WithReplicateHeader_Call) Run(run func(rh *message.ReplicateHeader)) *MockMutableMessage_WithReplicateHeader_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(*message.ReplicateHeader))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockMutableMessage_WithReplicateHeader_Call) Return(_a0 message.MutableMessage) *MockMutableMessage_WithReplicateHeader_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockMutableMessage_WithReplicateHeader_Call) RunAndReturn(run func(*message.ReplicateHeader) message.MutableMessage) *MockMutableMessage_WithReplicateHeader_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// WithTimeTick provides a mock function with given fields: tt
|
||||
func (_m *MockMutableMessage) WithTimeTick(tt uint64) message.MutableMessage {
|
||||
ret := _m.Called(tt)
|
||||
|
||||
@ -53,20 +53,23 @@ func NewImmutableMesasge(
|
||||
}
|
||||
}
|
||||
|
||||
// MustNewReplicateMessage creates a new replicate message.
|
||||
func MustNewReplicateMessage(clustrID string, im *commonpb.ImmutableMessage) ReplicateMutableMessage {
|
||||
m, err := NewReplicateMessage(clustrID, im)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return m
|
||||
}
|
||||
|
||||
// NewReplicateMessage creates a new replicate message.
|
||||
func NewReplicateMessage(clustrID string, im *commonpb.ImmutableMessage) ReplicateMutableMessage {
|
||||
func NewReplicateMessage(clustrID string, im *commonpb.ImmutableMessage) (ReplicateMutableMessage, error) {
|
||||
messageID := MustUnmarshalMessageID(im.GetId())
|
||||
msg := NewImmutableMesasge(messageID, im.GetPayload(), im.GetProperties()).(*immutableMessageImpl)
|
||||
rh, err := EncodeProto(&messagespb.ReplicateHeader{
|
||||
ClusterId: clustrID,
|
||||
MessageId: msg.MessageID().IntoProto(),
|
||||
LastConfirmedMessageId: msg.LastConfirmedMessageID().IntoProto(),
|
||||
TimeTick: msg.TimeTick(),
|
||||
Vchannel: msg.VChannel(),
|
||||
})
|
||||
if err != nil {
|
||||
panic("failed to encode replicate header")
|
||||
if msg.ReplicateHeader() != nil {
|
||||
return nil, errors.New("message is already a replicate message")
|
||||
}
|
||||
|
||||
m := &messageImpl{
|
||||
payload: msg.payload,
|
||||
properties: msg.properties.Clone(),
|
||||
@ -75,8 +78,14 @@ func NewReplicateMessage(clustrID string, im *commonpb.ImmutableMessage) Replica
|
||||
m.properties.Delete(messageTimeTick)
|
||||
m.properties.Delete(messageLastConfirmed)
|
||||
m.properties.Delete(messageWALTerm)
|
||||
m.properties.Set(messageReplicateMesssageHeader, rh)
|
||||
return m
|
||||
m.WithReplicateHeader(&ReplicateHeader{
|
||||
ClusterID: clustrID,
|
||||
MessageID: msg.MessageID(),
|
||||
LastConfirmedMessageID: msg.LastConfirmedMessageID(),
|
||||
TimeTick: msg.TimeTick(),
|
||||
VChannel: msg.VChannel(),
|
||||
})
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func MilvusMessageToImmutableMessage(im *commonpb.ImmutableMessage) ImmutableMessage {
|
||||
@ -381,10 +390,10 @@ func newImmutableTxnMesasgeFromWAL(
|
||||
for idx, m := range body {
|
||||
body[idx] = m.(*immutableMessageImpl).cloneForTxnBody(commit.TimeTick(), commit.LastConfirmedMessageID())
|
||||
}
|
||||
|
||||
immutableMessage := msg.WithTimeTick(commit.TimeTick()).
|
||||
WithLastConfirmed(commit.LastConfirmedMessageID()).
|
||||
WithTxnContext(*commit.TxnContext()).
|
||||
WithReplicateHeader(commit.ReplicateHeader()).
|
||||
IntoImmutableMessage(commit.MessageID())
|
||||
return &immutableTxnMessageImpl{
|
||||
immutableMessageImpl: *immutableMessage.(*immutableMessageImpl),
|
||||
|
||||
@ -72,7 +72,14 @@ func TestImmutableTxnBuilder(t *testing.T) {
|
||||
WithBody(&message.CommitTxnMessageBody{}).
|
||||
WithVChannel("v1").
|
||||
MustBuildMutable()
|
||||
immutableCommit := commit.WithTimeTick(3).WithTxnContext(txnCtx).WithLastConfirmed(msgID).IntoImmutableMessage(msgID)
|
||||
rh := message.ReplicateHeader{
|
||||
ClusterID: "by-dev",
|
||||
MessageID: msgID,
|
||||
LastConfirmedMessageID: msgID,
|
||||
TimeTick: 3,
|
||||
VChannel: "v1",
|
||||
}
|
||||
immutableCommit := commit.WithTimeTick(3).WithTxnContext(txnCtx).WithLastConfirmed(msgID).WithReplicateHeader(&rh).IntoImmutableMessage(msgID)
|
||||
log.Info("test", zap.Object("msg", immutableCommit))
|
||||
|
||||
assert.NotZero(t, b.EstimateSize())
|
||||
@ -81,6 +88,11 @@ func TestImmutableTxnBuilder(t *testing.T) {
|
||||
assert.Len(t, msgs, 1)
|
||||
immutableTxnMsg, err := b.Build(message.MustAsImmutableCommitTxnMessageV2(immutableCommit))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, "by-dev", immutableTxnMsg.ReplicateHeader().ClusterID)
|
||||
assert.Equal(t, uint64(3), immutableTxnMsg.ReplicateHeader().TimeTick)
|
||||
assert.Equal(t, "v1", immutableTxnMsg.ReplicateHeader().VChannel)
|
||||
assert.Equal(t, msgID, immutableTxnMsg.ReplicateHeader().MessageID)
|
||||
assert.Equal(t, msgID, immutableTxnMsg.ReplicateHeader().LastConfirmedMessageID)
|
||||
log.Info("test", zap.Object("msg", immutableTxnMsg))
|
||||
}
|
||||
|
||||
@ -96,7 +108,7 @@ func TestReplicateBuilder(t *testing.T) {
|
||||
msgID := walimplstest.NewTestMessageID(1)
|
||||
immutableMsg := msgs[0].WithTimeTick(100).WithLastConfirmed(msgID).IntoImmutableMessage(msgID)
|
||||
|
||||
replicateMsg := message.NewReplicateMessage("by-dev", immutableMsg.IntoImmutableMessageProto())
|
||||
replicateMsg := message.MustNewReplicateMessage("by-dev", immutableMsg.IntoImmutableMessageProto())
|
||||
assert.NotNil(t, replicateMsg)
|
||||
assert.Equal(t, "by-dev", replicateMsg.ReplicateHeader().ClusterID)
|
||||
assert.Equal(t, uint64(100), replicateMsg.ReplicateHeader().TimeTick)
|
||||
|
||||
@ -112,6 +112,10 @@ type MutableMessage interface {
|
||||
// !!! preserved for streaming system internal usage, don't call it outside of streaming system.
|
||||
WithTxnContext(txnCtx TxnContext) MutableMessage
|
||||
|
||||
// WithReplicateHeader sets the replicate header of current message.
|
||||
// !!! preserved for streaming system internal usage, don't call it outside of streaming system.
|
||||
WithReplicateHeader(rh *ReplicateHeader) MutableMessage
|
||||
|
||||
// IntoImmutableMessage converts the mutable message to immutable message.
|
||||
IntoImmutableMessage(msgID MessageID) ImmutableMessage
|
||||
}
|
||||
|
||||
@ -101,6 +101,28 @@ func (m *messageImpl) WithWALTerm(term int64) MutableMessage {
|
||||
return m
|
||||
}
|
||||
|
||||
// WithReplicateHeader sets the replicate header of current message.
|
||||
func (m *messageImpl) WithReplicateHeader(rh *ReplicateHeader) MutableMessage {
|
||||
if rh == nil {
|
||||
return m
|
||||
}
|
||||
if m.properties.Exist(messageReplicateMesssageHeader) {
|
||||
panic("replicate header already set in properties of message")
|
||||
}
|
||||
rhProto, err := EncodeProto(&messagespb.ReplicateHeader{
|
||||
ClusterId: rh.ClusterID,
|
||||
MessageId: rh.MessageID.IntoProto(),
|
||||
LastConfirmedMessageId: rh.LastConfirmedMessageID.IntoProto(),
|
||||
TimeTick: rh.TimeTick,
|
||||
Vchannel: rh.VChannel,
|
||||
})
|
||||
if err != nil {
|
||||
panic("should not happen on replicate header proto")
|
||||
}
|
||||
m.properties.Set(messageReplicateMesssageHeader, rhProto)
|
||||
return m
|
||||
}
|
||||
|
||||
// WithTimeTick sets the time tick of current message.
|
||||
func (m *messageImpl) WithTimeTick(tt uint64) MutableMessage {
|
||||
m.properties.Set(messageTimeTick, EncodeUint64(tt))
|
||||
|
||||
@ -300,6 +300,11 @@ func IsControlChannel(channel string) bool {
|
||||
return strings.HasSuffix(channel, ControlChannelSuffix)
|
||||
}
|
||||
|
||||
// IsOnPhysicalChannel checks if the channel is on the physical channel
|
||||
func IsOnPhysicalChannel(channel string, physicalChannel string) bool {
|
||||
return strings.HasPrefix(channel, physicalChannel)
|
||||
}
|
||||
|
||||
// ToPhysicalChannel get physical channel name from virtual channel name
|
||||
func ToPhysicalChannel(vchannel string) string {
|
||||
if IsPhysicalChannel(vchannel) {
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user