milvus/internal/streamingcoord/server/broadcaster/pending_broadcast_task.go
Zhen Ye 19e5e9f910
enhance: broadcaster will lock resource until message acked (#44508)
issue: #43897

- Return LastConfirmedMessageID when wal append operation.
- Add resource-key-based locker for broadcast-ack operation to protect
the coord state when executing ddl.
- Resource-key-based locker is held until the broadcast operation is
acked.
- ResourceKey support shared and exclusive lock.
- Add FastAck execute ack right away after the broadcast done to speed
up ddl.
- Ack callback will support broadcast message result now.
- Add tombstone for broadcaster to avoid to repeatedly commit DDL and
ABA issue.

---------

Signed-off-by: chyezh <chyezh@outlook.com>
2025-09-24 20:58:05 +08:00

121 lines
3.9 KiB
Go

package broadcaster
import (
"context"
"time"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"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/typeutil"
)
var errBroadcastTaskIsNotDone = errors.New("broadcast task is not done")
// newPendingBroadcastTask creates a new pendingBroadcastTask.
func newPendingBroadcastTask(task *broadcastTask) *pendingBroadcastTask {
msgs := task.PendingBroadcastMessages()
if len(msgs) == 0 {
return nil
}
return &pendingBroadcastTask{
broadcastTask: task,
pendingMessages: msgs,
appendResult: make(map[string]*types.AppendResult, len(msgs)),
BackoffWithInstant: typeutil.NewBackoffWithInstant(typeutil.BackoffTimerConfig{
Default: 10 * time.Second,
Backoff: typeutil.BackoffConfig{
InitialInterval: 10 * time.Millisecond,
Multiplier: 2.0,
MaxInterval: 10 * time.Second,
},
}),
}
}
// pendingBroadcastTask is a task that is pending to be broadcasted.
type pendingBroadcastTask struct {
*broadcastTask
pendingMessages []message.MutableMessage
appendResult map[string]*types.AppendResult
*typeutil.BackoffWithInstant
}
// Execute reexecute the task, return nil if the task is done, otherwise not done.
// Execute can be repeated called until the task is done.
// Same semantics as the `Poll` operation in eventloop.
func (b *pendingBroadcastTask) Execute(ctx context.Context) error {
if err := b.broadcastTask.InitializeRecovery(ctx); err != nil {
b.Logger().Warn("broadcast task initialize recovery failed", zap.Error(err))
return err
}
if len(b.pendingMessages) > 0 {
b.Logger().Debug("broadcast task is polling to make sent...", zap.Int("pendingMessages", len(b.pendingMessages)))
resps := streaming.WAL().AppendMessages(ctx, b.pendingMessages...)
newPendings := make([]message.MutableMessage, 0)
for idx, resp := range resps.Responses {
if resp.Error != nil {
b.Logger().Warn("broadcast task append message failed", zap.Int("idx", idx), zap.Error(resp.Error))
newPendings = append(newPendings, b.pendingMessages[idx])
continue
}
b.appendResult[b.pendingMessages[idx].VChannel()] = resp.AppendResult
}
b.pendingMessages = newPendings
b.Logger().Info("broadcast task make a new broadcast done", zap.Int("backoffRetryMessages", len(b.pendingMessages)))
}
if len(b.pendingMessages) == 0 {
// trigger a fast ack operation when the broadcast operation is done.
if err := b.broadcastTask.FastAck(ctx, b.appendResult); err != nil {
b.Logger().Warn("broadcast task save task failed", zap.Error(err))
return err
}
return nil
}
b.UpdateInstantWithNextBackOff()
return errBroadcastTaskIsNotDone
}
// pendingBroadcastTaskArray is a heap of pendingBroadcastTask.
type pendingBroadcastTaskArray []*pendingBroadcastTask
// Len returns the length of the heap.
func (h pendingBroadcastTaskArray) Len() int {
return len(h)
}
// Less returns true if the element at index i is less than the element at index j.
func (h pendingBroadcastTaskArray) Less(i, j int) bool {
return h[i].NextInstant().Before(h[j].NextInstant())
}
// Swap swaps the elements at indexes i and j.
func (h pendingBroadcastTaskArray) Swap(i, j int) { h[i], h[j] = h[j], h[i] }
// Push pushes the last one at len.
func (h *pendingBroadcastTaskArray) Push(x interface{}) {
// Push and Pop use pointer receivers because they modify the slice's length,
// not just its contents.
*h = append(*h, x.(*pendingBroadcastTask))
}
// Pop pop the last one at len.
func (h *pendingBroadcastTaskArray) Pop() interface{} {
old := *h
n := len(old)
x := old[n-1]
old[n-1] = nil // release the memory of underlying array.
*h = old[0 : n-1]
return x
}
// Peek returns the element at the top of the heap.
// Panics if the heap is empty.
func (h *pendingBroadcastTaskArray) Peek() interface{} {
return (*h)[0]
}