milvus/internal/proxy/replicate/replicate_stream_server.go
Zhen Ye 318db122b8
enhance: cherry pick patch of new DDL framework and CDC (#45025)
issue: #43897, #44123
pr: #44898
related pr: #44607 #44642 #44792 #44809 #44564 #44560 #44735 #44822
#44865 #44850 #44942 #44874 #44963 #44886 #44898

enhance: remove redundant channel manager from datacoord (#44532)

issue: #41611

- After enabling streaming arch, channel manager of data coord is a
redundant component.


fix: Fix CDC OOM due to high buffer size (#44607)

Fix CDC OOM by:
1. free msg buffer manually.
2. limit max msg buffer size.
3. reduce scanner msg hander buffer size.

issue: https://github.com/milvus-io/milvus/issues/44123

fix: remove wrong start timetick to avoid filtering DML whose timetick
is less than it. (#44691)

issue: #41611

- introduced by #44532

enhance: support remove cluster from replicate topology (#44642)

issue: #44558, #44123
- Update config(A->C) to A and C, config(B) to B on replicate topology
(A->B,A->C) can remove the B from replicate topology
- Fix some metric error of CDC

fix: check if qn is sqn with label and streamingnode list (#44792)

issue: #44014

- On standalone, the query node inside need to load segment and watch
channel, so the querynode is not a embeded querynode in streamingnode
without `LabelStreamingNodeEmbeddedQueryNode`. The channel dist manager
can not confirm a standalone node is a embededStreamingNode.

Bug is introduced by #44099

enhance: Make GetReplicateInfo API work at the pchannel level (#44809)

issue: https://github.com/milvus-io/milvus/issues/44123

enhance: Speed up CDC scheduling (#44564)

Make CDC watch etcd replicate pchannel meta instead of listing them
periodically.

issue: https://github.com/milvus-io/milvus/issues/44123


enhance: refactor update replicate config operation using
wal-broadcast-based DDL/DCL framework (#44560)

issue: #43897

- UpdateReplicateConfig operation will broadcast AlterReplicateConfig
message into all pchannels with cluster-exclusive-lock.
- Begin txn message will use commit message timetick now (to avoid
timetick rollback when CDC with txn message).
- If current cluster is secondary, the UpdateReplicateConfig will wait
until the replicate configuration is consistent with the config
replicated from primary.


enhance: support rbac with WAL-based DDL framework (#44735)

issue: #43897

- RBAC(Roles/Users/Privileges/Privilege Groups) is implemented by
WAL-based DDL framework now.
- Support following message type in wal `AlterUser`, `DropUser`,
`AlterRole`, `DropRole`, `AlterUserRole`, `DropUserRole`,
`AlterPrivilege`, `DropPrivilege`, `AlterPrivilegeGroup`,
`DropPrivilegeGroup`, `RestoreRBAC`.
- RBAC can be synced by new CDC now.
- Refactor some UT for RBAC.


enhance: support database with WAL-based DDL framework (#44822)

issue: #43897

- Database related DDL is implemented by WAL-based DDL framework now.
- Support following message type in wal CreateDatabase, AlterDatabase,
DropDatabase.
- Database DDL can be synced by new CDC now.
- Refactor some UT for Database DDL.

enhance: support alias with WAL-based DDL framework (#44865)

issue: #43897

- Alias related DDL is implemented by WAL-based DDL framework now.
- Support following message type in wal AlterAlias, DropAlias.
- Alias DDL can be synced by new CDC now.
- Refactor some UT for Alias DDL.

enhance: Disable import for replicating cluster (#44850)

1. Import in replicating cluster is not supported yet, so disable it for
now.
2. Remove GetReplicateConfiguration wal API

issue: https://github.com/milvus-io/milvus/issues/44123


fix: use short debug string to avoid newline in debug logs (#44925)

issue: #44924

fix: rerank before requery if reranker didn't use field data (#44942)

issue: #44918


enhance: support resource group with WAL-based DDL framework (#44874)

issue: #43897

- Resource group related DDL is implemented by WAL-based DDL framework
now.
- Support following message type in wal AlterResourceGroup,
DropResourceGroup.
- Resource group DDL can be synced by new CDC now.
- Refactor some UT for resource group DDL.


fix: Fix Fix replication txn data loss during chaos (#44963)

Only confirm CommitMsg for txn messages to prevent data loss.

issue: https://github.com/milvus-io/milvus/issues/44962,
https://github.com/milvus-io/milvus/issues/44123

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.


fix: Fix primary-secondary replication switch blocking (#44898)

1. Fix primary-secondary replication switchover blocking by delete
replicate pchannel meta using modRevision.
2. Stop channel replicator(scanner) when cluster role changes to prevent
continued message consumption and replication.
3. Close Milvus client to prevent goroutine leak.
4. Create Milvus client once for a channel replicator.
5. Simplify CDC controller and resources.

issue: https://github.com/milvus-io/milvus/issues/44123

---------

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
Signed-off-by: chyezh <chyezh@outlook.com>
Co-authored-by: yihao.dai <yihao.dai@zilliz.com>
2025-11-03 15:39:33 +08:00

166 lines
4.9 KiB
Go

package replicate
import (
"io"
"sync"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
)
const replicateRespChanLength = 128
func CreateReplicateServer(streamServer milvuspb.MilvusService_CreateReplicateStreamServer) (*ReplicateStreamServer, error) {
clusterID, err := contextutil.GetClusterID(streamServer.Context())
if err != nil {
return nil, err
}
return &ReplicateStreamServer{
clusterID: clusterID,
streamServer: streamServer,
replicateRespCh: make(chan *milvuspb.ReplicateResponse, replicateRespChanLength),
wg: sync.WaitGroup{},
}, nil
}
// ReplicateStreamServer is a ReplicateStreamServer of replicate messages.
type ReplicateStreamServer struct {
clusterID string
streamServer milvuspb.MilvusService_CreateReplicateStreamServer
replicateRespCh chan *milvuspb.ReplicateResponse
wg sync.WaitGroup
}
// Execute starts the replicate server.
func (p *ReplicateStreamServer) Execute() error {
// Start a recv arm to handle the control message from client.
go func() {
// recv loop will be blocked until the stream is closed.
_ = p.recvLoop()
}()
// Start a send loop on current main goroutine.
// the loop will be blocked until the stream is closed.
err := p.sendLoop()
return err
}
// sendLoop sends the message to client.
func (p *ReplicateStreamServer) sendLoop() (err error) {
defer func() {
if err != nil {
log.Warn("send arm of stream closed by unexpected error", zap.Error(err))
return
}
log.Info("send arm of stream closed")
}()
for {
select {
case resp, ok := <-p.replicateRespCh:
if !ok {
return nil
}
if err := p.streamServer.Send(resp); err != nil {
return err
}
case <-p.streamServer.Context().Done():
return errors.Wrap(p.streamServer.Context().Err(), "cancel send loop by stream server")
}
}
}
// recvLoop receives the message from client.
func (p *ReplicateStreamServer) recvLoop() (err error) {
defer func() {
p.wg.Wait()
close(p.replicateRespCh)
if err != nil {
log.Warn("recv arm of stream closed by unexpected error", zap.Error(err))
return
}
log.Info("recv arm of stream closed")
}()
for {
req, err := p.streamServer.Recv()
if err == io.EOF {
return nil
}
if err != nil {
return err
}
switch req := req.Request.(type) {
case *milvuspb.ReplicateRequest_ReplicateMessage:
err := p.handleReplicateMessage(req)
if err != nil {
return err
}
default:
log.Warn("unknown request type", zap.Any("request", req))
}
}
}
// handleReplicateMessage handles the replicate message request.
func (p *ReplicateStreamServer) handleReplicateMessage(req *milvuspb.ReplicateRequest_ReplicateMessage) error {
p.wg.Add(1)
defer p.wg.Done()
reqMsg := req.ReplicateMessage.GetMessage()
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()),
zap.Uint64("sourceTimeTick", sourceTs),
log.FieldMessage(msg),
)
// Append message to wal.
_, err = streaming.WAL().Replicate().Append(p.streamServer.Context(), msg)
if err == nil {
p.sendReplicateResult(sourceTs, msg)
return nil
}
if status.AsStreamingError(err).IsIgnoredOperation() {
log.Info("append replicate message to wal ignored", log.FieldMessage(msg), zap.Error(err))
return nil
}
// unexpected error, will close the stream and wait for client to reconnect.
log.Warn("append replicate message to wal failed", log.FieldMessage(msg), zap.Error(err))
return err
}
// sendReplicateResult sends the replicate result to client.
func (p *ReplicateStreamServer) sendReplicateResult(sourceTimeTick uint64, msg message.ReplicateMutableMessage) {
if msg.TxnContext() != nil && msg.MessageType() != message.MessageTypeCommitTxn {
// Only confirm the commit message of a transaction.
return
}
resp := &milvuspb.ReplicateResponse{
Response: &milvuspb.ReplicateResponse_ReplicateConfirmedMessageInfo{
ReplicateConfirmedMessageInfo: &milvuspb.ReplicateConfirmedMessageInfo{
ConfirmedTimeTick: sourceTimeTick,
},
},
}
// If server context is canceled, it means the stream has been closed.
// all pending response message should be dropped, client side will handle it.
select {
case p.replicateRespCh <- resp:
log.Debug("send replicate message response to client", zap.Uint64("confirmedTimeTick", sourceTimeTick))
case <-p.streamServer.Context().Done():
log.Warn("stream closed before replicate message response sent", zap.Uint64("confirmedTimeTick", sourceTimeTick))
return
}
}