mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
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>
121 lines
3.9 KiB
Go
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]
|
|
}
|