Zhen Ye ce164db1f3
fix: wal state may be unconsistent after recovering from crash (#45092)
issue: #45088, #45086

- Message on control channel should trigger the checkpoint update.
- LastConfrimedMessageID should be recovered from the minimum of
checkpoint or the LastConfirmedMessageID of uncommitted txn.
- Add more log info for wal debugging.

---------

Signed-off-by: chyezh <chyezh@outlook.com>
2025-10-29 16:26:10 +08:00

130 lines
5.2 KiB
Go

package adaptor
import (
"context"
"time"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/mvcc"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/wab"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/utility"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
"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/syncutil"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
// buildInterceptorParams builds the interceptor params for the walimpls.
func buildInterceptorParams(ctx context.Context, underlyingWALImpls walimpls.WALImpls, cp *utility.WALCheckpoint) (*interceptors.InterceptorBuildParam, error) {
var lastConfirmedMessageID message.MessageID
if cp != nil {
// lastConfirmedMessageID is used to promise we can use it to read all messages which timetick is greater than timetick of current message.
// When we send the first time tick message, its timetick is always greater than the timetick of the previous message.
// But for the uncommitted message, its timetick is undetermined, and wal support to recover the uncommitted-txn.
// For protecting the `LastConfirmedMessageID` promise,
// we use the checkpoint (checkpoint is always see the committed message) to promise we can see the uncommitted message
// when using the FirstTimeTickMessage as the poisition to read.
lastConfirmedMessageID = cp.MessageID
}
msg, err := sendFirstTimeTick(ctx, underlyingWALImpls, lastConfirmedMessageID)
if err != nil {
return nil, err
}
capacity := int(paramtable.Get().StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize())
keepalive := paramtable.Get().StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse()
writeAheadBuffer := wab.NewWriteAheadBuffer(
underlyingWALImpls.Channel().Name,
resource.Resource().Logger().With(),
capacity,
keepalive,
msg,
)
mvccManager := mvcc.NewMVCCManager(msg.TimeTick())
return &interceptors.InterceptorBuildParam{
ChannelInfo: underlyingWALImpls.Channel(),
WAL: syncutil.NewFuture[wal.WAL](),
LastTimeTickMessage: msg,
WriteAheadBuffer: writeAheadBuffer,
MVCCManager: mvccManager,
}, nil
}
// sendFirstTimeTick sends the first timetick message to walimpls.
// It is used to
// 1. make a fence operation with the underlying walimpls
// 2. get position of wal to determine the end of current wal.
// 3. make all un-synced messages synced by the timetick message, so the un-synced messages can be seen by the recovery storage.
func sendFirstTimeTick(ctx context.Context, underlyingWALImpls walimpls.WALImpls, lastConfirmedMessageID message.MessageID) (msg message.ImmutableMessage, err error) {
logger := resource.Resource().Logger().With(zap.String("channel", underlyingWALImpls.Channel().String()))
if lastConfirmedMessageID != nil {
logger = logger.With(zap.Stringer("lastConfirmedMessageID", lastConfirmedMessageID))
}
logger.Info("start to sync first time tick")
defer func() {
if err != nil {
logger.Error("sync first time tick failed", zap.Error(err))
return
}
logger.Info("sync first time tick done", zap.String("msgID", msg.MessageID().String()), zap.Uint64("timetick", msg.TimeTick()))
}()
backoffTimer := typeutil.NewBackoffTimer(typeutil.BackoffTimerConfig{
Default: 5 * time.Second,
Backoff: typeutil.BackoffConfig{
InitialInterval: 20 * time.Millisecond,
Multiplier: 2.0,
MaxInterval: 5 * time.Second,
},
})
backoffTimer.EnableBackoff()
var lastErr error
sourceID := paramtable.GetNodeID()
// Send first timetick message to wal before interceptor is ready.
for count := 0; ; count++ {
if count > 0 {
nextTimer, nextBalanceInterval := backoffTimer.NextTimer()
logger.Warn(
"send first time tick failed",
zap.Duration("nextBalanceInterval", nextBalanceInterval),
zap.Int("retryCount", count),
zap.Error(lastErr),
)
select {
case <-ctx.Done():
return nil, ctx.Err()
case <-nextTimer:
}
}
// Sent first timetick message to wal before ready.
// New TT is always greater than all tt on previous streamingnode.
// A fencing operation of underlying WAL is needed to make exclusive produce of topic.
// Otherwise, the TT principle may be violated.
// And sendTsMsg must be done, to help ackManager to get first LastConfirmedMessageID
// !!! Send a timetick message into walimpls directly is safe.
resource.Resource().TSOAllocator().Sync()
ts, err := resource.Resource().TSOAllocator().Allocate(ctx)
if err != nil {
lastErr = errors.Wrap(err, "allocate timestamp failed")
continue
}
msg := timetick.NewTimeTickMsg(ts, lastConfirmedMessageID, sourceID, true)
msgID, err := underlyingWALImpls.Append(ctx, msg)
if err != nil {
lastErr = errors.Wrap(err, "send first timestamp message failed")
continue
}
return msg.IntoImmutableMessage(msgID), nil
}
}