mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
enhance: get cchannel before build message (#44229)
issue: #43897 - support never expire txn message. Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
parent
4a01c726f3
commit
cbe4c3d231
@ -9,29 +9,9 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
)
|
||||
|
||||
// routePChannel routes the pchannel of the vchannel.
|
||||
// If the vchannel is control channel, it will return the pchannel of the cchannel.
|
||||
// Otherwise, it will return the pchannel of the vchannel.
|
||||
// TODO: support cross-cluster replication, so the remote vchannel should be mapping to the pchannel of the local cluster.
|
||||
func (w *walAccesserImpl) routePChannel(ctx context.Context, vchannel string) (string, error) {
|
||||
if vchannel == message.ControlChannel {
|
||||
assignments, err := w.streamingCoordClient.Assignment().GetLatestAssignments(ctx)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return assignments.PChannelOfCChannel(), nil
|
||||
}
|
||||
pchannel := funcutil.ToPhysicalChannel(vchannel)
|
||||
return pchannel, nil
|
||||
}
|
||||
|
||||
// appendToWAL appends the message to the wal.
|
||||
func (w *walAccesserImpl) appendToWAL(ctx context.Context, msg message.MutableMessage) (*types.AppendResult, error) {
|
||||
pchannel, err := w.routePChannel(ctx, msg.VChannel())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// get producer of pchannel.
|
||||
pchannel := funcutil.ToPhysicalChannel(msg.VChannel())
|
||||
p := w.getProducer(pchannel)
|
||||
return p.Produce(ctx, msg)
|
||||
}
|
||||
|
||||
@ -103,7 +103,7 @@ func (m *delegatorMsgstreamAdaptor) Seek(ctx context.Context, msgPositions []*ms
|
||||
zap.Uint64("timestamp", position.GetTimestamp()),
|
||||
)
|
||||
handler := adaptor.NewMsgPackAdaptorHandler()
|
||||
if position.GetChannelName() == message.ControlChannel {
|
||||
if funcutil.IsControlChannel(position.GetChannelName()) {
|
||||
panic("should never seek from control channel at delegator msgstream adaptor")
|
||||
}
|
||||
pchannel := funcutil.ToPhysicalChannel(position.GetChannelName())
|
||||
|
||||
@ -108,6 +108,10 @@ type Balancer interface {
|
||||
|
||||
// WALAccesser is the interfaces to interact with the milvus write ahead log.
|
||||
type WALAccesser interface {
|
||||
// ControlChannel returns the control channel name of the wal.
|
||||
// It will return the channel name of the control channel of the wal.
|
||||
ControlChannel() string
|
||||
|
||||
// Balancer returns the balancer management of the wal.
|
||||
Balancer() Balancer
|
||||
|
||||
|
||||
@ -29,6 +29,7 @@ import (
|
||||
"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/streaming/walimpls/impls/rmq"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
)
|
||||
|
||||
var expectErr = make(chan error, 10)
|
||||
@ -138,6 +139,10 @@ func (n *noopTxn) Rollback(ctx context.Context) error {
|
||||
|
||||
type noopWALAccesser struct{}
|
||||
|
||||
func (n *noopWALAccesser) ControlChannel() string {
|
||||
return funcutil.GetControlChannel("noop")
|
||||
}
|
||||
|
||||
func (n *noopWALAccesser) Balancer() Balancer {
|
||||
return &noopBalancer{}
|
||||
}
|
||||
|
||||
@ -17,6 +17,7 @@ import (
|
||||
"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/conc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
@ -70,6 +71,15 @@ func (w *walAccesserImpl) Local() Local {
|
||||
return localServiceImpl{w}
|
||||
}
|
||||
|
||||
// ControlChannel returns the control channel name of the wal.
|
||||
func (w *walAccesserImpl) ControlChannel() string {
|
||||
last, err := w.streamingCoordClient.Assignment().GetLatestAssignments(context.Background())
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return funcutil.GetControlChannel(last.PChannelOfCChannel())
|
||||
}
|
||||
|
||||
// RawAppend writes a record to the log.
|
||||
func (w *walAccesserImpl) RawAppend(ctx context.Context, msg message.MutableMessage, opts ...AppendOption) (*types.AppendResult, error) {
|
||||
assertValidMessage(msg)
|
||||
@ -94,10 +104,7 @@ func (w *walAccesserImpl) Read(ctx context.Context, opts ReadOption) Scanner {
|
||||
}
|
||||
|
||||
if opts.VChannel != "" {
|
||||
pchannel, err := w.routePChannel(ctx, opts.VChannel)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
pchannel := funcutil.ToPhysicalChannel(opts.VChannel)
|
||||
if opts.PChannel != "" && opts.PChannel != pchannel {
|
||||
panic("pchannel is not match with vchannel")
|
||||
}
|
||||
|
||||
@ -243,6 +243,51 @@ func (_c *MockWALAccesser_Broadcast_Call) RunAndReturn(run func() streaming.Broa
|
||||
return _c
|
||||
}
|
||||
|
||||
// ControlChannel provides a mock function with no fields
|
||||
func (_m *MockWALAccesser) ControlChannel() string {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for ControlChannel")
|
||||
}
|
||||
|
||||
var r0 string
|
||||
if rf, ok := ret.Get(0).(func() string); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
r0 = ret.Get(0).(string)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockWALAccesser_ControlChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ControlChannel'
|
||||
type MockWALAccesser_ControlChannel_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// ControlChannel is a helper method to define mock.On call
|
||||
func (_e *MockWALAccesser_Expecter) ControlChannel() *MockWALAccesser_ControlChannel_Call {
|
||||
return &MockWALAccesser_ControlChannel_Call{Call: _e.mock.On("ControlChannel")}
|
||||
}
|
||||
|
||||
func (_c *MockWALAccesser_ControlChannel_Call) Run(run func()) *MockWALAccesser_ControlChannel_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockWALAccesser_ControlChannel_Call) Return(_a0 string) *MockWALAccesser_ControlChannel_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockWALAccesser_ControlChannel_Call) RunAndReturn(run func() string) *MockWALAccesser_ControlChannel_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Local provides a mock function with no fields
|
||||
func (_m *MockWALAccesser) Local() streaming.Local {
|
||||
ret := _m.Called()
|
||||
|
||||
@ -18,6 +18,7 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/options"
|
||||
"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/syncutil"
|
||||
)
|
||||
@ -208,6 +209,11 @@ func (impl *WALFlusherImpl) dispatch(msg message.ImmutableMessage) (err error) {
|
||||
}
|
||||
}()
|
||||
|
||||
// wal flusher will not handle the control channel message.
|
||||
if funcutil.IsControlChannel(msg.VChannel()) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Do the data sync service management here.
|
||||
switch msg.MessageType() {
|
||||
case message.MessageTypeCreateCollection:
|
||||
|
||||
@ -2,6 +2,7 @@ package txn
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math"
|
||||
"sync"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil"
|
||||
@ -129,6 +130,9 @@ func (s *TxnSession) isExpiredOrDone(ts uint64) bool {
|
||||
|
||||
// expiredTimeTick returns the expired time tick of the session.
|
||||
func (s *TxnSession) expiredTimeTick() uint64 {
|
||||
if s.txnContext.Keepalive == message.TxnKeepaliveInfinite {
|
||||
return math.MaxUint64
|
||||
}
|
||||
return tsoutil.AddPhysicalDurationOnTs(s.lastTimetick, s.txnContext.Keepalive)
|
||||
}
|
||||
|
||||
|
||||
@ -15,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/syncutil"
|
||||
)
|
||||
@ -145,6 +146,9 @@ func (r *recoveryStorageImpl) ObserveMessage(ctx context.Context, msg message.Im
|
||||
return err
|
||||
}
|
||||
}
|
||||
if funcutil.IsControlChannel(msg.VChannel()) {
|
||||
return nil
|
||||
}
|
||||
|
||||
r.mu.Lock()
|
||||
defer r.mu.Unlock()
|
||||
@ -238,7 +242,7 @@ func (r *recoveryStorageImpl) observeMessage(msg message.ImmutableMessage) {
|
||||
// The incoming message id is always sorted with timetick.
|
||||
func (r *recoveryStorageImpl) handleMessage(msg message.ImmutableMessage) {
|
||||
if msg.VChannel() != "" && msg.MessageType() != message.MessageTypeCreateCollection &&
|
||||
msg.MessageType() != message.MessageTypeDropCollection && r.vchannels[msg.VChannel()] == nil && msg.VChannel() != message.ControlChannel {
|
||||
msg.MessageType() != message.MessageTypeDropCollection && r.vchannels[msg.VChannel()] == nil && !funcutil.IsControlChannel(msg.VChannel()) {
|
||||
r.detectInconsistency(msg, "vchannel not found")
|
||||
}
|
||||
|
||||
|
||||
@ -164,13 +164,13 @@ func recoverMessageFromHeader(tsMsg msgstream.TsMsg, msg message.ImmutableMessag
|
||||
}
|
||||
// insertMsg has multiple partition and segment assignment is done by insert message header.
|
||||
// so recover insert message from header before send it.
|
||||
return recoverInsertMsgFromHeader(tsMsg.(*msgstream.InsertMsg), insertMessage.Header(), msg.TimeTick())
|
||||
return recoverInsertMsgFromHeader(tsMsg.(*msgstream.InsertMsg), insertMessage)
|
||||
case message.MessageTypeDelete:
|
||||
deleteMessage, err := message.AsImmutableDeleteMessageV1(msg)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "Failed to convert message to delete message")
|
||||
}
|
||||
return recoverDeleteMsgFromHeader(tsMsg.(*msgstream.DeleteMsg), deleteMessage.Header(), msg.TimeTick())
|
||||
return recoverDeleteMsgFromHeader(tsMsg.(*msgstream.DeleteMsg), deleteMessage)
|
||||
case message.MessageTypeImport:
|
||||
importMessage, err := message.AsImmutableImportMessageV1(msg)
|
||||
if err != nil {
|
||||
@ -183,7 +183,10 @@ func recoverMessageFromHeader(tsMsg msgstream.TsMsg, msg message.ImmutableMessag
|
||||
}
|
||||
|
||||
// recoverInsertMsgFromHeader recovers insert message from header.
|
||||
func recoverInsertMsgFromHeader(insertMsg *msgstream.InsertMsg, header *message.InsertMessageHeader, timetick uint64) (msgstream.TsMsg, error) {
|
||||
func recoverInsertMsgFromHeader(insertMsg *msgstream.InsertMsg, msg message.ImmutableInsertMessageV1) (msgstream.TsMsg, error) {
|
||||
header := msg.Header()
|
||||
timetick := msg.TimeTick()
|
||||
|
||||
if insertMsg.GetCollectionID() != header.GetCollectionId() {
|
||||
panic("unreachable code, collection id is not equal")
|
||||
}
|
||||
@ -208,10 +211,14 @@ func recoverInsertMsgFromHeader(insertMsg *msgstream.InsertMsg, header *message.
|
||||
}
|
||||
insertMsg.Timestamps = timestamps
|
||||
insertMsg.Base.Timestamp = timetick
|
||||
insertMsg.ShardName = msg.VChannel()
|
||||
return insertMsg, nil
|
||||
}
|
||||
|
||||
func recoverDeleteMsgFromHeader(deleteMsg *msgstream.DeleteMsg, header *message.DeleteMessageHeader, timetick uint64) (msgstream.TsMsg, error) {
|
||||
func recoverDeleteMsgFromHeader(deleteMsg *msgstream.DeleteMsg, msg message.ImmutableDeleteMessageV1) (msgstream.TsMsg, error) {
|
||||
header := msg.Header()
|
||||
timetick := msg.TimeTick()
|
||||
|
||||
if deleteMsg.GetCollectionID() != header.GetCollectionId() {
|
||||
panic("unreachable code, collection id is not equal")
|
||||
}
|
||||
@ -220,6 +227,7 @@ func recoverDeleteMsgFromHeader(deleteMsg *msgstream.DeleteMsg, header *message.
|
||||
timestamps[i] = timetick
|
||||
}
|
||||
deleteMsg.Timestamps = timestamps
|
||||
deleteMsg.ShardName = msg.VChannel()
|
||||
return deleteMsg, nil
|
||||
}
|
||||
|
||||
|
||||
@ -2,6 +2,7 @@ package message
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"reflect"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
@ -293,6 +294,9 @@ type ImmutableTxnMessageBuilder struct {
|
||||
|
||||
// ExpiredTimeTick returns the expired time tick of the txn.
|
||||
func (b *ImmutableTxnMessageBuilder) ExpiredTimeTick() uint64 {
|
||||
if b.txnCtx.Keepalive == TxnKeepaliveInfinite {
|
||||
return math.MaxUint64
|
||||
}
|
||||
if len(b.messages) > 0 {
|
||||
return tsoutil.AddPhysicalDurationOnTs(b.messages[len(b.messages)-1].TimeTick(), b.txnCtx.Keepalive)
|
||||
}
|
||||
|
||||
@ -19,6 +19,10 @@ const (
|
||||
TxnStateRollbacked TxnState = messagespb.TxnState_TxnRollbacked
|
||||
|
||||
NonTxnID = TxnID(-1)
|
||||
|
||||
// TxnKeepaliveInfinite is the infinite keepalive duration.
|
||||
// If the keepalive is infinite, the txn will never be expired.
|
||||
TxnKeepaliveInfinite = -1 * time.Second
|
||||
)
|
||||
|
||||
// NewTxnContextFromProto generates TxnContext from proto message.
|
||||
|
||||
@ -6,10 +6,6 @@ import (
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
// ControlChannel is the name of control channel which is used to identify the control channel from other vchannels.
|
||||
// It's just a hint, which is not the real virtual channel name of control channel.
|
||||
const ControlChannel string = "__cchan"
|
||||
|
||||
// AsImmutableTxnMessage converts an ImmutableMessage to ImmutableTxnMessage
|
||||
var AsImmutableTxnMessage = func(msg ImmutableMessage) ImmutableTxnMessage {
|
||||
underlying, ok := msg.(*immutableTxnMessageImpl)
|
||||
|
||||
@ -42,6 +42,10 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
const (
|
||||
ControlChannelSuffix = "vcchan" // is the suffix of the virtual control channel
|
||||
)
|
||||
|
||||
// CheckGrpcReady wait for context timeout, or wait 100ms then send nil to targetCh
|
||||
func CheckGrpcReady(ctx context.Context, targetCh chan error) {
|
||||
timer := time.NewTimer(100 * time.Millisecond)
|
||||
@ -291,6 +295,11 @@ func IsPhysicalChannel(channel string) bool {
|
||||
return !strings.Contains(channel[i+1:], "v")
|
||||
}
|
||||
|
||||
// IsControlChannel checks if the channel is a control channel
|
||||
func IsControlChannel(channel string) bool {
|
||||
return strings.HasSuffix(channel, ControlChannelSuffix)
|
||||
}
|
||||
|
||||
// ToPhysicalChannel get physical channel name from virtual channel name
|
||||
func ToPhysicalChannel(vchannel string) string {
|
||||
if IsPhysicalChannel(vchannel) {
|
||||
@ -303,6 +312,11 @@ func ToPhysicalChannel(vchannel string) string {
|
||||
return vchannel[:index]
|
||||
}
|
||||
|
||||
// GetControlChannel returns the control channel name of the pchannel.
|
||||
func GetControlChannel(pchannel string) string {
|
||||
return fmt.Sprintf("%s_%s", pchannel, ControlChannelSuffix)
|
||||
}
|
||||
|
||||
func GetVirtualChannel(pchannel string, collectionID int64, idx int) string {
|
||||
return fmt.Sprintf("%s_%dv%d", pchannel, collectionID, idx)
|
||||
}
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user