enhance: broadcast with event-based notification (#39550)

issue: #38399
pr: #39522

- broadcast message can carry multi resource key now.
- implement event-based notification for broadcast messages
- broadcast message use broadcast id as a unique identifier in message
- broadcasted message on vchannels keep the broadcasted vchannel now.
- broadcasted message and broadcast message have a common broadcast
header now.

---------

Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
Zhen Ye 2025-02-07 11:50:50 +08:00 committed by GitHub
parent ddc5b299ad
commit 858dc10ef9
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
73 changed files with 6341 additions and 1296 deletions

View File

@ -15,10 +15,14 @@ packages:
github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster: github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster:
interfaces: interfaces:
AppendOperator: AppendOperator:
Watcher:
github.com/milvus-io/milvus/internal/streamingcoord/client: github.com/milvus-io/milvus/internal/streamingcoord/client:
interfaces: interfaces:
Client: Client:
BroadcastService: BroadcastService:
github.com/milvus-io/milvus/internal/streamingcoord/client/broadcast:
interfaces:
Watcher:
github.com/milvus-io/milvus/internal/streamingnode/client/manager: github.com/milvus-io/milvus/internal/streamingnode/client/manager:
interfaces: interfaces:
ManagerClient: ManagerClient:

View File

@ -58,9 +58,6 @@ func EnableLocalClientRole(cfg *LocalClientRoleConfig) {
// RegisterQueryCoordServer register query coord server // RegisterQueryCoordServer register query coord server
func RegisterQueryCoordServer(server querypb.QueryCoordServer) { func RegisterQueryCoordServer(server querypb.QueryCoordServer) {
if !enableLocal.EnableQueryCoord {
return
}
newLocalClient := grpcclient.NewLocalGRPCClient(&querypb.QueryCoord_ServiceDesc, server, querypb.NewQueryCoordClient) newLocalClient := grpcclient.NewLocalGRPCClient(&querypb.QueryCoord_ServiceDesc, server, querypb.NewQueryCoordClient)
glocalClient.queryCoordClient.Set(&nopCloseQueryCoordClient{newLocalClient}) glocalClient.queryCoordClient.Set(&nopCloseQueryCoordClient{newLocalClient})
log.Ctx(context.TODO()).Info("register query coord server", zap.Any("enableLocalClient", enableLocal)) log.Ctx(context.TODO()).Info("register query coord server", zap.Any("enableLocalClient", enableLocal))
@ -68,9 +65,6 @@ func RegisterQueryCoordServer(server querypb.QueryCoordServer) {
// RegsterDataCoordServer register data coord server // RegsterDataCoordServer register data coord server
func RegisterDataCoordServer(server datapb.DataCoordServer) { func RegisterDataCoordServer(server datapb.DataCoordServer) {
if !enableLocal.EnableDataCoord {
return
}
newLocalClient := grpcclient.NewLocalGRPCClient(&datapb.DataCoord_ServiceDesc, server, datapb.NewDataCoordClient) newLocalClient := grpcclient.NewLocalGRPCClient(&datapb.DataCoord_ServiceDesc, server, datapb.NewDataCoordClient)
glocalClient.dataCoordClient.Set(&nopCloseDataCoordClient{newLocalClient}) glocalClient.dataCoordClient.Set(&nopCloseDataCoordClient{newLocalClient})
log.Ctx(context.TODO()).Info("register data coord server", zap.Any("enableLocalClient", enableLocal)) log.Ctx(context.TODO()).Info("register data coord server", zap.Any("enableLocalClient", enableLocal))
@ -78,9 +72,6 @@ func RegisterDataCoordServer(server datapb.DataCoordServer) {
// RegisterRootCoordServer register root coord server // RegisterRootCoordServer register root coord server
func RegisterRootCoordServer(server rootcoordpb.RootCoordServer) { func RegisterRootCoordServer(server rootcoordpb.RootCoordServer) {
if !enableLocal.EnableRootCoord {
return
}
newLocalClient := grpcclient.NewLocalGRPCClient(&rootcoordpb.RootCoord_ServiceDesc, server, rootcoordpb.NewRootCoordClient) newLocalClient := grpcclient.NewLocalGRPCClient(&rootcoordpb.RootCoord_ServiceDesc, server, rootcoordpb.NewRootCoordClient)
glocalClient.rootCoordClient.Set(&nopCloseRootCoordClient{newLocalClient}) glocalClient.rootCoordClient.Set(&nopCloseRootCoordClient{newLocalClient})
log.Ctx(context.TODO()).Info("register root coord server", zap.Any("enableLocalClient", enableLocal)) log.Ctx(context.TODO()).Info("register root coord server", zap.Any("enableLocalClient", enableLocal))

View File

@ -34,9 +34,10 @@ func TestRegistry(t *testing.T) {
RegisterRootCoordServer(&rootcoordpb.UnimplementedRootCoordServer{}) RegisterRootCoordServer(&rootcoordpb.UnimplementedRootCoordServer{})
RegisterDataCoordServer(&datapb.UnimplementedDataCoordServer{}) RegisterDataCoordServer(&datapb.UnimplementedDataCoordServer{})
RegisterQueryCoordServer(&querypb.UnimplementedQueryCoordServer{}) RegisterQueryCoordServer(&querypb.UnimplementedQueryCoordServer{})
assert.False(t, glocalClient.dataCoordClient.Ready()) assert.True(t, glocalClient.dataCoordClient.Ready())
assert.False(t, glocalClient.queryCoordClient.Ready()) assert.True(t, glocalClient.queryCoordClient.Ready())
assert.False(t, glocalClient.rootCoordClient.Ready()) assert.True(t, glocalClient.rootCoordClient.Ready())
ResetRegistration()
enableLocal = &LocalClientRoleConfig{} enableLocal = &LocalClientRoleConfig{}

View File

@ -0,0 +1,32 @@
//go:build test
// +build test
package coordclient
import (
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/util/syncutil"
)
// ResetRegistration resets the global local client to initial state.
// This function is only used in test.
func ResetRegistration() {
glocalClient = &localClient{
queryCoordClient: syncutil.NewFuture[types.QueryCoordClient](),
dataCoordClient: syncutil.NewFuture[types.DataCoordClient](),
rootCoordClient: syncutil.NewFuture[types.RootCoordClient](),
}
}
// ResetQueryCoordRegistration resets the query coord client to initial state.
func ResetQueryCoordRegistration() {
glocalClient.queryCoordClient = syncutil.NewFuture[types.QueryCoordClient]()
}
func ResetRootCoordRegistration() {
glocalClient.rootCoordClient = syncutil.NewFuture[types.RootCoordClient]()
}
func ResetDataCoordRegistration() {
glocalClient.dataCoordClient = syncutil.NewFuture[types.DataCoordClient]()
}

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
"github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/proto/indexpb"
@ -38,6 +39,7 @@ import (
func Test_NewServer(t *testing.T) { func Test_NewServer(t *testing.T) {
paramtable.Init() paramtable.Init()
coordclient.ResetRegistration()
ctx := context.Background() ctx := context.Background()
mockDataCoord := mocks.NewMockDataCoord(t) mockDataCoord := mocks.NewMockDataCoord(t)
@ -335,6 +337,7 @@ func Test_Run(t *testing.T) {
t.Run("test run success", func(t *testing.T) { t.Run("test run success", func(t *testing.T) {
parameters := []string{"tikv", "etcd"} parameters := []string{"tikv", "etcd"}
for _, v := range parameters { for _, v := range parameters {
coordclient.ResetRegistration()
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background() ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
@ -370,6 +373,7 @@ func Test_Run(t *testing.T) {
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, "etcd") paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, "etcd")
t.Run("test init error", func(t *testing.T) { t.Run("test init error", func(t *testing.T) {
coordclient.ResetRegistration()
ctx := context.Background() ctx := context.Background()
server, err := NewServer(ctx, nil) server, err := NewServer(ctx, nil)
assert.NotNil(t, server) assert.NotNil(t, server)
@ -390,6 +394,7 @@ func Test_Run(t *testing.T) {
}) })
t.Run("test register error", func(t *testing.T) { t.Run("test register error", func(t *testing.T) {
coordclient.ResetRegistration()
ctx := context.Background() ctx := context.Background()
server, err := NewServer(ctx, nil) server, err := NewServer(ctx, nil)
assert.NoError(t, err) assert.NoError(t, err)
@ -411,6 +416,7 @@ func Test_Run(t *testing.T) {
}) })
t.Run("test start error", func(t *testing.T) { t.Run("test start error", func(t *testing.T) {
coordclient.ResetRegistration()
ctx := context.Background() ctx := context.Background()
server, err := NewServer(ctx, nil) server, err := NewServer(ctx, nil)
assert.NoError(t, err) assert.NoError(t, err)
@ -433,6 +439,7 @@ func Test_Run(t *testing.T) {
}) })
t.Run("test stop error", func(t *testing.T) { t.Run("test stop error", func(t *testing.T) {
coordclient.ResetRegistration()
ctx := context.Background() ctx := context.Background()
server, err := NewServer(ctx, nil) server, err := NewServer(ctx, nil)
assert.NoError(t, err) assert.NoError(t, err)

View File

@ -28,6 +28,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
"github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/proto/querypb"
@ -46,6 +47,7 @@ func TestMain(m *testing.M) {
func Test_NewServer(t *testing.T) { func Test_NewServer(t *testing.T) {
parameters := []string{"tikv", "etcd"} parameters := []string{"tikv", "etcd"}
for _, v := range parameters { for _, v := range parameters {
coordclient.ResetRegistration()
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background() ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
@ -358,6 +360,7 @@ func Test_NewServer(t *testing.T) {
func TestServer_Run1(t *testing.T) { func TestServer_Run1(t *testing.T) {
parameters := []string{"tikv", "etcd"} parameters := []string{"tikv", "etcd"}
for _, v := range parameters { for _, v := range parameters {
coordclient.ResetRegistration()
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
t.Skip() t.Skip()
ctx := context.Background() ctx := context.Background()

View File

@ -31,6 +31,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
"github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/rootcoord"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
@ -121,6 +122,7 @@ func TestRun(t *testing.T) {
paramtable.Init() paramtable.Init()
parameters := []string{"tikv", "etcd"} parameters := []string{"tikv", "etcd"}
for _, v := range parameters { for _, v := range parameters {
coordclient.ResetRegistration()
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background() ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
@ -228,6 +230,7 @@ func TestServerRun_DataCoordClientInitErr(t *testing.T) {
paramtable.Init() paramtable.Init()
parameters := []string{"tikv", "etcd"} parameters := []string{"tikv", "etcd"}
for _, v := range parameters { for _, v := range parameters {
coordclient.ResetRegistration()
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background() ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
@ -258,6 +261,7 @@ func TestServerRun_DataCoordClientStartErr(t *testing.T) {
paramtable.Init() paramtable.Init()
parameters := []string{"tikv", "etcd"} parameters := []string{"tikv", "etcd"}
for _, v := range parameters { for _, v := range parameters {
coordclient.ResetRegistration()
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background() ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
@ -288,6 +292,7 @@ func TestServerRun_QueryCoordClientInitErr(t *testing.T) {
paramtable.Init() paramtable.Init()
parameters := []string{"tikv", "etcd"} parameters := []string{"tikv", "etcd"}
for _, v := range parameters { for _, v := range parameters {
coordclient.ResetRegistration()
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background() ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
@ -318,6 +323,7 @@ func TestServer_QueryCoordClientStartErr(t *testing.T) {
paramtable.Init() paramtable.Init()
parameters := []string{"tikv", "etcd"} parameters := []string{"tikv", "etcd"}
for _, v := range parameters { for _, v := range parameters {
coordclient.ResetRegistration()
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background() ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {

View File

@ -17,12 +17,6 @@ func (w *walAccesserImpl) appendToWAL(ctx context.Context, msg message.MutableMe
return p.Produce(ctx, msg) return p.Produce(ctx, msg)
} }
func (w *walAccesserImpl) broadcastToWAL(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
// The broadcast operation will be sent to the coordinator.
// The coordinator will dispatch the message to all the vchannels with an eventual consistency guarantee.
return w.streamingCoordClient.Broadcast().Broadcast(ctx, msg)
}
// createOrGetProducer creates or get a producer. // createOrGetProducer creates or get a producer.
// vchannel in same pchannel can share the same producer. // vchannel in same pchannel can share the same producer.
func (w *walAccesserImpl) getProducer(pchannel string) *producer.ResumableProducer { func (w *walAccesserImpl) getProducer(pchannel string) *producer.ResumableProducer {

View File

@ -0,0 +1,52 @@
package streaming
import (
"context"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
var _ Broadcast = broadcast{}
type broadcast struct {
*walAccesserImpl
}
func (b broadcast) Append(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
assertValidBroadcastMessage(msg)
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
return nil, ErrWALAccesserClosed
}
defer b.lifetime.Done()
// The broadcast operation will be sent to the coordinator.
// The coordinator will dispatch the message to all the vchannels with an eventual consistency guarantee.
return b.streamingCoordClient.Broadcast().Broadcast(ctx, msg)
}
func (b broadcast) Ack(ctx context.Context, req types.BroadcastAckRequest) error {
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
return ErrWALAccesserClosed
}
defer b.lifetime.Done()
return b.streamingCoordClient.Broadcast().Ack(ctx, req)
}
func (b broadcast) BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error {
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
return ErrWALAccesserClosed
}
defer b.lifetime.Done()
return b.streamingCoordClient.Broadcast().BlockUntilEvent(ctx, message.NewResourceKeyAckOneBroadcastEvent(rk))
}
func (b broadcast) BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error {
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
return ErrWALAccesserClosed
}
defer b.lifetime.Done()
return b.streamingCoordClient.Broadcast().BlockUntilEvent(ctx, message.NewResourceKeyAckAllBroadcastEvent(rk))
}

View File

@ -81,16 +81,20 @@ type Scanner interface {
// WALAccesser is the interfaces to interact with the milvus write ahead log. // WALAccesser is the interfaces to interact with the milvus write ahead log.
type WALAccesser interface { type WALAccesser interface {
// Txn returns a transaction for writing records to the log. // Txn returns a transaction for writing records to one vchannel.
// It promises the atomicity written of the messages.
// Once the txn is returned, the Commit or Rollback operation must be called once, otherwise resource leak on wal. // Once the txn is returned, the Commit or Rollback operation must be called once, otherwise resource leak on wal.
Txn(ctx context.Context, opts TxnOption) (Txn, error) Txn(ctx context.Context, opts TxnOption) (Txn, error)
// RawAppend writes a records to the log. // RawAppend writes a records to the log.
RawAppend(ctx context.Context, msgs message.MutableMessage, opts ...AppendOption) (*types.AppendResult, error) RawAppend(ctx context.Context, msgs message.MutableMessage, opts ...AppendOption) (*types.AppendResult, error)
// BroadcastAppend sends a broadcast message to all target vchannels. // Broadcast returns a broadcast service of wal.
// BroadcastAppend guarantees the atomicity written of the messages and eventual consistency. // Broadcast support cross-vchannel message broadcast.
BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) // It promises the atomicity written of the messages and eventual consistency.
// And the broadcasted message must be acked cat consuming-side, otherwise resource leak on broadcast.
// Broadcast also support the resource-key to achieve a resource-exclusive acquirsion.
Broadcast() Broadcast
// Read returns a scanner for reading records from the wal. // Read returns a scanner for reading records from the wal.
Read(ctx context.Context, opts ReadOption) Scanner Read(ctx context.Context, opts ReadOption) Scanner
@ -100,15 +104,33 @@ type WALAccesser interface {
// If the messages is belong to one vchannel, it will be sent as a transaction. // If the messages is belong to one vchannel, it will be sent as a transaction.
// Otherwise, it will be sent as individual messages. // Otherwise, it will be sent as individual messages.
// !!! This function do not promise the atomicity and deliver order of the messages appending. // !!! This function do not promise the atomicity and deliver order of the messages appending.
// TODO: Remove after we support cross-wal txn.
AppendMessages(ctx context.Context, msgs ...message.MutableMessage) AppendResponses AppendMessages(ctx context.Context, msgs ...message.MutableMessage) AppendResponses
// AppendMessagesWithOption appends messages to the wal with the given option. // AppendMessagesWithOption appends messages to the wal with the given option.
// Same with AppendMessages, but with the given option. // Same with AppendMessages, but with the given option.
// TODO: Remove after we support cross-wal txn.
AppendMessagesWithOption(ctx context.Context, opts AppendOption, msgs ...message.MutableMessage) AppendResponses AppendMessagesWithOption(ctx context.Context, opts AppendOption, msgs ...message.MutableMessage) AppendResponses
} }
// Broadcast is the interface for writing broadcast message into the wal.
type Broadcast interface {
// Append of Broadcast sends a broadcast message to all target vchannels.
// Guarantees the atomicity written of the messages and eventual consistency.
// The resource-key bound at the message will be held until the message is acked at consumer.
// Once the resource-key is held, the append operation will be rejected.
// Use resource-key to make a sequential operation at same resource-key.
Append(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)
// Ack acknowledges a broadcast message at the specified vchannel.
// It must be called after the message is comsumed by the unique-consumer.
Ack(ctx context.Context, req types.BroadcastAckRequest) error
// BlockUntilResourceKeyAckOnce blocks until the resource-key-bind broadcast message is acked at any one vchannel.
BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error
// BlockUntilResourceKeyAckOnce blocks until the resource-key-bind broadcast message is acked at all vchannel.
BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error
}
// Txn is the interface for writing transaction into the wal. // Txn is the interface for writing transaction into the wal.
type Txn interface { type Txn interface {
// Append writes a record to the log. // Append writes a record to the log.

View File

@ -6,11 +6,16 @@ import (
"testing" "testing"
"time" "time"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/util/streamingutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options" "github.com/milvus-io/milvus/pkg/streaming/util/options"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
_ "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/pulsar" _ "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/pulsar"
"github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/paramtable"
) )
@ -20,13 +25,101 @@ var vChannels = []string{
"by-dev-rootcoord-dml_5", "by-dev-rootcoord-dml_5",
} }
var collectionName = "test"
func TestMain(m *testing.M) { func TestMain(m *testing.M) {
streamingutil.SetStreamingServiceEnabled()
paramtable.Init() paramtable.Init()
m.Run() m.Run()
} }
func TestStreamingBroadcast(t *testing.T) {
t.Skip("cat not running without streaming service at background")
streamingutil.SetStreamingServiceEnabled()
streaming.Init()
defer streaming.Release()
err := streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(context.Background(), message.NewCollectionNameResourceKey(collectionName))
assert.NoError(t, err)
msg, _ := message.NewCreateCollectionMessageBuilderV1().
WithHeader(&message.CreateCollectionMessageHeader{
CollectionId: 1,
PartitionIds: []int64{1, 2, 3},
}).
WithBody(&msgpb.CreateCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_CreateCollection,
Timestamp: 1,
},
CollectionID: 1,
CollectionName: collectionName,
}).
WithBroadcast(vChannels, message.NewCollectionNameResourceKey(collectionName)).
BuildBroadcast()
resp, err := streaming.WAL().Broadcast().Append(context.Background(), msg)
assert.NoError(t, err)
assert.NotNil(t, resp)
t.Logf("CreateCollection: %+v\t%+v\n", resp, err)
// repeated broadcast with same resource key should be rejected
resp2, err := streaming.WAL().Broadcast().Append(context.Background(), msg)
assert.Error(t, err)
assert.True(t, status.AsStreamingError(err).IsResourceAcquired())
assert.Nil(t, resp2)
// resource key should be block until ack.
ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100)
defer cancel()
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName))
assert.ErrorIs(t, err, context.DeadlineExceeded)
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
defer cancel()
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName))
assert.ErrorIs(t, err, context.DeadlineExceeded)
err = streaming.WAL().Broadcast().Ack(context.Background(), types.BroadcastAckRequest{
BroadcastID: resp.BroadcastID,
VChannel: vChannels[0],
})
assert.NoError(t, err)
// all should be blocked
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
defer cancel()
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName))
assert.ErrorIs(t, err, context.DeadlineExceeded)
// once should be returned
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
defer cancel()
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName))
assert.NoError(t, err)
err = streaming.WAL().Broadcast().Ack(context.Background(), types.BroadcastAckRequest{
BroadcastID: resp.BroadcastID,
VChannel: vChannels[1],
})
assert.NoError(t, err)
// all should be blocked
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
defer cancel()
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName))
assert.NoError(t, err)
// once should be returned
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
defer cancel()
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName))
assert.NoError(t, err)
}
func TestStreamingProduce(t *testing.T) { func TestStreamingProduce(t *testing.T) {
t.Skip() t.Skip("cat not running without streaming service at background")
streamingutil.SetStreamingServiceEnabled()
streaming.Init() streaming.Init()
defer streaming.Release() defer streaming.Release()
msg, _ := message.NewCreateCollectionMessageBuilderV1(). msg, _ := message.NewCreateCollectionMessageBuilderV1().
@ -40,11 +133,12 @@ func TestStreamingProduce(t *testing.T) {
Timestamp: 1, Timestamp: 1,
}, },
CollectionID: 1, CollectionID: 1,
CollectionName: collectionName,
}). }).
WithBroadcast(vChannels). WithBroadcast(vChannels).
BuildBroadcast() BuildBroadcast()
resp, err := streaming.WAL().BroadcastAppend(context.Background(), msg) resp, err := streaming.WAL().Broadcast().Append(context.Background(), msg)
t.Logf("CreateCollection: %+v\t%+v\n", resp, err) t.Logf("CreateCollection: %+v\t%+v\n", resp, err)
for i := 0; i < 500; i++ { for i := 0; i < 500; i++ {
@ -101,12 +195,12 @@ func TestStreamingProduce(t *testing.T) {
}). }).
WithBroadcast(vChannels). WithBroadcast(vChannels).
BuildBroadcast() BuildBroadcast()
resp, err = streaming.WAL().BroadcastAppend(context.Background(), msg) resp, err = streaming.WAL().Broadcast().Append(context.Background(), msg)
t.Logf("DropCollection: %+v\t%+v\n", resp, err) t.Logf("DropCollection: %+v\t%+v\n", resp, err)
} }
func TestStreamingConsume(t *testing.T) { func TestStreamingConsume(t *testing.T) {
t.Skip() t.Skip("cat not running without streaming service at background")
streaming.Init() streaming.Init()
defer streaming.Release() defer streaming.Release()
ch := make(message.ChanMessageHandler, 10) ch := make(message.ChanMessageHandler, 10)

View File

@ -71,16 +71,6 @@ func (w *walAccesserImpl) RawAppend(ctx context.Context, msg message.MutableMess
return w.appendToWAL(ctx, msg) return w.appendToWAL(ctx, msg)
} }
func (w *walAccesserImpl) BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
assertValidBroadcastMessage(msg)
if !w.lifetime.Add(typeutil.LifetimeStateWorking) {
return nil, ErrWALAccesserClosed
}
defer w.lifetime.Done()
return w.broadcastToWAL(ctx, msg)
}
// Read returns a scanner for reading records from the wal. // Read returns a scanner for reading records from the wal.
func (w *walAccesserImpl) Read(_ context.Context, opts ReadOption) Scanner { func (w *walAccesserImpl) Read(_ context.Context, opts ReadOption) Scanner {
if !w.lifetime.Add(typeutil.LifetimeStateWorking) { if !w.lifetime.Add(typeutil.LifetimeStateWorking) {
@ -104,6 +94,11 @@ func (w *walAccesserImpl) Read(_ context.Context, opts ReadOption) Scanner {
return rc return rc
} }
// Broadcast returns a broadcast for broadcasting records to the wal.
func (w *walAccesserImpl) Broadcast() Broadcast {
return broadcast{w}
}
func (w *walAccesserImpl) Txn(ctx context.Context, opts TxnOption) (Txn, error) { func (w *walAccesserImpl) Txn(ctx context.Context, opts TxnOption) (Txn, error) {
if !w.lifetime.Add(typeutil.LifetimeStateWorking) { if !w.lifetime.Add(typeutil.LifetimeStateWorking) {
return nil, ErrWALAccesserClosed return nil, ErrWALAccesserClosed

View File

@ -33,6 +33,7 @@ func TestWAL(t *testing.T) {
broadcastServce := mock_client.NewMockBroadcastService(t) broadcastServce := mock_client.NewMockBroadcastService(t)
broadcastServce.EXPECT().Broadcast(mock.Anything, mock.Anything).RunAndReturn( broadcastServce.EXPECT().Broadcast(mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, bmm message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { func(ctx context.Context, bmm message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
bmm = bmm.WithBroadcastID(1)
result := make(map[string]*types.AppendResult) result := make(map[string]*types.AppendResult)
for idx, msg := range bmm.SplitIntoMutableMessage() { for idx, msg := range bmm.SplitIntoMutableMessage() {
result[msg.VChannel()] = &types.AppendResult{ result[msg.VChannel()] = &types.AppendResult{
@ -44,6 +45,8 @@ func TestWAL(t *testing.T) {
AppendResults: result, AppendResults: result,
}, nil }, nil
}) })
broadcastServce.EXPECT().Ack(mock.Anything, mock.Anything).Return(nil)
broadcastServce.EXPECT().BlockUntilEvent(mock.Anything, mock.Anything).Return(nil)
coordClient.EXPECT().Broadcast().Return(broadcastServce) coordClient.EXPECT().Broadcast().Return(broadcastServce)
handler := mock_handler.NewMockHandlerClient(t) handler := mock_handler.NewMockHandlerClient(t)
handler.EXPECT().Close().Return() handler.EXPECT().Close().Return()
@ -129,17 +132,36 @@ func TestWAL(t *testing.T) {
) )
assert.NoError(t, resp.UnwrapFirstError()) assert.NoError(t, resp.UnwrapFirstError())
r, err := w.BroadcastAppend(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3})) r, err := w.Broadcast().Append(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3}))
assert.NoError(t, err) assert.NoError(t, err)
assert.Len(t, r.AppendResults, 3) assert.Len(t, r.AppendResults, 3)
err = w.Broadcast().Ack(ctx, types.BroadcastAckRequest{BroadcastID: 1, VChannel: vChannel1})
assert.NoError(t, err)
err = w.Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey("r1"))
assert.NoError(t, err)
err = w.Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey("r2"))
assert.NoError(t, err)
w.Close() w.Close()
resp = w.AppendMessages(ctx, newInsertMessage(vChannel1)) resp = w.AppendMessages(ctx, newInsertMessage(vChannel1))
assert.Error(t, resp.UnwrapFirstError()) assert.Error(t, resp.UnwrapFirstError())
r, err = w.BroadcastAppend(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3}))
r, err = w.Broadcast().Append(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3}))
assert.Error(t, err) assert.Error(t, err)
assert.Nil(t, r) assert.Nil(t, r)
err = w.Broadcast().Ack(ctx, types.BroadcastAckRequest{BroadcastID: 1, VChannel: vChannel1})
assert.Error(t, err)
err = w.Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey("r1"))
assert.Error(t, err)
err = w.Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey("r2"))
assert.Error(t, err)
} }
func newInsertMessage(vChannel string) message.MutableMessage { func newInsertMessage(vChannel string) message.MutableMessage {

View File

@ -218,7 +218,7 @@ type StreamingCoordCataLog interface {
// SaveBroadcastTask save the broadcast task to metastore. // SaveBroadcastTask save the broadcast task to metastore.
// Make the task recoverable after restart. // Make the task recoverable after restart.
// When broadcast task is done, it will be removed from metastore. // When broadcast task is done, it will be removed from metastore.
SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error SaveBroadcastTask(ctx context.Context, broadcastID uint64, task *streamingpb.BroadcastTask) error
} }
// StreamingNodeCataLog is the interface for streamingnode catalog // StreamingNodeCataLog is the interface for streamingnode catalog

View File

@ -86,8 +86,8 @@ func (c *catalog) ListBroadcastTask(ctx context.Context) ([]*streamingpb.Broadca
return infos, nil return infos, nil
} }
func (c *catalog) SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error { func (c *catalog) SaveBroadcastTask(ctx context.Context, broadcastID uint64, task *streamingpb.BroadcastTask) error {
key := buildBroadcastTaskPath(task.TaskId) key := buildBroadcastTaskPath(broadcastID)
if task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE { if task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE {
return c.metaKV.Remove(ctx, key) return c.metaKV.Remove(ctx, key)
} }
@ -104,6 +104,6 @@ func buildPChannelInfoPath(name string) string {
} }
// buildBroadcastTaskPath builds the path for broadcast task. // buildBroadcastTaskPath builds the path for broadcast task.
func buildBroadcastTaskPath(id int64) string { func buildBroadcastTaskPath(id uint64) string {
return BroadcastTaskPrefix + strconv.FormatInt(id, 10) return BroadcastTaskPrefix + strconv.FormatUint(id, 10)
} }

View File

@ -66,13 +66,11 @@ func TestCatalog(t *testing.T) {
assert.Len(t, metas, 2) assert.Len(t, metas, 2)
// BroadcastTask test // BroadcastTask test
err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ err = catalog.SaveBroadcastTask(context.Background(), 1, &streamingpb.BroadcastTask{
TaskId: 1,
State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING,
}) })
assert.NoError(t, err) assert.NoError(t, err)
err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ err = catalog.SaveBroadcastTask(context.Background(), 2, &streamingpb.BroadcastTask{
TaskId: 2,
State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING,
}) })
assert.NoError(t, err) assert.NoError(t, err)
@ -84,8 +82,7 @@ func TestCatalog(t *testing.T) {
assert.Equal(t, streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, task.State) assert.Equal(t, streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, task.State)
} }
err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ err = catalog.SaveBroadcastTask(context.Background(), 1, &streamingpb.BroadcastTask{
TaskId: 1,
State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE, State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE,
}) })
assert.NoError(t, err) assert.NoError(t, err)
@ -116,6 +113,6 @@ func TestCatalog(t *testing.T) {
Node: &streamingpb.StreamingNodeInfo{ServerId: 1}, Node: &streamingpb.StreamingNodeInfo{ServerId: 1},
}}) }})
assert.Error(t, err) assert.Error(t, err)
err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{}) err = catalog.SaveBroadcastTask(context.Background(), 1, &streamingpb.BroadcastTask{})
assert.Error(t, err) assert.Error(t, err)
} }

View File

@ -27,7 +27,7 @@ func (_m *MockWALAccesser) EXPECT() *MockWALAccesser_Expecter {
} }
// AppendMessages provides a mock function with given fields: ctx, msgs // AppendMessages provides a mock function with given fields: ctx, msgs
func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.MutableMessage) streaming.AppendResponses { func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.MutableMessage) types.AppendResponses {
_va := make([]interface{}, len(msgs)) _va := make([]interface{}, len(msgs))
for _i := range msgs { for _i := range msgs {
_va[_i] = msgs[_i] _va[_i] = msgs[_i]
@ -41,11 +41,11 @@ func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.M
panic("no return value specified for AppendMessages") panic("no return value specified for AppendMessages")
} }
var r0 streaming.AppendResponses var r0 types.AppendResponses
if rf, ok := ret.Get(0).(func(context.Context, ...message.MutableMessage) streaming.AppendResponses); ok { if rf, ok := ret.Get(0).(func(context.Context, ...message.MutableMessage) types.AppendResponses); ok {
r0 = rf(ctx, msgs...) r0 = rf(ctx, msgs...)
} else { } else {
r0 = ret.Get(0).(streaming.AppendResponses) r0 = ret.Get(0).(types.AppendResponses)
} }
return r0 return r0
@ -77,18 +77,18 @@ func (_c *MockWALAccesser_AppendMessages_Call) Run(run func(ctx context.Context,
return _c return _c
} }
func (_c *MockWALAccesser_AppendMessages_Call) Return(_a0 streaming.AppendResponses) *MockWALAccesser_AppendMessages_Call { func (_c *MockWALAccesser_AppendMessages_Call) Return(_a0 types.AppendResponses) *MockWALAccesser_AppendMessages_Call {
_c.Call.Return(_a0) _c.Call.Return(_a0)
return _c return _c
} }
func (_c *MockWALAccesser_AppendMessages_Call) RunAndReturn(run func(context.Context, ...message.MutableMessage) streaming.AppendResponses) *MockWALAccesser_AppendMessages_Call { func (_c *MockWALAccesser_AppendMessages_Call) RunAndReturn(run func(context.Context, ...message.MutableMessage) types.AppendResponses) *MockWALAccesser_AppendMessages_Call {
_c.Call.Return(run) _c.Call.Return(run)
return _c return _c
} }
// AppendMessagesWithOption provides a mock function with given fields: ctx, opts, msgs // AppendMessagesWithOption provides a mock function with given fields: ctx, opts, msgs
func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts streaming.AppendOption, msgs ...message.MutableMessage) streaming.AppendResponses { func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts streaming.AppendOption, msgs ...message.MutableMessage) types.AppendResponses {
_va := make([]interface{}, len(msgs)) _va := make([]interface{}, len(msgs))
for _i := range msgs { for _i := range msgs {
_va[_i] = msgs[_i] _va[_i] = msgs[_i]
@ -102,11 +102,11 @@ func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts st
panic("no return value specified for AppendMessagesWithOption") panic("no return value specified for AppendMessagesWithOption")
} }
var r0 streaming.AppendResponses var r0 types.AppendResponses
if rf, ok := ret.Get(0).(func(context.Context, streaming.AppendOption, ...message.MutableMessage) streaming.AppendResponses); ok { if rf, ok := ret.Get(0).(func(context.Context, streaming.AppendOption, ...message.MutableMessage) types.AppendResponses); ok {
r0 = rf(ctx, opts, msgs...) r0 = rf(ctx, opts, msgs...)
} else { } else {
r0 = ret.Get(0).(streaming.AppendResponses) r0 = ret.Get(0).(types.AppendResponses)
} }
return r0 return r0
@ -139,71 +139,59 @@ func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Run(run func(ctx contex
return _c return _c
} }
func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Return(_a0 streaming.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Return(_a0 types.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call {
_c.Call.Return(_a0) _c.Call.Return(_a0)
return _c return _c
} }
func (_c *MockWALAccesser_AppendMessagesWithOption_Call) RunAndReturn(run func(context.Context, streaming.AppendOption, ...message.MutableMessage) streaming.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { func (_c *MockWALAccesser_AppendMessagesWithOption_Call) RunAndReturn(run func(context.Context, streaming.AppendOption, ...message.MutableMessage) types.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call {
_c.Call.Return(run) _c.Call.Return(run)
return _c return _c
} }
// BroadcastAppend provides a mock function with given fields: ctx, msg // Broadcast provides a mock function with given fields:
func (_m *MockWALAccesser) BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { func (_m *MockWALAccesser) Broadcast() streaming.Broadcast {
ret := _m.Called(ctx, msg) ret := _m.Called()
if len(ret) == 0 { if len(ret) == 0 {
panic("no return value specified for BroadcastAppend") panic("no return value specified for Broadcast")
} }
var r0 *types.BroadcastAppendResult var r0 streaming.Broadcast
var r1 error if rf, ok := ret.Get(0).(func() streaming.Broadcast); ok {
if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)); ok { r0 = rf()
return rf(ctx, msg)
}
if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) *types.BroadcastAppendResult); ok {
r0 = rf(ctx, msg)
} else { } else {
if ret.Get(0) != nil { if ret.Get(0) != nil {
r0 = ret.Get(0).(*types.BroadcastAppendResult) r0 = ret.Get(0).(streaming.Broadcast)
} }
} }
if rf, ok := ret.Get(1).(func(context.Context, message.BroadcastMutableMessage) error); ok { return r0
r1 = rf(ctx, msg)
} else {
r1 = ret.Error(1)
}
return r0, r1
} }
// MockWALAccesser_BroadcastAppend_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastAppend' // MockWALAccesser_Broadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Broadcast'
type MockWALAccesser_BroadcastAppend_Call struct { type MockWALAccesser_Broadcast_Call struct {
*mock.Call *mock.Call
} }
// BroadcastAppend is a helper method to define mock.On call // Broadcast is a helper method to define mock.On call
// - ctx context.Context func (_e *MockWALAccesser_Expecter) Broadcast() *MockWALAccesser_Broadcast_Call {
// - msg message.BroadcastMutableMessage return &MockWALAccesser_Broadcast_Call{Call: _e.mock.On("Broadcast")}
func (_e *MockWALAccesser_Expecter) BroadcastAppend(ctx interface{}, msg interface{}) *MockWALAccesser_BroadcastAppend_Call {
return &MockWALAccesser_BroadcastAppend_Call{Call: _e.mock.On("BroadcastAppend", ctx, msg)}
} }
func (_c *MockWALAccesser_BroadcastAppend_Call) Run(run func(ctx context.Context, msg message.BroadcastMutableMessage)) *MockWALAccesser_BroadcastAppend_Call { func (_c *MockWALAccesser_Broadcast_Call) Run(run func()) *MockWALAccesser_Broadcast_Call {
_c.Call.Run(func(args mock.Arguments) { _c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(message.BroadcastMutableMessage)) run()
}) })
return _c return _c
} }
func (_c *MockWALAccesser_BroadcastAppend_Call) Return(_a0 *types.BroadcastAppendResult, _a1 error) *MockWALAccesser_BroadcastAppend_Call { func (_c *MockWALAccesser_Broadcast_Call) Return(_a0 streaming.Broadcast) *MockWALAccesser_Broadcast_Call {
_c.Call.Return(_a0, _a1) _c.Call.Return(_a0)
return _c return _c
} }
func (_c *MockWALAccesser_BroadcastAppend_Call) RunAndReturn(run func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)) *MockWALAccesser_BroadcastAppend_Call { func (_c *MockWALAccesser_Broadcast_Call) RunAndReturn(run func() streaming.Broadcast) *MockWALAccesser_Broadcast_Call {
_c.Call.Return(run) _c.Call.Return(run)
return _c return _c
} }

View File

@ -139,17 +139,17 @@ func (_c *MockStreamingCoordCataLog_ListPChannel_Call) RunAndReturn(run func(con
return _c return _c
} }
// SaveBroadcastTask provides a mock function with given fields: ctx, task // SaveBroadcastTask provides a mock function with given fields: ctx, broadcastID, task
func (_m *MockStreamingCoordCataLog) SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error { func (_m *MockStreamingCoordCataLog) SaveBroadcastTask(ctx context.Context, broadcastID uint64, task *streamingpb.BroadcastTask) error {
ret := _m.Called(ctx, task) ret := _m.Called(ctx, broadcastID, task)
if len(ret) == 0 { if len(ret) == 0 {
panic("no return value specified for SaveBroadcastTask") panic("no return value specified for SaveBroadcastTask")
} }
var r0 error var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastTask) error); ok { if rf, ok := ret.Get(0).(func(context.Context, uint64, *streamingpb.BroadcastTask) error); ok {
r0 = rf(ctx, task) r0 = rf(ctx, broadcastID, task)
} else { } else {
r0 = ret.Error(0) r0 = ret.Error(0)
} }
@ -164,14 +164,15 @@ type MockStreamingCoordCataLog_SaveBroadcastTask_Call struct {
// SaveBroadcastTask is a helper method to define mock.On call // SaveBroadcastTask is a helper method to define mock.On call
// - ctx context.Context // - ctx context.Context
// - broadcastID uint64
// - task *streamingpb.BroadcastTask // - task *streamingpb.BroadcastTask
func (_e *MockStreamingCoordCataLog_Expecter) SaveBroadcastTask(ctx interface{}, task interface{}) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { func (_e *MockStreamingCoordCataLog_Expecter) SaveBroadcastTask(ctx interface{}, broadcastID interface{}, task interface{}) *MockStreamingCoordCataLog_SaveBroadcastTask_Call {
return &MockStreamingCoordCataLog_SaveBroadcastTask_Call{Call: _e.mock.On("SaveBroadcastTask", ctx, task)} return &MockStreamingCoordCataLog_SaveBroadcastTask_Call{Call: _e.mock.On("SaveBroadcastTask", ctx, broadcastID, task)}
} }
func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) Run(run func(ctx context.Context, task *streamingpb.BroadcastTask)) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) Run(run func(ctx context.Context, broadcastID uint64, task *streamingpb.BroadcastTask)) *MockStreamingCoordCataLog_SaveBroadcastTask_Call {
_c.Call.Run(func(args mock.Arguments) { _c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*streamingpb.BroadcastTask)) run(args[0].(context.Context), args[1].(uint64), args[2].(*streamingpb.BroadcastTask))
}) })
return _c return _c
} }
@ -181,7 +182,7 @@ func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) Return(_a0 error) *M
return _c return _c
} }
func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) RunAndReturn(run func(context.Context, *streamingpb.BroadcastTask) error) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) RunAndReturn(run func(context.Context, uint64, *streamingpb.BroadcastTask) error) *MockStreamingCoordCataLog_SaveBroadcastTask_Call {
_c.Call.Return(run) _c.Call.Return(run)
return _c return _c
} }

View File

@ -0,0 +1,163 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_broadcast
import (
context "context"
messagespb "github.com/milvus-io/milvus/pkg/proto/messagespb"
mock "github.com/stretchr/testify/mock"
)
// MockWatcher is an autogenerated mock type for the Watcher type
type MockWatcher struct {
mock.Mock
}
type MockWatcher_Expecter struct {
mock *mock.Mock
}
func (_m *MockWatcher) EXPECT() *MockWatcher_Expecter {
return &MockWatcher_Expecter{mock: &_m.Mock}
}
// Close provides a mock function with given fields:
func (_m *MockWatcher) Close() {
_m.Called()
}
// MockWatcher_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
type MockWatcher_Close_Call struct {
*mock.Call
}
// Close is a helper method to define mock.On call
func (_e *MockWatcher_Expecter) Close() *MockWatcher_Close_Call {
return &MockWatcher_Close_Call{Call: _e.mock.On("Close")}
}
func (_c *MockWatcher_Close_Call) Run(run func()) *MockWatcher_Close_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWatcher_Close_Call) Return() *MockWatcher_Close_Call {
_c.Call.Return()
return _c
}
func (_c *MockWatcher_Close_Call) RunAndReturn(run func()) *MockWatcher_Close_Call {
_c.Call.Return(run)
return _c
}
// EventChan provides a mock function with given fields:
func (_m *MockWatcher) EventChan() <-chan *messagespb.BroadcastEvent {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for EventChan")
}
var r0 <-chan *messagespb.BroadcastEvent
if rf, ok := ret.Get(0).(func() <-chan *messagespb.BroadcastEvent); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(<-chan *messagespb.BroadcastEvent)
}
}
return r0
}
// MockWatcher_EventChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'EventChan'
type MockWatcher_EventChan_Call struct {
*mock.Call
}
// EventChan is a helper method to define mock.On call
func (_e *MockWatcher_Expecter) EventChan() *MockWatcher_EventChan_Call {
return &MockWatcher_EventChan_Call{Call: _e.mock.On("EventChan")}
}
func (_c *MockWatcher_EventChan_Call) Run(run func()) *MockWatcher_EventChan_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWatcher_EventChan_Call) Return(_a0 <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWatcher_EventChan_Call) RunAndReturn(run func() <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call {
_c.Call.Return(run)
return _c
}
// ObserveResourceKeyEvent provides a mock function with given fields: ctx, ev
func (_m *MockWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *messagespb.BroadcastEvent) error {
ret := _m.Called(ctx, ev)
if len(ret) == 0 {
panic("no return value specified for ObserveResourceKeyEvent")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *messagespb.BroadcastEvent) error); ok {
r0 = rf(ctx, ev)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockWatcher_ObserveResourceKeyEvent_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ObserveResourceKeyEvent'
type MockWatcher_ObserveResourceKeyEvent_Call struct {
*mock.Call
}
// ObserveResourceKeyEvent is a helper method to define mock.On call
// - ctx context.Context
// - ev *messagespb.BroadcastEvent
func (_e *MockWatcher_Expecter) ObserveResourceKeyEvent(ctx interface{}, ev interface{}) *MockWatcher_ObserveResourceKeyEvent_Call {
return &MockWatcher_ObserveResourceKeyEvent_Call{Call: _e.mock.On("ObserveResourceKeyEvent", ctx, ev)}
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Run(run func(ctx context.Context, ev *messagespb.BroadcastEvent)) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*messagespb.BroadcastEvent))
})
return _c
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Return(_a0 error) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) RunAndReturn(run func(context.Context, *messagespb.BroadcastEvent) error) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Return(run)
return _c
}
// NewMockWatcher creates a new instance of MockWatcher. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockWatcher(t interface {
mock.TestingT
Cleanup(func())
}) *MockWatcher {
mock := &MockWatcher{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -5,7 +5,9 @@ package mock_client
import ( import (
context "context" context "context"
messagespb "github.com/milvus-io/milvus/pkg/proto/messagespb"
message "github.com/milvus-io/milvus/pkg/streaming/util/message" message "github.com/milvus-io/milvus/pkg/streaming/util/message"
mock "github.com/stretchr/testify/mock" mock "github.com/stretchr/testify/mock"
types "github.com/milvus-io/milvus/pkg/streaming/util/types" types "github.com/milvus-io/milvus/pkg/streaming/util/types"
@ -24,6 +26,100 @@ func (_m *MockBroadcastService) EXPECT() *MockBroadcastService_Expecter {
return &MockBroadcastService_Expecter{mock: &_m.Mock} return &MockBroadcastService_Expecter{mock: &_m.Mock}
} }
// Ack provides a mock function with given fields: ctx, req
func (_m *MockBroadcastService) Ack(ctx context.Context, req types.BroadcastAckRequest) error {
ret := _m.Called(ctx, req)
if len(ret) == 0 {
panic("no return value specified for Ack")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, types.BroadcastAckRequest) error); ok {
r0 = rf(ctx, req)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockBroadcastService_Ack_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Ack'
type MockBroadcastService_Ack_Call struct {
*mock.Call
}
// Ack is a helper method to define mock.On call
// - ctx context.Context
// - req types.BroadcastAckRequest
func (_e *MockBroadcastService_Expecter) Ack(ctx interface{}, req interface{}) *MockBroadcastService_Ack_Call {
return &MockBroadcastService_Ack_Call{Call: _e.mock.On("Ack", ctx, req)}
}
func (_c *MockBroadcastService_Ack_Call) Run(run func(ctx context.Context, req types.BroadcastAckRequest)) *MockBroadcastService_Ack_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(types.BroadcastAckRequest))
})
return _c
}
func (_c *MockBroadcastService_Ack_Call) Return(_a0 error) *MockBroadcastService_Ack_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockBroadcastService_Ack_Call) RunAndReturn(run func(context.Context, types.BroadcastAckRequest) error) *MockBroadcastService_Ack_Call {
_c.Call.Return(run)
return _c
}
// BlockUntilEvent provides a mock function with given fields: ctx, ev
func (_m *MockBroadcastService) BlockUntilEvent(ctx context.Context, ev *messagespb.BroadcastEvent) error {
ret := _m.Called(ctx, ev)
if len(ret) == 0 {
panic("no return value specified for BlockUntilEvent")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *messagespb.BroadcastEvent) error); ok {
r0 = rf(ctx, ev)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockBroadcastService_BlockUntilEvent_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilEvent'
type MockBroadcastService_BlockUntilEvent_Call struct {
*mock.Call
}
// BlockUntilEvent is a helper method to define mock.On call
// - ctx context.Context
// - ev *messagespb.BroadcastEvent
func (_e *MockBroadcastService_Expecter) BlockUntilEvent(ctx interface{}, ev interface{}) *MockBroadcastService_BlockUntilEvent_Call {
return &MockBroadcastService_BlockUntilEvent_Call{Call: _e.mock.On("BlockUntilEvent", ctx, ev)}
}
func (_c *MockBroadcastService_BlockUntilEvent_Call) Run(run func(ctx context.Context, ev *messagespb.BroadcastEvent)) *MockBroadcastService_BlockUntilEvent_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*messagespb.BroadcastEvent))
})
return _c
}
func (_c *MockBroadcastService_BlockUntilEvent_Call) Return(_a0 error) *MockBroadcastService_BlockUntilEvent_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockBroadcastService_BlockUntilEvent_Call) RunAndReturn(run func(context.Context, *messagespb.BroadcastEvent) error) *MockBroadcastService_BlockUntilEvent_Call {
_c.Call.Return(run)
return _c
}
// Broadcast provides a mock function with given fields: ctx, msg // Broadcast provides a mock function with given fields: ctx, msg
func (_m *MockBroadcastService) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { func (_m *MockBroadcastService) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
ret := _m.Called(ctx, msg) ret := _m.Called(ctx, msg)
@ -83,6 +179,38 @@ func (_c *MockBroadcastService_Broadcast_Call) RunAndReturn(run func(context.Con
return _c return _c
} }
// Close provides a mock function with given fields:
func (_m *MockBroadcastService) Close() {
_m.Called()
}
// MockBroadcastService_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
type MockBroadcastService_Close_Call struct {
*mock.Call
}
// Close is a helper method to define mock.On call
func (_e *MockBroadcastService_Expecter) Close() *MockBroadcastService_Close_Call {
return &MockBroadcastService_Close_Call{Call: _e.mock.On("Close")}
}
func (_c *MockBroadcastService_Close_Call) Run(run func()) *MockBroadcastService_Close_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockBroadcastService_Close_Call) Return() *MockBroadcastService_Close_Call {
_c.Call.Return()
return _c
}
func (_c *MockBroadcastService_Close_Call) RunAndReturn(run func()) *MockBroadcastService_Close_Call {
_c.Call.Return(run)
return _c
}
// NewMockBroadcastService creates a new instance of MockBroadcastService. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // NewMockBroadcastService creates a new instance of MockBroadcastService. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value. // The first argument is typically a *testing.T value.
func NewMockBroadcastService(t interface { func NewMockBroadcastService(t interface {

View File

@ -0,0 +1,163 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_broadcaster
import (
context "context"
messagespb "github.com/milvus-io/milvus/pkg/proto/messagespb"
mock "github.com/stretchr/testify/mock"
)
// MockWatcher is an autogenerated mock type for the Watcher type
type MockWatcher struct {
mock.Mock
}
type MockWatcher_Expecter struct {
mock *mock.Mock
}
func (_m *MockWatcher) EXPECT() *MockWatcher_Expecter {
return &MockWatcher_Expecter{mock: &_m.Mock}
}
// Close provides a mock function with given fields:
func (_m *MockWatcher) Close() {
_m.Called()
}
// MockWatcher_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
type MockWatcher_Close_Call struct {
*mock.Call
}
// Close is a helper method to define mock.On call
func (_e *MockWatcher_Expecter) Close() *MockWatcher_Close_Call {
return &MockWatcher_Close_Call{Call: _e.mock.On("Close")}
}
func (_c *MockWatcher_Close_Call) Run(run func()) *MockWatcher_Close_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWatcher_Close_Call) Return() *MockWatcher_Close_Call {
_c.Call.Return()
return _c
}
func (_c *MockWatcher_Close_Call) RunAndReturn(run func()) *MockWatcher_Close_Call {
_c.Call.Return(run)
return _c
}
// EventChan provides a mock function with given fields:
func (_m *MockWatcher) EventChan() <-chan *messagespb.BroadcastEvent {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for EventChan")
}
var r0 <-chan *messagespb.BroadcastEvent
if rf, ok := ret.Get(0).(func() <-chan *messagespb.BroadcastEvent); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(<-chan *messagespb.BroadcastEvent)
}
}
return r0
}
// MockWatcher_EventChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'EventChan'
type MockWatcher_EventChan_Call struct {
*mock.Call
}
// EventChan is a helper method to define mock.On call
func (_e *MockWatcher_Expecter) EventChan() *MockWatcher_EventChan_Call {
return &MockWatcher_EventChan_Call{Call: _e.mock.On("EventChan")}
}
func (_c *MockWatcher_EventChan_Call) Run(run func()) *MockWatcher_EventChan_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWatcher_EventChan_Call) Return(_a0 <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWatcher_EventChan_Call) RunAndReturn(run func() <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call {
_c.Call.Return(run)
return _c
}
// ObserveResourceKeyEvent provides a mock function with given fields: ctx, ev
func (_m *MockWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *messagespb.BroadcastEvent) error {
ret := _m.Called(ctx, ev)
if len(ret) == 0 {
panic("no return value specified for ObserveResourceKeyEvent")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *messagespb.BroadcastEvent) error); ok {
r0 = rf(ctx, ev)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockWatcher_ObserveResourceKeyEvent_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ObserveResourceKeyEvent'
type MockWatcher_ObserveResourceKeyEvent_Call struct {
*mock.Call
}
// ObserveResourceKeyEvent is a helper method to define mock.On call
// - ctx context.Context
// - ev *messagespb.BroadcastEvent
func (_e *MockWatcher_Expecter) ObserveResourceKeyEvent(ctx interface{}, ev interface{}) *MockWatcher_ObserveResourceKeyEvent_Call {
return &MockWatcher_ObserveResourceKeyEvent_Call{Call: _e.mock.On("ObserveResourceKeyEvent", ctx, ev)}
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Run(run func(ctx context.Context, ev *messagespb.BroadcastEvent)) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*messagespb.BroadcastEvent))
})
return _c
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Return(_a0 error) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) RunAndReturn(run func(context.Context, *messagespb.BroadcastEvent) error) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Return(run)
return _c
}
// NewMockWatcher creates a new instance of MockWatcher. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockWatcher(t interface {
mock.TestingT
Cleanup(func())
}) *MockWatcher {
mock := &MockWatcher{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -43,6 +43,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord" grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord"
grpcdatacoordclient2 "github.com/milvus-io/milvus/internal/distributed/datacoord/client" grpcdatacoordclient2 "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode" grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode"
@ -366,6 +367,8 @@ func TestProxy(t *testing.T) {
var wg sync.WaitGroup var wg sync.WaitGroup
paramtable.Init() paramtable.Init()
params := paramtable.Get() params := paramtable.Get()
coordclient.ResetRegistration()
params.RootCoordGrpcServerCfg.IP = "localhost" params.RootCoordGrpcServerCfg.IP = "localhost"
params.QueryCoordGrpcServerCfg.IP = "localhost" params.QueryCoordGrpcServerCfg.IP = "localhost"
params.DataCoordGrpcServerCfg.IP = "localhost" params.DataCoordGrpcServerCfg.IP = "localhost"

View File

@ -14,14 +14,6 @@ import (
var _ task = (*broadcastTask)(nil) var _ task = (*broadcastTask)(nil)
// newBroadcastTask creates a new broadcast task.
func newBroadcastTask(ctx context.Context, core *Core, msgs []message.MutableMessage) *broadcastTask {
return &broadcastTask{
baseTask: newBaseTask(ctx, core),
msgs: msgs,
}
}
// BroadcastTask is used to implement the broadcast operation based on the msgstream // BroadcastTask is used to implement the broadcast operation based on the msgstream
// by using the streaming service interface. // by using the streaming service interface.
// msgstream will be deprecated since 2.6.0 with streaming service, so those code will be removed in the future version. // msgstream will be deprecated since 2.6.0 with streaming service, so those code will be removed in the future version.

View File

@ -31,6 +31,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
"github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks"
mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks"
@ -1430,6 +1431,7 @@ func TestRootcoord_EnableActiveStandby(t *testing.T) {
randVal := rand.Int() randVal := rand.Int()
paramtable.Init() paramtable.Init()
registry.ResetRegistration() registry.ResetRegistration()
coordclient.ResetRegistration()
Params.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal)) Params.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal))
// Need to reset global etcd to follow new path // Need to reset global etcd to follow new path
kvfactory.CloseEtcdClient() kvfactory.CloseEtcdClient()
@ -1491,6 +1493,7 @@ func TestRootcoord_DisableActiveStandby(t *testing.T) {
randVal := rand.Int() randVal := rand.Int()
paramtable.Init() paramtable.Init()
registry.ResetRegistration() registry.ResetRegistration()
coordclient.ResetRegistration()
Params.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal)) Params.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal))
// Need to reset global etcd to follow new path // Need to reset global etcd to follow new path
kvfactory.CloseEtcdClient() kvfactory.CloseEtcdClient()

View File

@ -2,30 +2,47 @@ package broadcast
import ( import (
"context" "context"
"time"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/messagespb" "github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/typeutil"
) )
// NewBroadcastService creates a new broadcast service. var logger = log.With(log.FieldComponent("broadcast-client"))
func NewBroadcastService(walName string, service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]) *BroadcastServiceImpl {
return &BroadcastServiceImpl{ // NewGRPCBroadcastService creates a new broadcast service with grpc.
func NewGRPCBroadcastService(walName string, service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]) *GRPCBroadcastServiceImpl {
rw := newResumingWatcher(&grpcWatcherBuilder{
broadcastService: service,
}, &typeutil.BackoffTimerConfig{
Default: 5 * time.Second,
Backoff: typeutil.BackoffConfig{
InitialInterval: 50 * time.Millisecond,
Multiplier: 2.0,
MaxInterval: 5 * time.Second,
},
})
return &GRPCBroadcastServiceImpl{
walName: walName, walName: walName,
service: service, service: service,
w: rw,
} }
} }
// BroadcastServiceImpl is the implementation of BroadcastService. // GRPCBroadcastServiceImpl is the implementation of BroadcastService based on grpc service.
type BroadcastServiceImpl struct { // If the streaming coord is not deployed at current node, these implementation will be used.
type GRPCBroadcastServiceImpl struct {
walName string walName string
service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient] service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]
w *resumingWatcher
} }
// Broadcast sends a broadcast message to the streaming coord to perform a broadcast. func (c *GRPCBroadcastServiceImpl) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
func (c *BroadcastServiceImpl) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
client, err := c.service.GetService(ctx) client, err := c.service.GetService(ctx)
if err != nil { if err != nil {
return nil, err return nil, err
@ -52,5 +69,28 @@ func (c *BroadcastServiceImpl) Broadcast(ctx context.Context, msg message.Broadc
Extra: result.GetExtra(), Extra: result.GetExtra(),
} }
} }
return &types.BroadcastAppendResult{AppendResults: results}, nil return &types.BroadcastAppendResult{
BroadcastID: resp.BroadcastId,
AppendResults: results,
}, nil
}
func (c *GRPCBroadcastServiceImpl) Ack(ctx context.Context, req types.BroadcastAckRequest) error {
client, err := c.service.GetService(ctx)
if err != nil {
return err
}
_, err = client.Ack(ctx, &streamingpb.BroadcastAckRequest{
BroadcastId: req.BroadcastID,
Vchannel: req.VChannel,
})
return err
}
func (c *GRPCBroadcastServiceImpl) BlockUntilEvent(ctx context.Context, ev *message.BroadcastEvent) error {
return c.w.ObserveResourceKeyEvent(ctx, ev)
}
func (c *GRPCBroadcastServiceImpl) Close() {
c.w.Close()
} }

View File

@ -0,0 +1,33 @@
package broadcast
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest"
)
func TestBroadcast(t *testing.T) {
s := newMockServer(t, 0)
bs := NewGRPCBroadcastService(walimplstest.WALName, s)
msg, _ := message.NewDropCollectionMessageBuilderV1().
WithHeader(&message.DropCollectionMessageHeader{}).
WithBody(&msgpb.DropCollectionRequest{}).
WithBroadcast([]string{"v1"}, message.NewCollectionNameResourceKey("r1")).
BuildBroadcast()
_, err := bs.Broadcast(context.Background(), msg)
assert.NoError(t, err)
err = bs.Ack(context.Background(), types.BroadcastAckRequest{
VChannel: "v1",
BroadcastID: 1,
})
assert.NoError(t, err)
err = bs.BlockUntilEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("r1")))
assert.NoError(t, err)
bs.Close()
}

View File

@ -0,0 +1,192 @@
package broadcast
import (
"context"
"io"
"time"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/util/syncutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type grpcWatcherBuilder struct {
broadcastService lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]
}
func (b *grpcWatcherBuilder) Build(ctx context.Context) (Watcher, error) {
service, err := b.broadcastService.GetService(ctx)
if err != nil {
return nil, errors.Wrap(err, "failed to get broadcast service")
}
bt := syncutil.NewAsyncTaskNotifier[struct{}]()
// TODO: Here we make a broken stream by passing a context.
// Implement a graceful closing should be better.
streamCtx, cancel := context.WithCancel(context.Background())
svr, err := service.Watch(streamCtx)
if err != nil {
cancel()
return nil, errors.Wrap(err, "failed to create broadcast watcher server client")
}
w := &grpcWatcherClient{
lifetime: typeutil.NewLifetime(),
backgroundTask: bt,
streamServerCancel: cancel,
streamClient: svr,
input: make(chan *message.BroadcastEvent),
output: make(chan *message.BroadcastEvent),
sendExitCh: make(chan struct{}),
recvExitCh: make(chan struct{}),
}
w.SetLogger(logger)
go w.executeBackgroundTask()
return w, nil
}
type grpcWatcherClient struct {
log.Binder
lifetime *typeutil.Lifetime
backgroundTask *syncutil.AsyncTaskNotifier[struct{}]
streamServerCancel context.CancelFunc
streamClient streamingpb.StreamingCoordBroadcastService_WatchClient
input chan *message.BroadcastEvent
output chan *message.BroadcastEvent
recvExitCh chan struct{}
sendExitCh chan struct{}
}
func (c *grpcWatcherClient) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
if !c.lifetime.Add(typeutil.LifetimeStateWorking) {
return errWatcherClosed
}
defer c.lifetime.Done()
select {
case <-ctx.Done():
return ctx.Err()
case <-c.backgroundTask.Context().Done():
return c.backgroundTask.Context().Err()
case c.input <- ev:
return nil
}
}
func (c *grpcWatcherClient) EventChan() <-chan *message.BroadcastEvent {
return c.output
}
func (c *grpcWatcherClient) gracefulClose() error {
c.lifetime.SetState(typeutil.LifetimeStateStopped)
// cancel the background task and wait for all request to finish
c.backgroundTask.Cancel()
c.lifetime.Wait()
select {
case <-c.backgroundTask.FinishChan():
return nil
case <-time.After(100 * time.Millisecond):
return context.DeadlineExceeded
}
}
func (c *grpcWatcherClient) Close() {
// Try to make a graceful close.
if err := c.gracefulClose(); err != nil {
c.Logger().Warn("failed to close the broadcast watcher gracefully, a froce closing will be applied", zap.Error(err))
}
c.streamServerCancel()
c.backgroundTask.BlockUntilFinish()
}
func (c *grpcWatcherClient) executeBackgroundTask() {
defer func() {
close(c.output)
c.backgroundTask.Finish(struct{}{})
}()
go c.recvLoop()
go c.sendLoop()
<-c.recvExitCh
<-c.sendExitCh
}
// sendLoop send the incoming event to the remote server.
// If the input channel is closed, it will send a close message to the remote server and return.
func (c *grpcWatcherClient) sendLoop() (err error) {
defer func() {
if err != nil {
c.Logger().Warn("send arm of stream closed by unexpected error", zap.Error(err))
} else {
c.Logger().Info("send arm of stream closed")
}
if err := c.streamClient.CloseSend(); err != nil {
c.Logger().Warn("failed to close send", zap.Error(err))
}
close(c.sendExitCh)
}()
for {
select {
case <-c.backgroundTask.Context().Done():
// send close message stop the loop.
// then the server will close the recv arm and return io.EOF.
// recv arm can be closed after that.
return c.streamClient.Send(&streamingpb.BroadcastWatchRequest{
Command: &streamingpb.BroadcastWatchRequest_Close{
Close: &streamingpb.CloseBroadcastWatchRequest{},
},
})
case ev := <-c.input:
if err := c.streamClient.Send(&streamingpb.BroadcastWatchRequest{
Command: &streamingpb.BroadcastWatchRequest_CreateEventWatch{
CreateEventWatch: &streamingpb.BroadcastCreateEventWatchRequest{
Event: ev,
},
},
}); err != nil {
return err
}
}
}
}
// recvLoop receive the event from the remote server.
func (c *grpcWatcherClient) recvLoop() (err error) {
defer func() {
if err != nil {
c.Logger().Warn("recv arm of stream closed by unexpected error", zap.Error(err))
} else {
c.Logger().Info("recv arm of stream closed")
}
close(c.recvExitCh)
}()
for {
resp, err := c.streamClient.Recv()
if errors.Is(err, io.EOF) {
return nil
}
if err != nil {
return err
}
switch resp := resp.Response.(type) {
case *streamingpb.BroadcastWatchResponse_EventDone:
select {
case c.output <- resp.EventDone.Event:
case <-c.backgroundTask.Context().Done():
c.Logger().Info("recv arm close when send event to output channel, skip wait for io.EOF")
return nil
}
case *streamingpb.BroadcastWatchResponse_Close:
// nothing to do now, just wait io.EOF.
default:
c.Logger().Warn("unknown response type", zap.Any("response", resp))
}
}
}

View File

@ -0,0 +1,125 @@
package broadcast
import (
"context"
"io"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/pkg/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest"
)
func TestWatcher(t *testing.T) {
s := newMockServer(t, 0)
b := grpcWatcherBuilder{broadcastService: s}
w, err := b.Build(context.Background())
assert.NoError(t, err)
done := make(chan struct{})
cnt := 0
go func() {
defer close(done)
for range w.EventChan() {
cnt++
}
}()
for i := 0; i < 10; i++ {
err = w.ObserveResourceKeyEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
assert.NoError(t, err)
}
time.Sleep(10 * time.Millisecond)
w.Close()
<-done
assert.Equal(t, 10, cnt)
err = w.ObserveResourceKeyEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
assert.Error(t, err)
// Test ungraceful close
s = newMockServer(t, 10*time.Second)
b2 := grpcWatcherBuilder{broadcastService: s}
w2, err := b2.Build(context.Background())
assert.NoError(t, err)
w2.Close()
}
func newMockServer(t *testing.T, sendDelay time.Duration) lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient] {
s := mock_lazygrpc.NewMockService[streamingpb.StreamingCoordBroadcastServiceClient](t)
c := mock_streamingpb.NewMockStreamingCoordBroadcastServiceClient(t)
s.EXPECT().GetService(mock.Anything).Return(c, nil)
var ctx context.Context
cc := mock_streamingpb.NewMockStreamingCoordBroadcastService_WatchClient(t)
c.EXPECT().Watch(mock.Anything).RunAndReturn(func(ctx2 context.Context, co ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error) {
ctx = ctx2
return cc, nil
})
c.EXPECT().Broadcast(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastResponse{
Results: map[string]*streamingpb.ProduceMessageResponseResult{
"v1": {
Id: &messagespb.MessageID{
Id: walimplstest.NewTestMessageID(1).Marshal(),
},
},
},
BroadcastId: 1,
}, nil).Maybe()
c.EXPECT().Ack(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastAckResponse{}, nil).Maybe()
output := make(chan *streamingpb.BroadcastWatchRequest, 10)
cc.EXPECT().Recv().RunAndReturn(func() (*streamingpb.BroadcastWatchResponse, error) {
var result *streamingpb.BroadcastWatchRequest
var ok bool
select {
case result, ok = <-output:
if !ok {
return nil, io.EOF
}
case <-ctx.Done():
return nil, ctx.Err()
}
switch cmd := result.Command.(type) {
case *streamingpb.BroadcastWatchRequest_Close:
return &streamingpb.BroadcastWatchResponse{
Response: &streamingpb.BroadcastWatchResponse_Close{Close: &streamingpb.CloseBroadcastWatchResponse{}},
}, nil
case *streamingpb.BroadcastWatchRequest_CreateEventWatch:
return &streamingpb.BroadcastWatchResponse{
Response: &streamingpb.BroadcastWatchResponse_EventDone{
EventDone: &streamingpb.BroadcastEventWatchResponse{
Event: cmd.CreateEventWatch.Event,
},
},
}, nil
default:
panic("unknown command")
}
})
cc.EXPECT().Send(mock.Anything).RunAndReturn(func(bwr *streamingpb.BroadcastWatchRequest) error {
select {
case <-time.After(sendDelay):
case <-ctx.Done():
return ctx.Err()
}
select {
case output <- bwr:
return nil
case <-ctx.Done():
return ctx.Err()
}
})
cc.EXPECT().CloseSend().RunAndReturn(func() error {
close(output)
return nil
})
return s
}

View File

@ -0,0 +1,22 @@
package broadcast
import (
"context"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
type WatcherBuilder interface {
Build(ctx context.Context) (Watcher, error)
}
type Watcher interface {
// ObserveResourceKeyEvent observes the resource key event.
ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error
// EventChan returns the event channel.
EventChan() <-chan *message.BroadcastEvent
// Close closes the watcher.
Close()
}

View File

@ -0,0 +1,171 @@
package broadcast
import (
"context"
"time"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/util/syncutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
var errWatcherClosed = errors.New("watcher is closed")
// newResumingWatcher create a new resuming watcher.
func newResumingWatcher(b WatcherBuilder, backoffConfig *typeutil.BackoffTimerConfig) *resumingWatcher {
rw := &resumingWatcher{
backgroundTask: syncutil.NewAsyncTaskNotifier[struct{}](),
input: make(chan *pendingEvent),
evs: &pendingEvents{evs: make(map[string]*pendingEvent)},
watcherBuilder: b, // TODO: enable local watcher here.
}
rw.SetLogger(logger)
go rw.execute(backoffConfig)
return rw
}
// resumingWatcher is a watcher that can resume the watcher when it is unavailable.
type resumingWatcher struct {
log.Binder
backgroundTask *syncutil.AsyncTaskNotifier[struct{}]
input chan *pendingEvent
evs *pendingEvents
watcherBuilder WatcherBuilder
}
// ObserveResourceKeyEvent observes the resource key event.
func (r *resumingWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
notifier := make(chan struct{})
select {
case <-r.backgroundTask.Context().Done():
return errWatcherClosed
case <-ctx.Done():
return ctx.Err()
case r.input <- &pendingEvent{
ev: ev,
notifier: []chan<- struct{}{notifier},
}:
}
select {
case <-r.backgroundTask.Context().Done():
return errWatcherClosed
case <-ctx.Done():
return ctx.Err()
case <-notifier:
return nil
}
}
func (r *resumingWatcher) Close() {
r.backgroundTask.Cancel()
r.backgroundTask.BlockUntilFinish()
}
func (r *resumingWatcher) execute(backoffConfig *typeutil.BackoffTimerConfig) {
backoff := typeutil.NewBackoffTimer(backoffConfig)
nextTimer := time.After(0)
var watcher Watcher
defer func() {
if watcher != nil {
watcher.Close()
}
r.backgroundTask.Finish(struct{}{})
}()
for {
var eventChan <-chan *message.BroadcastEvent
if watcher != nil {
eventChan = watcher.EventChan()
}
select {
case <-r.backgroundTask.Context().Done():
return
case ev := <-r.input:
if !r.evs.AddPendingEvent(ev) && watcher != nil {
if err := watcher.ObserveResourceKeyEvent(r.backgroundTask.Context(), ev.ev); err != nil {
watcher.Close()
watcher = nil
}
}
case ev, ok := <-eventChan:
if !ok {
watcher.Close()
watcher = nil
break
}
r.evs.Notify(ev)
case <-nextTimer:
var err error
if watcher, err = r.createNewWatcher(); err != nil {
r.Logger().Warn("create new watcher failed", zap.Error(err))
break
}
r.Logger().Info("create new watcher successful")
backoff.DisableBackoff()
nextTimer = nil
}
if watcher == nil {
backoff.EnableBackoff()
var interval time.Duration
nextTimer, interval = backoff.NextTimer()
r.Logger().Warn("watcher is unavailable, resuming it after interval", zap.Duration("interval", interval))
}
}
}
func (r *resumingWatcher) createNewWatcher() (Watcher, error) {
watcher, err := r.watcherBuilder.Build(r.backgroundTask.Context())
if err != nil {
return nil, err
}
if err := r.evs.SendAll(r.backgroundTask.Context(), watcher); err != nil {
watcher.Close()
return nil, errors.Wrapf(err, "send all pending events to watcher failed")
}
return watcher, nil
}
type pendingEvents struct {
evs map[string]*pendingEvent
}
// AddPendingEvent adds a pending event.
// Return true if the event is already in the pending events.
func (evs *pendingEvents) AddPendingEvent(ev *pendingEvent) bool {
id := message.UniqueKeyOfBroadcastEvent(ev.ev)
if existEv, ok := evs.evs[id]; ok {
existEv.notifier = append(existEv.notifier, ev.notifier...)
return true
}
evs.evs[id] = ev
return false
}
func (evs *pendingEvents) Notify(ev *message.BroadcastEvent) {
id := message.UniqueKeyOfBroadcastEvent(ev)
if existEv, ok := evs.evs[id]; ok {
for _, notifier := range existEv.notifier {
close(notifier)
}
delete(evs.evs, id)
}
}
func (evs *pendingEvents) SendAll(ctx context.Context, w Watcher) error {
for _, ev := range evs.evs {
if err := w.ObserveResourceKeyEvent(ctx, ev.ev); err != nil {
return err
}
}
return nil
}
type pendingEvent struct {
ev *message.BroadcastEvent
notifier []chan<- struct{}
}

View File

@ -0,0 +1,101 @@
package broadcast
import (
"context"
"fmt"
"math/rand"
"sync"
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"go.uber.org/atomic"
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/client/mock_broadcast"
"github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type mockBuilder struct {
built func(ctx context.Context) (Watcher, error)
}
func (b *mockBuilder) Build(ctx context.Context) (Watcher, error) {
return b.built(ctx)
}
func TestWatcherResuming(t *testing.T) {
ctx := context.Background()
b := newMockWatcherBuilder(t)
rw := newResumingWatcher(b, &typeutil.BackoffTimerConfig{
Default: 500 * time.Millisecond,
Backoff: typeutil.BackoffConfig{
InitialInterval: 10 * time.Millisecond,
Multiplier: 2.0,
MaxInterval: 500 * time.Millisecond,
},
})
wg := &sync.WaitGroup{}
for i := 0; i < 100; i++ {
wg.Add(1)
go func() {
defer wg.Done()
id := rand.Int31n(10)
rk := message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey(fmt.Sprintf("c%d", id)))
err := rw.ObserveResourceKeyEvent(ctx, rk)
assert.NoError(t, err)
}()
}
wg.Wait()
rw.Close()
err := rw.ObserveResourceKeyEvent(ctx, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
assert.ErrorIs(t, err, errWatcherClosed)
}
func newMockWatcherBuilder(t *testing.T) WatcherBuilder {
return &mockBuilder{built: func(ctx context.Context) (Watcher, error) {
w := mock_broadcast.NewMockWatcher(t)
n := rand.Int31n(10)
if n < 3 {
return nil, errors.New("err")
}
// ill watcher
k := atomic.NewInt32(n)
o := rand.Int31n(20) + n
mu := sync.Mutex{}
closed := false
output := make(chan *message.BroadcastEvent, 500)
w.EXPECT().ObserveResourceKeyEvent(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, be *messagespb.BroadcastEvent) error {
k2 := k.Inc()
if k2 >= o {
return errors.New("err")
}
mu.Lock()
if closed {
return errors.New("closed")
}
go func() {
defer mu.Unlock()
time.Sleep(time.Duration(rand.Int31n(5)) * time.Millisecond)
output <- be
}()
return nil
}).Maybe()
w.EXPECT().EventChan().RunAndReturn(func() <-chan *messagespb.BroadcastEvent {
mu.Lock()
defer mu.Unlock()
if !closed && rand.Int31n(100) < 50 {
close(output)
closed = true
}
return output
}).Maybe()
w.EXPECT().Close().Return()
return w, nil
}}
}

View File

@ -40,6 +40,15 @@ type AssignmentService interface {
type BroadcastService interface { type BroadcastService interface {
// Broadcast sends a broadcast message to the streaming service. // Broadcast sends a broadcast message to the streaming service.
Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)
// Ack sends a broadcast ack to the streaming service.
Ack(ctx context.Context, req types.BroadcastAckRequest) error
// BlockUntilEvent blocks until the event happens.
BlockUntilEvent(ctx context.Context, ev *message.BroadcastEvent) error
// Close closes the broadcast service.
Close()
} }
// Client is the interface of log service client. // Client is the interface of log service client.
@ -84,7 +93,7 @@ func NewClient(etcdCli *clientv3.Client) Client {
conn: conn, conn: conn,
rb: rb, rb: rb,
assignmentService: assignmentServiceImpl, assignmentService: assignmentServiceImpl,
broadcastService: broadcast.NewBroadcastService(util.MustSelectWALName(), broadcastService), broadcastService: broadcast.NewGRPCBroadcastService(util.MustSelectWALName(), broadcastService),
} }
} }

View File

@ -2,7 +2,6 @@ package client
import ( import (
"github.com/milvus-io/milvus/internal/streamingcoord/client/assignment" "github.com/milvus-io/milvus/internal/streamingcoord/client/assignment"
"github.com/milvus-io/milvus/internal/streamingcoord/client/broadcast"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver"
) )
@ -12,7 +11,7 @@ type clientImpl struct {
conn lazygrpc.Conn conn lazygrpc.Conn
rb resolver.Builder rb resolver.Builder
assignmentService *assignment.AssignmentServiceImpl assignmentService *assignment.AssignmentServiceImpl
broadcastService *broadcast.BroadcastServiceImpl broadcastService BroadcastService
} }
func (c *clientImpl) Broadcast() BroadcastService { func (c *clientImpl) Broadcast() BroadcastService {
@ -29,6 +28,7 @@ func (c *clientImpl) Close() {
if c.assignmentService != nil { if c.assignmentService != nil {
c.assignmentService.Close() c.assignmentService.Close()
} }
c.broadcastService.Close()
c.conn.Close() c.conn.Close()
c.rb.Close() c.rb.Close()
} }

View File

@ -0,0 +1,182 @@
package broadcaster
import (
"context"
"fmt"
"sync"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/util/syncutil"
)
// newBroadcastTaskManager creates a new broadcast task manager with recovery info.
func newBroadcastTaskManager(protos []*streamingpb.BroadcastTask) (*broadcastTaskManager, []*pendingBroadcastTask) {
logger := resource.Resource().Logger().With(log.FieldComponent("broadcaster"))
recoveryTasks := make([]*broadcastTask, 0, len(protos))
for _, proto := range protos {
t := newBroadcastTaskFromProto(proto)
t.SetLogger(logger.With(zap.Uint64("broadcastID", t.header.BroadcastID)))
recoveryTasks = append(recoveryTasks, t)
}
rks := make(map[message.ResourceKey]uint64, len(recoveryTasks))
tasks := make(map[uint64]*broadcastTask, len(recoveryTasks))
pendingTasks := make([]*pendingBroadcastTask, 0, len(recoveryTasks))
for _, task := range recoveryTasks {
for rk := range task.header.ResourceKeys {
if oldTaskID, ok := rks[rk]; ok {
panic(fmt.Sprintf("unreachable: dirty recovery info in metastore, broadcast ids: [%d, %d]", oldTaskID, task.header.BroadcastID))
}
rks[rk] = task.header.BroadcastID
}
tasks[task.header.BroadcastID] = task
if task.task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING {
// only the task is pending need to be reexecuted.
pendingTasks = append(pendingTasks, newPendingBroadcastTask(task))
}
}
m := &broadcastTaskManager{
Binder: log.Binder{},
cond: syncutil.NewContextCond(&sync.Mutex{}),
tasks: tasks,
resourceKeys: rks,
version: 1,
}
m.SetLogger(logger)
return m, pendingTasks
}
// broadcastTaskManager is the manager of the broadcast task.
type broadcastTaskManager struct {
log.Binder
cond *syncutil.ContextCond
tasks map[uint64]*broadcastTask // map the broadcastID to the broadcastTaskState
resourceKeys map[message.ResourceKey]uint64 // map the resource key to the broadcastID
version int // version is used to make sure that there's no update lost for watcher.
}
// AddTask adds a new broadcast task into the manager.
func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.BroadcastMutableMessage) (*pendingBroadcastTask, error) {
id, err := resource.Resource().IDAllocator().Allocate(ctx)
if err != nil {
return nil, errors.Wrapf(err, "allocate new id failed")
}
msg = msg.WithBroadcastID(id)
task, err := bm.addBroadcastTask(msg)
if err != nil {
return nil, err
}
return newPendingBroadcastTask(task), nil
}
// Ack acknowledges the message at the specified vchannel.
func (bm *broadcastTaskManager) Ack(ctx context.Context, broadcastID uint64, vchannel string) error {
task, ok := bm.getBroadcastTaskByID(broadcastID)
if !ok {
bm.Logger().Warn("broadcast task not found, it may already acked, ignore the request", zap.Uint64("broadcastID", broadcastID), zap.String("vchannel", vchannel))
return nil
}
if err := task.Ack(ctx, vchannel); err != nil {
return err
}
if task.State() == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE {
bm.removeBroadcastTask(broadcastID)
} else {
bm.increaseVersion()
}
return nil
}
// WatchAtVersion watches the version of the broadcast task manager.
// When the version is greater than the input version, the watcher will be notified.
func (bm *broadcastTaskManager) WatchAtVersion(version int) <-chan struct{} {
bm.cond.L.Lock()
if bm.version > version {
bm.cond.L.Unlock()
ch := make(chan struct{})
close(ch)
return ch
}
return bm.cond.WaitChan()
}
// CurrentVersion returns the current version of the broadcast task manager.
func (bm *broadcastTaskManager) CurrentVersion() int {
bm.cond.L.Lock()
defer bm.cond.L.Unlock()
return bm.version
}
// GetBroadcastTaskByResourceKey returns the broadcast task by the resource key.
func (bm *broadcastTaskManager) GetBroadcastTaskByResourceKey(resourceKey message.ResourceKey) (*broadcastTask, bool) {
bm.cond.L.Lock()
defer bm.cond.L.Unlock()
broadcastID, ok := bm.resourceKeys[resourceKey]
if !ok {
return nil, false
}
task, ok := bm.tasks[broadcastID]
return task, ok
}
// addBroadcastTask adds the broadcast task into the manager.
func (bm *broadcastTaskManager) addBroadcastTask(msg message.BroadcastMutableMessage) (*broadcastTask, error) {
newIncomingTask := newBroadcastTaskFromBroadcastMessage(msg)
header := newIncomingTask.Header()
newIncomingTask.SetLogger(bm.Logger().With(zap.Uint64("broadcastID", header.BroadcastID)))
bm.cond.L.Lock()
defer bm.cond.L.Unlock()
// Check if the resource key is held by other task.
for key := range header.ResourceKeys {
if _, ok := bm.resourceKeys[key]; ok {
return nil, status.NewResourceAcquired(fmt.Sprintf("domain: %s, key: %s", key.Domain.String(), key.Key))
}
}
// setup the resource keys to make resource exclusive held.
for key := range header.ResourceKeys {
bm.resourceKeys[key] = header.BroadcastID
}
bm.tasks[header.BroadcastID] = newIncomingTask
return newIncomingTask, nil
}
// getBroadcastTaskByID return the task by the broadcastID.
func (bm *broadcastTaskManager) getBroadcastTaskByID(broadcastID uint64) (*broadcastTask, bool) {
bm.cond.L.Lock()
defer bm.cond.L.Unlock()
t, ok := bm.tasks[broadcastID]
return t, ok
}
// removeBroadcastTask removes the broadcast task by the broadcastID.
func (bm *broadcastTaskManager) removeBroadcastTask(broadcastID uint64) {
bm.cond.LockAndBroadcast()
defer bm.cond.L.Unlock()
task, ok := bm.tasks[broadcastID]
if !ok {
return
}
bm.version++
// remove the related resource keys
for key := range task.header.ResourceKeys {
delete(bm.resourceKeys, key)
}
delete(bm.tasks, broadcastID)
}
func (bm *broadcastTaskManager) increaseVersion() {
bm.cond.LockAndBroadcast()
bm.version++
bm.cond.L.Unlock()
}

View File

@ -0,0 +1,198 @@
package broadcaster
import (
"context"
"sync"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
// newBroadcastTaskFromProto creates a new broadcast task from the proto.
func newBroadcastTaskFromProto(proto *streamingpb.BroadcastTask) *broadcastTask {
msg := message.NewBroadcastMutableMessageBeforeAppend(proto.Message.Payload, proto.Message.Properties)
bh := msg.BroadcastHeader()
ackedCount := 0
for _, acked := range proto.AckedVchannelBitmap {
ackedCount += int(acked)
}
return &broadcastTask{
mu: sync.Mutex{},
header: bh,
task: proto,
ackedCount: ackedCount,
recoverPersisted: true, // the task is recovered from the recovery info, so it's persisted.
}
}
// newBroadcastTaskFromBroadcastMessage creates a new broadcast task from the broadcast message.
func newBroadcastTaskFromBroadcastMessage(msg message.BroadcastMutableMessage) *broadcastTask {
header := msg.BroadcastHeader()
return &broadcastTask{
Binder: log.Binder{},
mu: sync.Mutex{},
header: header,
task: &streamingpb.BroadcastTask{
Message: &messagespb.Message{Payload: msg.Payload(), Properties: msg.Properties().ToRawMap()},
State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING,
AckedVchannelBitmap: make([]byte, len(header.VChannels)),
},
ackedCount: 0,
recoverPersisted: false,
}
}
// broadcastTask is the state of the broadcast task.
type broadcastTask struct {
log.Binder
mu sync.Mutex
header *message.BroadcastHeader
task *streamingpb.BroadcastTask
ackedCount int // the count of the acked vchannels, the idompotenace is promised by task's bitmap.
// always keep same with the positive counter of task's acked_bitmap.
recoverPersisted bool // a flag to indicate that the task has been persisted into the recovery info and can be recovered.
}
// Header returns the header of the broadcast task.
func (b *broadcastTask) Header() *message.BroadcastHeader {
// header is a immutable field, no need to lock.
return b.header
}
// State returns the State of the broadcast task.
func (b *broadcastTask) State() streamingpb.BroadcastTaskState {
b.mu.Lock()
defer b.mu.Unlock()
return b.task.State
}
// PendingBroadcastMessages returns the pending broadcast message of current broad cast.
func (b *broadcastTask) PendingBroadcastMessages() []message.MutableMessage {
b.mu.Lock()
defer b.mu.Unlock()
msg := message.NewBroadcastMutableMessageBeforeAppend(b.task.Message.Payload, b.task.Message.Properties)
msgs := msg.SplitIntoMutableMessage()
// If there's no vchannel acked, return all the messages directly.
if b.ackedCount == 0 {
return msgs
}
// filter out the vchannel that has been acked.
pendingMessages := make([]message.MutableMessage, 0, len(msgs))
for i, msg := range msgs {
if b.task.AckedVchannelBitmap[i] != 0 {
continue
}
pendingMessages = append(pendingMessages, msg)
}
return pendingMessages
}
// InitializeRecovery initializes the recovery of the broadcast task.
func (b *broadcastTask) InitializeRecovery(ctx context.Context) error {
b.mu.Lock()
defer b.mu.Unlock()
if b.recoverPersisted {
return nil
}
if err := b.saveTask(ctx, b.Logger()); err != nil {
return err
}
b.recoverPersisted = true
return nil
}
// Ack acknowledges the message at the specified vchannel.
func (b *broadcastTask) Ack(ctx context.Context, vchannel string) error {
b.mu.Lock()
defer b.mu.Unlock()
b.setVChannelAcked(vchannel)
if b.isAllDone() {
// All vchannels are acked, mark the task as done, even if there are still pending messages on working.
// The pending messages is repeated sent operation, can be ignored.
b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE
}
// We should always save the task after acked.
// Even if the task mark as done in memory.
// Because the task is set as done in memory before save the recovery info.
return b.saveTask(ctx, b.Logger().With(zap.String("ackVChannel", vchannel)))
}
// setVChannelAcked sets the vchannel as acked.
func (b *broadcastTask) setVChannelAcked(vchannel string) {
idx, err := b.findIdxOfVChannel(vchannel)
if err != nil {
panic(err)
}
b.task.AckedVchannelBitmap[idx] = 1
// Check if all vchannels are acked.
ackedCount := 0
for _, acked := range b.task.AckedVchannelBitmap {
ackedCount += int(acked)
}
b.ackedCount = ackedCount
}
// findIdxOfVChannel finds the index of the vchannel in the broadcast task.
func (b *broadcastTask) findIdxOfVChannel(vchannel string) (int, error) {
for i, channelName := range b.header.VChannels {
if channelName == vchannel {
return i, nil
}
}
return -1, errors.Errorf("unreachable: vchannel is %s not found in the broadcast task", vchannel)
}
// isAllDone check if all the vchannels are acked.
func (b *broadcastTask) isAllDone() bool {
return b.ackedCount == len(b.header.VChannels)
}
// BroadcastDone marks the broadcast operation is done.
func (b *broadcastTask) BroadcastDone(ctx context.Context) error {
b.mu.Lock()
defer b.mu.Unlock()
if b.isAllDone() {
// If all vchannels are acked, mark the task as done.
b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE
} else {
// There's no more pending message, mark the task as wait ack.
b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_WAIT_ACK
}
return b.saveTask(ctx, b.Logger())
}
// IsAllAcked returns true if all the vchannels are acked.
func (b *broadcastTask) IsAllAcked() bool {
b.mu.Lock()
defer b.mu.Unlock()
return b.isAllDone()
}
// IsAcked returns true if any vchannel is acked.
func (b *broadcastTask) IsAcked() bool {
b.mu.Lock()
defer b.mu.Unlock()
return b.ackedCount > 0
}
// saveTask saves the broadcast task recovery info.
func (b *broadcastTask) saveTask(ctx context.Context, logger *log.MLogger) error {
logger = logger.With(zap.String("state", b.task.State.String()), zap.Int("ackedVChannelCount", b.ackedCount))
if err := resource.Resource().StreamingCatalog().SaveBroadcastTask(ctx, b.header.BroadcastID, b.task); err != nil {
logger.Warn("save broadcast task failed", zap.Error(err))
return err
}
logger.Info("save broadcast task done")
return nil
}

View File

@ -12,8 +12,26 @@ type Broadcaster interface {
// Broadcast broadcasts the message to all channels. // Broadcast broadcasts the message to all channels.
Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)
// Ack acknowledges the message at the specified vchannel.
Ack(ctx context.Context, req types.BroadcastAckRequest) error
// Watch watches the broadcast event.
NewWatcher() (Watcher, error)
// Close closes the broadcaster. // Close closes the broadcaster.
Close() Close()
} }
// Watcher is the interface for watching the broadcast event.
type Watcher interface {
// ObserveResourceKeyEvent observes the resource key event.
ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error
// EventChan returns the event channel.
EventChan() <-chan *message.BroadcastEvent
// Close closes the watcher.
Close()
}
type AppendOperator = registry.AppendOperator type AppendOperator = registry.AppendOperator

View File

@ -10,8 +10,6 @@ import (
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/contextutil" "github.com/milvus-io/milvus/pkg/util/contextutil"
@ -25,28 +23,20 @@ func RecoverBroadcaster(
ctx context.Context, ctx context.Context,
appendOperator *syncutil.Future[AppendOperator], appendOperator *syncutil.Future[AppendOperator],
) (Broadcaster, error) { ) (Broadcaster, error) {
logger := resource.Resource().Logger().With(log.FieldComponent("broadcaster"))
tasks, err := resource.Resource().StreamingCatalog().ListBroadcastTask(ctx) tasks, err := resource.Resource().StreamingCatalog().ListBroadcastTask(ctx)
if err != nil { if err != nil {
return nil, err return nil, err
} }
pendings := make([]*broadcastTask, 0, len(tasks)) manager, pendings := newBroadcastTaskManager(tasks)
for _, task := range tasks {
if task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING {
// recover pending task
t := newTask(task, logger)
pendings = append(pendings, t)
}
}
b := &broadcasterImpl{ b := &broadcasterImpl{
logger: logger, manager: manager,
lifetime: typeutil.NewLifetime(), lifetime: typeutil.NewLifetime(),
backgroundTaskNotifier: syncutil.NewAsyncTaskNotifier[struct{}](), backgroundTaskNotifier: syncutil.NewAsyncTaskNotifier[struct{}](),
pendings: pendings, pendings: pendings,
backoffs: typeutil.NewHeap[*broadcastTask](&broadcastTaskArray{}), backoffs: typeutil.NewHeap[*pendingBroadcastTask](&pendingBroadcastTaskArray{}),
backoffChan: make(chan *broadcastTask), backoffChan: make(chan *pendingBroadcastTask),
pendingChan: make(chan *broadcastTask), pendingChan: make(chan *pendingBroadcastTask),
workerChan: make(chan *broadcastTask), workerChan: make(chan *pendingBroadcastTask),
appendOperator: appendOperator, appendOperator: appendOperator,
} }
go b.execute() go b.execute()
@ -55,14 +45,14 @@ func RecoverBroadcaster(
// broadcasterImpl is the implementation of Broadcaster // broadcasterImpl is the implementation of Broadcaster
type broadcasterImpl struct { type broadcasterImpl struct {
logger *log.MLogger manager *broadcastTaskManager
lifetime *typeutil.Lifetime lifetime *typeutil.Lifetime
backgroundTaskNotifier *syncutil.AsyncTaskNotifier[struct{}] backgroundTaskNotifier *syncutil.AsyncTaskNotifier[struct{}]
pendings []*broadcastTask pendings []*pendingBroadcastTask
backoffs typeutil.Heap[*broadcastTask] backoffs typeutil.Heap[*pendingBroadcastTask]
pendingChan chan *broadcastTask pendingChan chan *pendingBroadcastTask
backoffChan chan *broadcastTask backoffChan chan *pendingBroadcastTask
workerChan chan *broadcastTask workerChan chan *pendingBroadcastTask
appendOperator *syncutil.Future[AppendOperator] // TODO: we can remove those lazy future in 2.6.0, by remove the msgstream broadcaster. appendOperator *syncutil.Future[AppendOperator] // TODO: we can remove those lazy future in 2.6.0, by remove the msgstream broadcaster.
} }
@ -72,18 +62,17 @@ func (b *broadcasterImpl) Broadcast(ctx context.Context, msg message.BroadcastMu
return nil, status.NewOnShutdownError("broadcaster is closing") return nil, status.NewOnShutdownError("broadcaster is closing")
} }
defer func() { defer func() {
b.lifetime.Done()
if err != nil { if err != nil {
b.logger.Warn("broadcast message failed", zap.Error(err)) b.Logger().Warn("broadcast message failed", zap.Error(err))
return return
} }
}() }()
// Once the task is persisted, it must be successful. t, err := b.manager.AddTask(ctx, msg)
task, err := b.persistBroadcastTask(ctx, msg)
if err != nil { if err != nil {
return nil, err return nil, err
} }
t := newTask(task, b.logger)
select { select {
case <-b.backgroundTaskNotifier.Context().Done(): case <-b.backgroundTaskNotifier.Context().Done():
// We can only check the background context but not the request context here. // We can only check the background context but not the request context here.
@ -98,24 +87,23 @@ func (b *broadcasterImpl) Broadcast(ctx context.Context, msg message.BroadcastMu
return t.BlockUntilTaskDone(ctx) return t.BlockUntilTaskDone(ctx)
} }
// persistBroadcastTask persists the broadcast task into catalog. // Ack acknowledges the message at the specified vchannel.
func (b *broadcasterImpl) persistBroadcastTask(ctx context.Context, msg message.BroadcastMutableMessage) (*streamingpb.BroadcastTask, error) { func (b *broadcasterImpl) Ack(ctx context.Context, req types.BroadcastAckRequest) error {
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
return status.NewOnShutdownError("broadcaster is closing")
}
defer b.lifetime.Done() defer b.lifetime.Done()
id, err := resource.Resource().IDAllocator().Allocate(ctx) return b.manager.Ack(ctx, req.BroadcastID, req.VChannel)
if err != nil { }
return nil, status.NewInner("allocate new id failed, %s", err.Error())
func (b *broadcasterImpl) NewWatcher() (Watcher, error) {
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
return nil, status.NewOnShutdownError("broadcaster is closing")
} }
task := &streamingpb.BroadcastTask{ defer b.lifetime.Done()
TaskId: int64(id),
Message: &messagespb.Message{Payload: msg.Payload(), Properties: msg.Properties().ToRawMap()}, return newWatcher(b), nil
State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING,
}
// Save the task into catalog to help recovery.
if err := resource.Resource().StreamingCatalog().SaveBroadcastTask(ctx, task); err != nil {
return nil, status.NewInner("save broadcast task failed, %s", err.Error())
}
return task, nil
} }
func (b *broadcasterImpl) Close() { func (b *broadcasterImpl) Close() {
@ -126,26 +114,30 @@ func (b *broadcasterImpl) Close() {
b.backgroundTaskNotifier.BlockUntilFinish() b.backgroundTaskNotifier.BlockUntilFinish()
} }
func (b *broadcasterImpl) Logger() *log.MLogger {
return b.manager.Logger()
}
// execute the broadcaster // execute the broadcaster
func (b *broadcasterImpl) execute() { func (b *broadcasterImpl) execute() {
workers := int(float64(hardware.GetCPUNum()) * paramtable.Get().StreamingCfg.WALBroadcasterConcurrencyRatio.GetAsFloat()) workers := int(float64(hardware.GetCPUNum()) * paramtable.Get().StreamingCfg.WALBroadcasterConcurrencyRatio.GetAsFloat())
if workers < 1 { if workers < 1 {
workers = 1 workers = 1
} }
b.logger.Info("broadcaster start to execute", zap.Int("workerNum", workers)) b.Logger().Info("broadcaster start to execute", zap.Int("workerNum", workers))
defer func() { defer func() {
b.backgroundTaskNotifier.Finish(struct{}{}) b.backgroundTaskNotifier.Finish(struct{}{})
b.logger.Info("broadcaster execute exit") b.Logger().Info("broadcaster execute exit")
}() }()
// Wait for appendOperator ready // Wait for appendOperator ready
appendOperator, err := b.appendOperator.GetWithContext(b.backgroundTaskNotifier.Context()) appendOperator, err := b.appendOperator.GetWithContext(b.backgroundTaskNotifier.Context())
if err != nil { if err != nil {
b.logger.Info("broadcaster is closed before appendOperator ready") b.Logger().Info("broadcaster is closed before appendOperator ready")
return return
} }
b.logger.Info("broadcaster appendOperator ready, begin to start workers and dispatch") b.Logger().Info("broadcaster appendOperator ready, begin to start workers and dispatch")
// Start n workers to handle the broadcast task. // Start n workers to handle the broadcast task.
wg := sync.WaitGroup{} wg := sync.WaitGroup{}
@ -165,8 +157,8 @@ func (b *broadcasterImpl) execute() {
func (b *broadcasterImpl) dispatch() { func (b *broadcasterImpl) dispatch() {
for { for {
var workerChan chan *broadcastTask var workerChan chan *pendingBroadcastTask
var nextTask *broadcastTask var nextTask *pendingBroadcastTask
var nextBackOff <-chan time.Time var nextBackOff <-chan time.Time
// Wait for new task. // Wait for new task.
if len(b.pendings) > 0 { if len(b.pendings) > 0 {
@ -176,7 +168,7 @@ func (b *broadcasterImpl) dispatch() {
if b.backoffs.Len() > 0 { if b.backoffs.Len() > 0 {
var nextInterval time.Duration var nextInterval time.Duration
nextBackOff, nextInterval = b.backoffs.Peek().NextTimer() nextBackOff, nextInterval = b.backoffs.Peek().NextTimer()
b.logger.Info("backoff task", zap.Duration("nextInterval", nextInterval)) b.Logger().Info("backoff task", zap.Duration("nextInterval", nextInterval))
} }
select { select {
@ -189,7 +181,7 @@ func (b *broadcasterImpl) dispatch() {
b.backoffs.Push(task) b.backoffs.Push(task)
case <-nextBackOff: case <-nextBackOff:
// backoff is done, move all the backoff done task into pending to retry. // backoff is done, move all the backoff done task into pending to retry.
newPops := make([]*broadcastTask, 0) newPops := make([]*pendingBroadcastTask, 0)
for b.backoffs.Len() > 0 && b.backoffs.Peek().NextInterval() < time.Millisecond { for b.backoffs.Len() > 0 && b.backoffs.Peek().NextInterval() < time.Millisecond {
newPops = append(newPops, b.backoffs.Pop()) newPops = append(newPops, b.backoffs.Pop())
} }
@ -205,7 +197,7 @@ func (b *broadcasterImpl) dispatch() {
} }
func (b *broadcasterImpl) worker(no int, appendOperator AppendOperator) { func (b *broadcasterImpl) worker(no int, appendOperator AppendOperator) {
logger := b.logger.With(zap.Int("workerNo", no)) logger := b.Logger().With(zap.Int("workerNo", no))
defer func() { defer func() {
logger.Info("broadcaster worker exit") logger.Info("broadcaster worker exit")
}() }()

View File

@ -3,6 +3,7 @@ package broadcaster
import ( import (
"context" "context"
"math/rand" "math/rand"
"sync"
"testing" "testing"
"time" "time"
@ -10,6 +11,7 @@ import (
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"go.uber.org/atomic" "go.uber.org/atomic"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/mocks/mock_metastore" "github.com/milvus-io/milvus/internal/mocks/mock_metastore"
@ -17,6 +19,7 @@ import (
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
internaltypes "github.com/milvus-io/milvus/internal/types" internaltypes "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/idalloc" "github.com/milvus-io/milvus/internal/util/idalloc"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/proto/messagespb" "github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message"
@ -24,6 +27,7 @@ import (
"github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest"
"github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/syncutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
) )
func TestBroadcaster(t *testing.T) { func TestBroadcaster(t *testing.T) {
@ -33,19 +37,37 @@ func TestBroadcaster(t *testing.T) {
meta.EXPECT().ListBroadcastTask(mock.Anything). meta.EXPECT().ListBroadcastTask(mock.Anything).
RunAndReturn(func(ctx context.Context) ([]*streamingpb.BroadcastTask, error) { RunAndReturn(func(ctx context.Context) ([]*streamingpb.BroadcastTask, error) {
return []*streamingpb.BroadcastTask{ return []*streamingpb.BroadcastTask{
createNewBroadcastTask(1, []string{"v1"}), createNewBroadcastTask(1, []string{"v1"}, message.NewCollectionNameResourceKey("c1")),
createNewBroadcastTask(2, []string{"v1", "v2"}), createNewBroadcastTask(2, []string{"v1", "v2"}, message.NewCollectionNameResourceKey("c2")),
createNewBroadcastTask(3, []string{"v1", "v2", "v3"}), createNewBroadcastTask(3, []string{"v1", "v2", "v3"}),
createNewWaitAckBroadcastTaskFromMessage(
createNewBroadcastMsg([]string{"v1", "v2", "v3"}).WithBroadcastID(4),
streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING,
[]byte{0x00, 0x01, 0x00}),
createNewWaitAckBroadcastTaskFromMessage(
createNewBroadcastMsg([]string{"v1", "v2", "v3"}).WithBroadcastID(5),
streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING,
[]byte{0x01, 0x01, 0x00}),
createNewWaitAckBroadcastTaskFromMessage(
createNewBroadcastMsg([]string{"v1", "v2", "v3"}).WithBroadcastID(6), // will be done directly.
streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING,
[]byte{0x01, 0x01, 0x01}),
createNewWaitAckBroadcastTaskFromMessage(
createNewBroadcastMsg([]string{"v1", "v2", "v3"},
message.NewCollectionNameResourceKey("c3"),
message.NewCollectionNameResourceKey("c4")).WithBroadcastID(7),
streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_WAIT_ACK,
[]byte{0x00, 0x00, 0x00}),
}, nil }, nil
}).Times(1) }).Times(1)
done := atomic.NewInt64(0) done := typeutil.NewConcurrentSet[uint64]()
meta.EXPECT().SaveBroadcastTask(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, bt *streamingpb.BroadcastTask) error { meta.EXPECT().SaveBroadcastTask(mock.Anything, mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, broadcastID uint64, bt *streamingpb.BroadcastTask) error {
// may failure // may failure
if rand.Int31n(10) < 3 { if rand.Int31n(10) < 3 {
return errors.New("save task failed") return errors.New("save task failed")
} }
if bt.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE { if bt.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE {
done.Inc() done.Insert(broadcastID)
} }
return nil return nil
}) })
@ -59,30 +81,145 @@ func TestBroadcaster(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
assert.NotNil(t, bc) assert.NotNil(t, bc)
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
return appended.Load() == 6 && done.Load() == 3 return appended.Load() == 9 && len(done.Collect()) == 1 // only one task is done,
}, 30*time.Second, 10*time.Millisecond) }, 30*time.Second, 10*time.Millisecond)
// Test ack here
wg := &sync.WaitGroup{}
asyncAck(wg, bc, 1, "v1")
asyncAck(wg, bc, 2, "v2")
asyncAck(wg, bc, 3, "v3")
asyncAck(wg, bc, 3, "v2")
// repeatoperation should be ok.
asyncAck(wg, bc, 1, "v1")
asyncAck(wg, bc, 2, "v2")
asyncAck(wg, bc, 3, "v3")
asyncAck(wg, bc, 3, "v2")
wg.Wait()
assert.Eventually(t, func() bool {
return len(done.Collect()) == 2
}, 30*time.Second, 10*time.Millisecond)
// Test broadcast here.
var result *types.BroadcastAppendResult var result *types.BroadcastAppendResult
for { for {
var err error var err error
result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"})) result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"}, message.NewCollectionNameResourceKey("c7")))
if err == nil { if err == nil {
break break
} }
} }
assert.Equal(t, int(appended.Load()), 9) assert.Equal(t, int(appended.Load()), 12)
assert.Equal(t, len(result.AppendResults), 3) assert.Equal(t, len(result.AppendResults), 3)
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
return done.Load() == 4 return len(done.Collect()) == 2
}, 30*time.Second, 10*time.Millisecond) }, 30*time.Second, 10*time.Millisecond)
// TODO: error path. // Test broadcast with a already exist resource key.
for {
var err error
_, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"}, message.NewCollectionNameResourceKey("c7")))
if err != nil {
assert.True(t, status.AsStreamingError(err).IsResourceAcquired())
break
}
}
// Test watch here.
w, err := bc.NewWatcher()
assert.NoError(t, err)
// Test a resource key that not exist.
assertResourceEventOK(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c5")))
assertResourceEventOK(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c5")))
// Test a resource key that already ack all.
assertResourceEventOK(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
assertResourceEventOK(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
// Test a resource key that partially ack.
assertResourceEventOK(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c2")))
assertResourceEventNotReady(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c2")))
// Test a resource key that not ack.
readyCh := assertResourceEventUntilReady(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c2")))
ack(bc, 2, "v1")
<-readyCh
// Test a resource key that not ack.
assertResourceEventNotReady(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c3")))
assertResourceEventNotReady(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c3")))
readyCh1 := assertResourceEventUntilReady(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c3")))
readyCh2 := assertResourceEventUntilReady(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c3")))
ack(bc, 7, "v1")
<-readyCh1
select {
case <-readyCh2:
assert.Fail(t, "should not ready")
case <-time.After(20 * time.Millisecond):
}
ack(bc, 7, "v2")
ack(bc, 7, "v3")
<-readyCh2
w2, _ := bc.NewWatcher()
w2.Close() // Close by watcher itself.
_, ok := <-w2.EventChan()
assert.False(t, ok)
bc.Close() bc.Close()
w.Close() // Close by broadcaster.
result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"})) result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"}))
assert.Error(t, err) assert.Error(t, err)
assert.Nil(t, result) assert.Nil(t, result)
err = bc.Ack(context.Background(), types.BroadcastAckRequest{BroadcastID: 3, VChannel: "v1"})
assert.Error(t, err)
ww, err := bc.NewWatcher()
assert.Error(t, err)
assert.Nil(t, ww)
}
func assertResourceEventOK(t *testing.T, w Watcher, ev1 *message.BroadcastEvent) {
w.ObserveResourceKeyEvent(context.Background(), ev1)
ev2 := <-w.EventChan()
assert.True(t, proto.Equal(ev1, ev2))
}
func assertResourceEventNotReady(t *testing.T, w Watcher, ev1 *message.BroadcastEvent) {
w.ObserveResourceKeyEvent(context.Background(), ev1)
select {
case ev2 := <-w.EventChan():
t.Errorf("should not receive event, %+v", ev2)
case <-time.After(10 * time.Millisecond):
return
}
}
func assertResourceEventUntilReady(t *testing.T, w Watcher, ev1 *message.BroadcastEvent) <-chan struct{} {
w.ObserveResourceKeyEvent(context.Background(), ev1)
done := make(chan struct{})
go func() {
ev2 := <-w.EventChan()
assert.True(t, proto.Equal(ev1, ev2))
close(done)
}()
return done
}
func ack(bc Broadcaster, broadcastID uint64, vchannel string) {
for {
if err := bc.Ack(context.Background(), types.BroadcastAckRequest{
BroadcastID: broadcastID,
VChannel: vchannel,
}); err == nil {
break
}
}
}
func asyncAck(wg *sync.WaitGroup, bc Broadcaster, broadcastID uint64, vchannel string) {
wg.Add(1)
go func() {
defer wg.Done()
ack(bc, broadcastID, vchannel)
}()
} }
func createOpeartor(t *testing.T) (*syncutil.Future[AppendOperator], *atomic.Int64) { func createOpeartor(t *testing.T) (*syncutil.Future[AppendOperator], *atomic.Int64) {
@ -122,11 +259,11 @@ func createOpeartor(t *testing.T) (*syncutil.Future[AppendOperator], *atomic.Int
return fOperator, appended return fOperator, appended
} }
func createNewBroadcastMsg(vchannels []string) message.BroadcastMutableMessage { func createNewBroadcastMsg(vchannels []string, rks ...message.ResourceKey) message.BroadcastMutableMessage {
msg, err := message.NewDropCollectionMessageBuilderV1(). msg, err := message.NewDropCollectionMessageBuilderV1().
WithHeader(&messagespb.DropCollectionMessageHeader{}). WithHeader(&messagespb.DropCollectionMessageHeader{}).
WithBody(&msgpb.DropCollectionRequest{}). WithBody(&msgpb.DropCollectionRequest{}).
WithBroadcast(vchannels). WithBroadcast(vchannels, rks...).
BuildBroadcast() BuildBroadcast()
if err != nil { if err != nil {
panic(err) panic(err)
@ -134,14 +271,29 @@ func createNewBroadcastMsg(vchannels []string) message.BroadcastMutableMessage {
return msg return msg
} }
func createNewBroadcastTask(taskID int64, vchannels []string) *streamingpb.BroadcastTask { func createNewBroadcastTask(broadcastID uint64, vchannels []string, rks ...message.ResourceKey) *streamingpb.BroadcastTask {
msg := createNewBroadcastMsg(vchannels) msg := createNewBroadcastMsg(vchannels, rks...).WithBroadcastID(broadcastID)
return &streamingpb.BroadcastTask{ return &streamingpb.BroadcastTask{
TaskId: taskID,
Message: &messagespb.Message{ Message: &messagespb.Message{
Payload: msg.Payload(), Payload: msg.Payload(),
Properties: msg.Properties().ToRawMap(), Properties: msg.Properties().ToRawMap(),
}, },
State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING,
AckedVchannelBitmap: make([]byte, len(vchannels)),
}
}
func createNewWaitAckBroadcastTaskFromMessage(
msg message.BroadcastMutableMessage,
state streamingpb.BroadcastTaskState,
bitmap []byte,
) *streamingpb.BroadcastTask {
return &streamingpb.BroadcastTask{
Message: &messagespb.Message{
Payload: msg.Payload(),
Properties: msg.Properties().ToRawMap(),
},
State: state,
AckedVchannelBitmap: bitmap,
} }
} }

View File

@ -7,9 +7,6 @@ import (
"github.com/cockroachdb/errors" "github.com/cockroachdb/errors"
"go.uber.org/zap" "go.uber.org/zap"
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/syncutil"
@ -18,13 +15,11 @@ import (
var errBroadcastTaskIsNotDone = errors.New("broadcast task is not done") var errBroadcastTaskIsNotDone = errors.New("broadcast task is not done")
// newTask creates a new task // newPendingBroadcastTask creates a new pendingBroadcastTask.
func newTask(task *streamingpb.BroadcastTask, logger *log.MLogger) *broadcastTask { func newPendingBroadcastTask(task *broadcastTask) *pendingBroadcastTask {
bt := message.NewBroadcastMutableMessage(task.Message.Payload, task.Message.Properties) msgs := task.PendingBroadcastMessages()
msgs := bt.SplitIntoMutableMessage() return &pendingBroadcastTask{
return &broadcastTask{ broadcastTask: task,
logger: logger.With(zap.Int64("taskID", task.TaskId), zap.Int("broadcastTotal", len(msgs))),
task: task,
pendingMessages: msgs, pendingMessages: msgs,
appendResult: make(map[string]*types.AppendResult, len(msgs)), appendResult: make(map[string]*types.AppendResult, len(msgs)),
future: syncutil.NewFuture[*types.BroadcastAppendResult](), future: syncutil.NewFuture[*types.BroadcastAppendResult](),
@ -39,10 +34,9 @@ func newTask(task *streamingpb.BroadcastTask, logger *log.MLogger) *broadcastTas
} }
} }
// broadcastTask is the task for broadcasting messages. // pendingBroadcastTask is a task that is pending to be broadcasted.
type broadcastTask struct { type pendingBroadcastTask struct {
logger *log.MLogger *broadcastTask
task *streamingpb.BroadcastTask
pendingMessages []message.MutableMessage pendingMessages []message.MutableMessage
appendResult map[string]*types.AppendResult appendResult map[string]*types.AppendResult
future *syncutil.Future[*types.BroadcastAppendResult] future *syncutil.Future[*types.BroadcastAppendResult]
@ -52,14 +46,20 @@ type broadcastTask struct {
// Execute reexecute the task, return nil if the task is done, otherwise not done. // Execute reexecute the task, return nil if the task is done, otherwise not done.
// Execute can be repeated called until the task is done. // Execute can be repeated called until the task is done.
// Same semantics as the `Poll` operation in eventloop. // Same semantics as the `Poll` operation in eventloop.
func (b *broadcastTask) Execute(ctx context.Context, operator AppendOperator) error { func (b *pendingBroadcastTask) Execute(ctx context.Context, operator AppendOperator) error {
if err := b.broadcastTask.InitializeRecovery(ctx); err != nil {
b.Logger().Warn("broadcast task initialize recovery failed", zap.Error(err))
b.UpdateInstantWithNextBackOff()
return err
}
if len(b.pendingMessages) > 0 { if len(b.pendingMessages) > 0 {
b.logger.Debug("broadcast task is polling to make sent...", zap.Int("pendingMessages", len(b.pendingMessages))) b.Logger().Debug("broadcast task is polling to make sent...", zap.Int("pendingMessages", len(b.pendingMessages)))
resps := operator.AppendMessages(ctx, b.pendingMessages...) resps := operator.AppendMessages(ctx, b.pendingMessages...)
newPendings := make([]message.MutableMessage, 0) newPendings := make([]message.MutableMessage, 0)
for idx, resp := range resps.Responses { for idx, resp := range resps.Responses {
if resp.Error != nil { if resp.Error != nil {
b.logger.Warn("broadcast task append message failed", zap.Int("idx", idx), zap.Error(resp.Error)) b.Logger().Warn("broadcast task append message failed", zap.Int("idx", idx), zap.Error(resp.Error))
newPendings = append(newPendings, b.pendingMessages[idx]) newPendings = append(newPendings, b.pendingMessages[idx])
continue continue
} }
@ -67,15 +67,15 @@ func (b *broadcastTask) Execute(ctx context.Context, operator AppendOperator) er
} }
b.pendingMessages = newPendings b.pendingMessages = newPendings
if len(newPendings) == 0 { if len(newPendings) == 0 {
b.future.Set(&types.BroadcastAppendResult{AppendResults: b.appendResult}) b.future.Set(&types.BroadcastAppendResult{
BroadcastID: b.header.BroadcastID,
AppendResults: b.appendResult,
})
} }
b.logger.Info("broadcast task make a new broadcast done", zap.Int("backoffRetryMessages", len(b.pendingMessages))) b.Logger().Info("broadcast task make a new broadcast done", zap.Int("backoffRetryMessages", len(b.pendingMessages)))
} }
if len(b.pendingMessages) == 0 { if len(b.pendingMessages) == 0 {
// There's no more pending message, mark the task as done. if err := b.broadcastTask.BroadcastDone(ctx); err != nil {
b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE
if err := resource.Resource().StreamingCatalog().SaveBroadcastTask(ctx, b.task); err != nil {
b.logger.Warn("save broadcast task failed", zap.Error(err))
b.UpdateInstantWithNextBackOff() b.UpdateInstantWithNextBackOff()
return err return err
} }
@ -86,34 +86,35 @@ func (b *broadcastTask) Execute(ctx context.Context, operator AppendOperator) er
} }
// BlockUntilTaskDone blocks until the task is done. // BlockUntilTaskDone blocks until the task is done.
func (b *broadcastTask) BlockUntilTaskDone(ctx context.Context) (*types.BroadcastAppendResult, error) { func (b *pendingBroadcastTask) BlockUntilTaskDone(ctx context.Context) (*types.BroadcastAppendResult, error) {
return b.future.GetWithContext(ctx) return b.future.GetWithContext(ctx)
} }
type broadcastTaskArray []*broadcastTask // pendingBroadcastTaskArray is a heap of pendingBroadcastTask.
type pendingBroadcastTaskArray []*pendingBroadcastTask
// Len returns the length of the heap. // Len returns the length of the heap.
func (h broadcastTaskArray) Len() int { func (h pendingBroadcastTaskArray) Len() int {
return len(h) return len(h)
} }
// Less returns true if the element at index i is less than the element at index j. // Less returns true if the element at index i is less than the element at index j.
func (h broadcastTaskArray) Less(i, j int) bool { func (h pendingBroadcastTaskArray) Less(i, j int) bool {
return h[i].NextInstant().Before(h[j].NextInstant()) return h[i].NextInstant().Before(h[j].NextInstant())
} }
// Swap swaps the elements at indexes i and j. // Swap swaps the elements at indexes i and j.
func (h broadcastTaskArray) Swap(i, j int) { h[i], h[j] = h[j], h[i] } func (h pendingBroadcastTaskArray) Swap(i, j int) { h[i], h[j] = h[j], h[i] }
// Push pushes the last one at len. // Push pushes the last one at len.
func (h *broadcastTaskArray) Push(x interface{}) { func (h *pendingBroadcastTaskArray) Push(x interface{}) {
// Push and Pop use pointer receivers because they modify the slice's length, // Push and Pop use pointer receivers because they modify the slice's length,
// not just its contents. // not just its contents.
*h = append(*h, x.(*broadcastTask)) *h = append(*h, x.(*pendingBroadcastTask))
} }
// Pop pop the last one at len. // Pop pop the last one at len.
func (h *broadcastTaskArray) Pop() interface{} { func (h *pendingBroadcastTaskArray) Pop() interface{} {
old := *h old := *h
n := len(old) n := len(old)
x := old[n-1] x := old[n-1]
@ -123,6 +124,6 @@ func (h *broadcastTaskArray) Pop() interface{} {
// Peek returns the element at the top of the heap. // Peek returns the element at the top of the heap.
// Panics if the heap is empty. // Panics if the heap is empty.
func (h *broadcastTaskArray) Peek() interface{} { func (h *pendingBroadcastTaskArray) Peek() interface{} {
return (*h)[0] return (*h)[0]
} }

View File

@ -0,0 +1,103 @@
package broadcaster
import (
"context"
"go.uber.org/zap"
"github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/util/syncutil"
)
// newWatcher creates a new watcher.
func newWatcher(broadcaster *broadcasterImpl) *watcherImpl {
w := &watcherImpl{
watcherBGNotifier: syncutil.NewAsyncTaskNotifier[struct{}](),
pendingEvents: make(map[string]*message.BroadcastEvent, 0),
broadcasterImpl: broadcaster,
version: 0,
input: make(chan *message.BroadcastEvent),
output: make(chan *message.BroadcastEvent),
}
go w.execute()
return w
}
// watcherImpl implement the Watcher interface.
type watcherImpl struct {
watcherBGNotifier *syncutil.AsyncTaskNotifier[struct{}]
pendingEvents map[string]*message.BroadcastEvent
*broadcasterImpl
version int
input chan *message.BroadcastEvent
output chan *message.BroadcastEvent
}
func (w *watcherImpl) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
select {
case w.input <- ev:
return nil
case <-w.backgroundTaskNotifier.Context().Done():
return w.backgroundTaskNotifier.Context().Err()
case <-w.watcherBGNotifier.Context().Done():
return w.watcherBGNotifier.Context().Err()
case <-ctx.Done():
return ctx.Err()
}
}
func (w *watcherImpl) EventChan() <-chan *message.BroadcastEvent {
return w.output
}
func (w *watcherImpl) execute() {
defer func() {
close(w.output)
w.watcherBGNotifier.Finish(struct{}{})
}()
for {
ch := w.manager.WatchAtVersion(w.version)
select {
case <-w.backgroundTaskNotifier.Context().Done():
w.Logger().Info("watcher is exit because of broadcaseter is closing", zap.Int("version", w.version))
return
case <-w.watcherBGNotifier.Context().Done():
w.Logger().Info("watcher is exit because of watcher itself is closing", zap.Int("version", w.version))
return
case <-ch:
w.update()
case ev := <-w.input:
w.pendingEvents[message.UniqueKeyOfBroadcastEvent(ev)] = ev
w.update()
}
}
}
func (w *watcherImpl) update() {
w.version = w.manager.CurrentVersion()
newPendingEvent := make(map[string]*message.BroadcastEvent, len(w.pendingEvents))
for key, pendingEvent := range w.pendingEvents {
switch ev := pendingEvent.Event.(type) {
case *messagespb.BroadcastEvent_ResourceKeyAckAll:
task, ok := w.manager.GetBroadcastTaskByResourceKey(message.NewResourceKeyFromProto(ev.ResourceKeyAckAll.ResourceKey))
if !ok || task.IsAllAcked() {
w.output <- pendingEvent
continue
}
case *messagespb.BroadcastEvent_ResourceKeyAckOne:
task, ok := w.manager.GetBroadcastTaskByResourceKey(message.NewResourceKeyFromProto(ev.ResourceKeyAckOne.ResourceKey))
if !ok || task.IsAcked() {
w.output <- pendingEvent
continue
}
}
newPendingEvent[key] = pendingEvent
}
w.pendingEvents = newPendingEvent
}
func (w *watcherImpl) Close() {
w.watcherBGNotifier.Cancel()
w.watcherBGNotifier.BlockUntilFinish()
}

View File

@ -4,8 +4,10 @@ import (
"context" "context"
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster" "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster"
"github.com/milvus-io/milvus/internal/streamingcoord/server/service/broadcast"
"github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/syncutil"
) )
@ -32,7 +34,7 @@ func (s *broadcastServceImpl) Broadcast(ctx context.Context, req *streamingpb.Br
if err != nil { if err != nil {
return nil, err return nil, err
} }
results, err := broadcaster.Broadcast(ctx, message.NewBroadcastMutableMessage(req.Message.Payload, req.Message.Properties)) results, err := broadcaster.Broadcast(ctx, message.NewBroadcastMutableMessageBeforeAppend(req.Message.Payload, req.Message.Properties))
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -40,5 +42,38 @@ func (s *broadcastServceImpl) Broadcast(ctx context.Context, req *streamingpb.Br
for vchannel, result := range results.AppendResults { for vchannel, result := range results.AppendResults {
protoResult[vchannel] = result.IntoProto() protoResult[vchannel] = result.IntoProto()
} }
return &streamingpb.BroadcastResponse{Results: protoResult}, nil return &streamingpb.BroadcastResponse{
BroadcastId: results.BroadcastID,
Results: protoResult,
}, nil
}
// Ack acknowledges the message at the specified vchannel.
func (s *broadcastServceImpl) Ack(ctx context.Context, req *streamingpb.BroadcastAckRequest) (*streamingpb.BroadcastAckResponse, error) {
broadcaster, err := s.broadcaster.GetWithContext(ctx)
if err != nil {
return nil, err
}
if err := broadcaster.Ack(ctx, types.BroadcastAckRequest{
BroadcastID: req.BroadcastId,
VChannel: req.Vchannel,
}); err != nil {
return nil, err
}
return &streamingpb.BroadcastAckResponse{}, nil
}
func (s *broadcastServceImpl) Watch(svr streamingpb.StreamingCoordBroadcastService_WatchServer) error {
broadcaster, err := s.broadcaster.GetWithContext(svr.Context())
if err != nil {
return err
}
watcher, err := broadcaster.NewWatcher()
if err != nil {
return err
}
defer watcher.Close()
server := broadcast.NewBroadcastWatchServer(watcher, svr)
return server.Execute()
} }

View File

@ -0,0 +1,30 @@
package broadcast
import (
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
type broadcastWatchGrpcServerHelper struct {
streamingpb.StreamingCoordBroadcastService_WatchServer
}
// SendResourceKeyEvent sends the resource key event to client.
func (h *broadcastWatchGrpcServerHelper) SendResourceKeyEvent(ev *message.BroadcastEvent) error {
return h.Send(&streamingpb.BroadcastWatchResponse{
Response: &streamingpb.BroadcastWatchResponse_EventDone{
EventDone: &streamingpb.BroadcastEventWatchResponse{
Event: ev,
},
},
})
}
// SendCloseResponse sends the close response to client.
func (h *broadcastWatchGrpcServerHelper) SendCloseResponse() error {
return h.Send(&streamingpb.BroadcastWatchResponse{
Response: &streamingpb.BroadcastWatchResponse_Close{
Close: &streamingpb.CloseBroadcastWatchResponse{},
},
})
}

View File

@ -0,0 +1,117 @@
package broadcast
import (
"context"
"io"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster"
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
)
var errClosedByUser = errors.New("closed by user")
func NewBroadcastWatchServer(
w broadcaster.Watcher,
streamServer streamingpb.StreamingCoordBroadcastService_WatchServer,
) *BroadcastWatchServer {
ctx, cancel := context.WithCancelCause(streamServer.Context())
s := &BroadcastWatchServer{
ctx: ctx,
cancel: cancel,
w: w,
streamServer: broadcastWatchGrpcServerHelper{
streamServer,
},
}
s.SetLogger(resource.Resource().Logger().With(log.FieldComponent("broadcast-watch-server")))
return s
}
type BroadcastWatchServer struct {
log.Binder
ctx context.Context
cancel context.CancelCauseFunc
w broadcaster.Watcher
streamServer broadcastWatchGrpcServerHelper
}
func (s *BroadcastWatchServer) 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.
// 1. close by client.
// 2. close by server context cancel by return of outside Execute.
_ = s.recvLoop()
}()
// Start a send loop on current main goroutine.
// the loop will be blocked until:
// 1. the stream is broken.
// 2. recv arm recv closed and all response is sent.
return s.sendLoop()
}
// recvLoop receives the message from client.
func (s *BroadcastWatchServer) recvLoop() (err error) {
defer func() {
if err != nil {
s.cancel(err)
s.Logger().Warn("recv arm of stream closed by unexpected error", zap.Error(err))
return
}
s.cancel(errClosedByUser)
s.Logger().Info("recv arm of stream closed")
}()
for {
req, err := s.streamServer.Recv()
if err == io.EOF {
return nil
}
if err != nil {
return err
}
switch req := req.Command.(type) {
case *streamingpb.BroadcastWatchRequest_CreateEventWatch:
// Add new incoming resource key int watcher.
s.w.ObserveResourceKeyEvent(s.streamServer.Context(), req.CreateEventWatch.Event)
case *streamingpb.BroadcastWatchRequest_Close:
// Ignore the command, the stream will be closed by client with io.EOF
default:
s.Logger().Warn("unknown command type ignored", zap.Any("command", req))
}
}
}
// sendLoop sends the message to client.
func (s *BroadcastWatchServer) sendLoop() (err error) {
defer func() {
if err != nil {
s.Logger().Warn("send arm of stream closed by unexpected error", zap.Error(err))
return
}
s.Logger().Info("send arm of stream closed")
}()
for {
select {
case ev, ok := <-s.w.EventChan():
if !ok {
return errors.New("watcher is closed")
}
if err := s.streamServer.SendResourceKeyEvent(ev); err != nil {
return err
}
case <-s.ctx.Done():
err := context.Cause(s.ctx)
if errors.Is(err, errClosedByUser) {
return s.streamServer.SendCloseResponse()
}
return err
}
}
}

View File

@ -0,0 +1,76 @@
package broadcast
import (
"context"
"io"
"testing"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_broadcaster"
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/pkg/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
func TestBroadcastWatch(t *testing.T) {
resource.InitForTest()
w := mock_broadcaster.NewMockWatcher(t)
input := make(chan *message.BroadcastEvent, 5)
output := make(chan *message.BroadcastEvent, 5)
w.EXPECT().ObserveResourceKeyEvent(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, ev *messagespb.BroadcastEvent) error {
output <- ev
return nil
})
w.EXPECT().EventChan().Return(output)
streamServer := mock_streamingpb.NewMockStreamingCoordBroadcastService_WatchServer(t)
streamServer.EXPECT().Context().Return(context.Background())
closed := false
streamServer.EXPECT().Recv().RunAndReturn(func() (*streamingpb.BroadcastWatchRequest, error) {
if closed {
return nil, io.EOF
}
ev, ok := <-input
if !ok {
closed = true
return &streamingpb.BroadcastWatchRequest{
Command: &streamingpb.BroadcastWatchRequest_Close{
Close: &streamingpb.CloseBroadcastWatchRequest{},
},
}, nil
}
return &streamingpb.BroadcastWatchRequest{
Command: &streamingpb.BroadcastWatchRequest_CreateEventWatch{
CreateEventWatch: &streamingpb.BroadcastCreateEventWatchRequest{
Event: ev,
},
},
}, nil
})
streamOutput := make(chan *message.BroadcastEvent, 5)
streamServer.EXPECT().Send(mock.Anything).RunAndReturn(func(bwr *streamingpb.BroadcastWatchResponse) error {
if bwr.GetEventDone() != nil {
streamOutput <- bwr.GetEventDone().Event
}
return nil
})
s := NewBroadcastWatchServer(w, streamServer)
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1"))
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c2"))
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c3"))
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c4"))
done := make(chan struct{})
go func() {
s.Execute()
close(done)
}()
for i := 0; i < 4; i++ {
<-streamOutput
}
close(input)
<-done
}

View File

@ -45,6 +45,8 @@ func (h *discoverGrpcServerHelper) SendFullAssignment(v typeutil.VersionInt64Pai
// SendCloseResponse sends the close response to client. // SendCloseResponse sends the close response to client.
func (h *discoverGrpcServerHelper) SendCloseResponse() error { func (h *discoverGrpcServerHelper) SendCloseResponse() error {
return h.Send(&streamingpb.AssignmentDiscoverResponse{ return h.Send(&streamingpb.AssignmentDiscoverResponse{
Response: &streamingpb.AssignmentDiscoverResponse_Close{}, Response: &streamingpb.AssignmentDiscoverResponse_Close{
Close: &streamingpb.CloseAssignmentDiscoverResponse{},
},
}) })
} }

View File

@ -230,10 +230,10 @@ func (p *producerImpl) sendLoop() (err error) {
} else { } else {
p.logger.Info("send arm of stream closed") p.logger.Info("send arm of stream closed")
} }
close(p.sendExitCh)
if err := p.grpcStreamClient.CloseSend(); err != nil { if err := p.grpcStreamClient.CloseSend(); err != nil {
p.logger.Warn("failed to close send", zap.Error(err)) p.logger.Warn("failed to close send", zap.Error(err))
} }
close(p.sendExitCh)
}() }()
for { for {
@ -265,16 +265,15 @@ func (p *producerImpl) recvLoop() (err error) {
defer func() { defer func() {
if err != nil { if err != nil {
p.logger.Warn("recv arm of stream closed by unexpected error", zap.Error(err)) p.logger.Warn("recv arm of stream closed by unexpected error", zap.Error(err))
return } else {
}
p.logger.Info("recv arm of stream closed") p.logger.Info("recv arm of stream closed")
}
close(p.recvExitCh) close(p.recvExitCh)
}() }()
for { for {
resp, err := p.grpcStreamClient.Recv() resp, err := p.grpcStreamClient.Recv()
if errors.Is(err, io.EOF) { if errors.Is(err, io.EOF) {
p.logger.Debug("stream closed successful")
return nil return nil
} }
if err != nil { if err != nil {

View File

@ -176,7 +176,7 @@ func (p *ProduceServer) handleProduce(req *streamingpb.ProduceMessageRequest) {
p.appendWG.Add(1) p.appendWG.Add(1)
p.logger.Debug("recv produce message from client", zap.Int64("requestID", req.RequestId)) p.logger.Debug("recv produce message from client", zap.Int64("requestID", req.RequestId))
// Update metrics. // Update metrics.
msg := message.NewMutableMessage(req.GetMessage().GetPayload(), req.GetMessage().GetProperties()) msg := message.NewMutableMessageBeforeAppend(req.GetMessage().GetPayload(), req.GetMessage().GetProperties())
metricsGuard := p.metrics.StartProduce() metricsGuard := p.metrics.StartProduce()
if err := p.validateMessage(msg); err != nil { if err := p.validateMessage(msg); err != nil {
p.logger.Warn("produce message validation failed", zap.Int64("requestID", req.RequestId), zap.Error(err)) p.logger.Warn("produce message validation failed", zap.Int64("requestID", req.RequestId), zap.Error(err))

View File

@ -55,6 +55,11 @@ func (e *StreamingError) IsTxnUnavilable() bool {
e.Code == streamingpb.StreamingCode_STREAMING_CODE_INVALID_TRANSACTION_STATE e.Code == streamingpb.StreamingCode_STREAMING_CODE_INVALID_TRANSACTION_STATE
} }
// IsResourceAcquired returns true if the resource is acquired.
func (e *StreamingError) IsResourceAcquired() bool {
return e.Code == streamingpb.StreamingCode_STREAMING_CODE_RESOURCE_ACQUIRED
}
// NewOnShutdownError creates a new StreamingError with code STREAMING_CODE_ON_SHUTDOWN. // NewOnShutdownError creates a new StreamingError with code STREAMING_CODE_ON_SHUTDOWN.
func NewOnShutdownError(format string, args ...interface{}) *StreamingError { func NewOnShutdownError(format string, args ...interface{}) *StreamingError {
return New(streamingpb.StreamingCode_STREAMING_CODE_ON_SHUTDOWN, format, args...) return New(streamingpb.StreamingCode_STREAMING_CODE_ON_SHUTDOWN, format, args...)
@ -116,6 +121,11 @@ func NewUnrecoverableError(format string, args ...interface{}) *StreamingError {
return New(streamingpb.StreamingCode_STREAMING_CODE_UNRECOVERABLE, format, args...) return New(streamingpb.StreamingCode_STREAMING_CODE_UNRECOVERABLE, format, args...)
} }
// NewResourceAcquired creates a new StreamingError with code STREAMING_CODE_RESOURCE_ACQUIRED.
func NewResourceAcquired(format string, args ...interface{}) *StreamingError {
return New(streamingpb.StreamingCode_STREAMING_CODE_RESOURCE_ACQUIRED, format, args...)
}
// New creates a new StreamingError with the given code and cause. // New creates a new StreamingError with the given code and cause.
func New(code streamingpb.StreamingCode, format string, args ...interface{}) *StreamingError { func New(code streamingpb.StreamingCode, format string, args ...interface{}) *StreamingError {
if len(args) == 0 { if len(args) == 0 {

View File

@ -56,4 +56,10 @@ func TestStreamingError(t *testing.T) {
assert.False(t, streamingErr.IsWrongStreamingNode()) assert.False(t, streamingErr.IsWrongStreamingNode())
pbErr = streamingErr.AsPBError() pbErr = streamingErr.AsPBError()
assert.Equal(t, streamingpb.StreamingCode_STREAMING_CODE_ON_SHUTDOWN, pbErr.Code) assert.Equal(t, streamingpb.StreamingCode_STREAMING_CODE_ON_SHUTDOWN, pbErr.Code)
streamingErr = NewResourceAcquired("test, %d", 1)
assert.Contains(t, streamingErr.Error(), "code: STREAMING_CODE_RESOURCE_ACQUIRED, cause: test, 1")
assert.False(t, streamingErr.IsWrongStreamingNode())
pbErr = streamingErr.AsPBError()
assert.Equal(t, streamingpb.StreamingCode_STREAMING_CODE_RESOURCE_ACQUIRED, pbErr.Code)
} }

View File

@ -39,3 +39,6 @@ packages:
StreamingNodeHandlerServiceClient: StreamingNodeHandlerServiceClient:
StreamingNodeHandlerService_ConsumeClient: StreamingNodeHandlerService_ConsumeClient:
StreamingNodeHandlerService_ProduceClient: StreamingNodeHandlerService_ProduceClient:
StreamingCoordBroadcastService_WatchServer:
StreamingCoordBroadcastServiceClient:
StreamingCoordBroadcastService_WatchClient:

37
pkg/log/with_logger.go Normal file
View File

@ -0,0 +1,37 @@
package log
var (
_ WithLogger = &Binder{}
_ LoggerBinder = &Binder{}
)
// WithLogger is an interface to help access local logger.
type WithLogger interface {
Logger() *MLogger
}
// LoggerBinder is an interface to help set logger.
type LoggerBinder interface {
SetLogger(logger *MLogger)
}
// Binder is a embedding type to help access local logger.
type Binder struct {
logger *MLogger
}
// SetLogger sets logger to Binder.
func (w *Binder) SetLogger(logger *MLogger) {
if w.logger != nil {
panic("logger already set")
}
w.logger = logger
}
// Logger returns the logger of Binder.
func (w *Binder) Logger() *MLogger {
if w.logger == nil {
return With()
}
return w.logger
}

View File

@ -0,0 +1,261 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_streamingpb
import (
context "context"
grpc "google.golang.org/grpc"
mock "github.com/stretchr/testify/mock"
streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb"
)
// MockStreamingCoordBroadcastServiceClient is an autogenerated mock type for the StreamingCoordBroadcastServiceClient type
type MockStreamingCoordBroadcastServiceClient struct {
mock.Mock
}
type MockStreamingCoordBroadcastServiceClient_Expecter struct {
mock *mock.Mock
}
func (_m *MockStreamingCoordBroadcastServiceClient) EXPECT() *MockStreamingCoordBroadcastServiceClient_Expecter {
return &MockStreamingCoordBroadcastServiceClient_Expecter{mock: &_m.Mock}
}
// Ack provides a mock function with given fields: ctx, in, opts
func (_m *MockStreamingCoordBroadcastServiceClient) Ack(ctx context.Context, in *streamingpb.BroadcastAckRequest, opts ...grpc.CallOption) (*streamingpb.BroadcastAckResponse, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx, in)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for Ack")
}
var r0 *streamingpb.BroadcastAckResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastAckRequest, ...grpc.CallOption) (*streamingpb.BroadcastAckResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastAckRequest, ...grpc.CallOption) *streamingpb.BroadcastAckResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*streamingpb.BroadcastAckResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *streamingpb.BroadcastAckRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockStreamingCoordBroadcastServiceClient_Ack_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Ack'
type MockStreamingCoordBroadcastServiceClient_Ack_Call struct {
*mock.Call
}
// Ack is a helper method to define mock.On call
// - ctx context.Context
// - in *streamingpb.BroadcastAckRequest
// - opts ...grpc.CallOption
func (_e *MockStreamingCoordBroadcastServiceClient_Expecter) Ack(ctx interface{}, in interface{}, opts ...interface{}) *MockStreamingCoordBroadcastServiceClient_Ack_Call {
return &MockStreamingCoordBroadcastServiceClient_Ack_Call{Call: _e.mock.On("Ack",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockStreamingCoordBroadcastServiceClient_Ack_Call) Run(run func(ctx context.Context, in *streamingpb.BroadcastAckRequest, opts ...grpc.CallOption)) *MockStreamingCoordBroadcastServiceClient_Ack_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), args[1].(*streamingpb.BroadcastAckRequest), variadicArgs...)
})
return _c
}
func (_c *MockStreamingCoordBroadcastServiceClient_Ack_Call) Return(_a0 *streamingpb.BroadcastAckResponse, _a1 error) *MockStreamingCoordBroadcastServiceClient_Ack_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockStreamingCoordBroadcastServiceClient_Ack_Call) RunAndReturn(run func(context.Context, *streamingpb.BroadcastAckRequest, ...grpc.CallOption) (*streamingpb.BroadcastAckResponse, error)) *MockStreamingCoordBroadcastServiceClient_Ack_Call {
_c.Call.Return(run)
return _c
}
// Broadcast provides a mock function with given fields: ctx, in, opts
func (_m *MockStreamingCoordBroadcastServiceClient) Broadcast(ctx context.Context, in *streamingpb.BroadcastRequest, opts ...grpc.CallOption) (*streamingpb.BroadcastResponse, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx, in)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for Broadcast")
}
var r0 *streamingpb.BroadcastResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastRequest, ...grpc.CallOption) (*streamingpb.BroadcastResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastRequest, ...grpc.CallOption) *streamingpb.BroadcastResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*streamingpb.BroadcastResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *streamingpb.BroadcastRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockStreamingCoordBroadcastServiceClient_Broadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Broadcast'
type MockStreamingCoordBroadcastServiceClient_Broadcast_Call struct {
*mock.Call
}
// Broadcast is a helper method to define mock.On call
// - ctx context.Context
// - in *streamingpb.BroadcastRequest
// - opts ...grpc.CallOption
func (_e *MockStreamingCoordBroadcastServiceClient_Expecter) Broadcast(ctx interface{}, in interface{}, opts ...interface{}) *MockStreamingCoordBroadcastServiceClient_Broadcast_Call {
return &MockStreamingCoordBroadcastServiceClient_Broadcast_Call{Call: _e.mock.On("Broadcast",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockStreamingCoordBroadcastServiceClient_Broadcast_Call) Run(run func(ctx context.Context, in *streamingpb.BroadcastRequest, opts ...grpc.CallOption)) *MockStreamingCoordBroadcastServiceClient_Broadcast_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), args[1].(*streamingpb.BroadcastRequest), variadicArgs...)
})
return _c
}
func (_c *MockStreamingCoordBroadcastServiceClient_Broadcast_Call) Return(_a0 *streamingpb.BroadcastResponse, _a1 error) *MockStreamingCoordBroadcastServiceClient_Broadcast_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockStreamingCoordBroadcastServiceClient_Broadcast_Call) RunAndReturn(run func(context.Context, *streamingpb.BroadcastRequest, ...grpc.CallOption) (*streamingpb.BroadcastResponse, error)) *MockStreamingCoordBroadcastServiceClient_Broadcast_Call {
_c.Call.Return(run)
return _c
}
// Watch provides a mock function with given fields: ctx, opts
func (_m *MockStreamingCoordBroadcastServiceClient) Watch(ctx context.Context, opts ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for Watch")
}
var r0 streamingpb.StreamingCoordBroadcastService_WatchClient
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error)); ok {
return rf(ctx, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, ...grpc.CallOption) streamingpb.StreamingCoordBroadcastService_WatchClient); ok {
r0 = rf(ctx, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(streamingpb.StreamingCoordBroadcastService_WatchClient)
}
}
if rf, ok := ret.Get(1).(func(context.Context, ...grpc.CallOption) error); ok {
r1 = rf(ctx, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockStreamingCoordBroadcastServiceClient_Watch_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Watch'
type MockStreamingCoordBroadcastServiceClient_Watch_Call struct {
*mock.Call
}
// Watch is a helper method to define mock.On call
// - ctx context.Context
// - opts ...grpc.CallOption
func (_e *MockStreamingCoordBroadcastServiceClient_Expecter) Watch(ctx interface{}, opts ...interface{}) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
return &MockStreamingCoordBroadcastServiceClient_Watch_Call{Call: _e.mock.On("Watch",
append([]interface{}{ctx}, opts...)...)}
}
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) Run(run func(ctx context.Context, opts ...grpc.CallOption)) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-1)
for i, a := range args[1:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), variadicArgs...)
})
return _c
}
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) Return(_a0 streamingpb.StreamingCoordBroadcastService_WatchClient, _a1 error) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) RunAndReturn(run func(context.Context, ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error)) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
_c.Call.Return(run)
return _c
}
// NewMockStreamingCoordBroadcastServiceClient creates a new instance of MockStreamingCoordBroadcastServiceClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockStreamingCoordBroadcastServiceClient(t interface {
mock.TestingT
Cleanup(func())
}) *MockStreamingCoordBroadcastServiceClient {
mock := &MockStreamingCoordBroadcastServiceClient{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -0,0 +1,430 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_streamingpb
import (
context "context"
mock "github.com/stretchr/testify/mock"
metadata "google.golang.org/grpc/metadata"
streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb"
)
// MockStreamingCoordBroadcastService_WatchClient is an autogenerated mock type for the StreamingCoordBroadcastService_WatchClient type
type MockStreamingCoordBroadcastService_WatchClient struct {
mock.Mock
}
type MockStreamingCoordBroadcastService_WatchClient_Expecter struct {
mock *mock.Mock
}
func (_m *MockStreamingCoordBroadcastService_WatchClient) EXPECT() *MockStreamingCoordBroadcastService_WatchClient_Expecter {
return &MockStreamingCoordBroadcastService_WatchClient_Expecter{mock: &_m.Mock}
}
// CloseSend provides a mock function with given fields:
func (_m *MockStreamingCoordBroadcastService_WatchClient) CloseSend() error {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for CloseSend")
}
var r0 error
if rf, ok := ret.Get(0).(func() error); ok {
r0 = rf()
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CloseSend'
type MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call struct {
*mock.Call
}
// CloseSend is a helper method to define mock.On call
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) CloseSend() *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
return &MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call{Call: _e.mock.On("CloseSend")}
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) RunAndReturn(run func() error) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
_c.Call.Return(run)
return _c
}
// Context provides a mock function with given fields:
func (_m *MockStreamingCoordBroadcastService_WatchClient) Context() context.Context {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Context")
}
var r0 context.Context
if rf, ok := ret.Get(0).(func() context.Context); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(context.Context)
}
}
return r0
}
// MockStreamingCoordBroadcastService_WatchClient_Context_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Context'
type MockStreamingCoordBroadcastService_WatchClient_Context_Call struct {
*mock.Call
}
// Context is a helper method to define mock.On call
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Context() *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
return &MockStreamingCoordBroadcastService_WatchClient_Context_Call{Call: _e.mock.On("Context")}
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) Return(_a0 context.Context) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) RunAndReturn(run func() context.Context) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
_c.Call.Return(run)
return _c
}
// Header provides a mock function with given fields:
func (_m *MockStreamingCoordBroadcastService_WatchClient) Header() (metadata.MD, error) {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Header")
}
var r0 metadata.MD
var r1 error
if rf, ok := ret.Get(0).(func() (metadata.MD, error)); ok {
return rf()
}
if rf, ok := ret.Get(0).(func() metadata.MD); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(metadata.MD)
}
}
if rf, ok := ret.Get(1).(func() error); ok {
r1 = rf()
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockStreamingCoordBroadcastService_WatchClient_Header_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Header'
type MockStreamingCoordBroadcastService_WatchClient_Header_Call struct {
*mock.Call
}
// Header is a helper method to define mock.On call
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Header() *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
return &MockStreamingCoordBroadcastService_WatchClient_Header_Call{Call: _e.mock.On("Header")}
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) Return(_a0 metadata.MD, _a1 error) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) RunAndReturn(run func() (metadata.MD, error)) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
_c.Call.Return(run)
return _c
}
// Recv provides a mock function with given fields:
func (_m *MockStreamingCoordBroadcastService_WatchClient) Recv() (*streamingpb.BroadcastWatchResponse, error) {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Recv")
}
var r0 *streamingpb.BroadcastWatchResponse
var r1 error
if rf, ok := ret.Get(0).(func() (*streamingpb.BroadcastWatchResponse, error)); ok {
return rf()
}
if rf, ok := ret.Get(0).(func() *streamingpb.BroadcastWatchResponse); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*streamingpb.BroadcastWatchResponse)
}
}
if rf, ok := ret.Get(1).(func() error); ok {
r1 = rf()
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockStreamingCoordBroadcastService_WatchClient_Recv_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Recv'
type MockStreamingCoordBroadcastService_WatchClient_Recv_Call struct {
*mock.Call
}
// Recv is a helper method to define mock.On call
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Recv() *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
return &MockStreamingCoordBroadcastService_WatchClient_Recv_Call{Call: _e.mock.On("Recv")}
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) Return(_a0 *streamingpb.BroadcastWatchResponse, _a1 error) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) RunAndReturn(run func() (*streamingpb.BroadcastWatchResponse, error)) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
_c.Call.Return(run)
return _c
}
// RecvMsg provides a mock function with given fields: m
func (_m *MockStreamingCoordBroadcastService_WatchClient) RecvMsg(m interface{}) error {
ret := _m.Called(m)
if len(ret) == 0 {
panic("no return value specified for RecvMsg")
}
var r0 error
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
r0 = rf(m)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RecvMsg'
type MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call struct {
*mock.Call
}
// RecvMsg is a helper method to define mock.On call
// - m interface{}
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) RecvMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
return &MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call{Call: _e.mock.On("RecvMsg", m)}
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(interface{}))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
_c.Call.Return(run)
return _c
}
// Send provides a mock function with given fields: _a0
func (_m *MockStreamingCoordBroadcastService_WatchClient) Send(_a0 *streamingpb.BroadcastWatchRequest) error {
ret := _m.Called(_a0)
if len(ret) == 0 {
panic("no return value specified for Send")
}
var r0 error
if rf, ok := ret.Get(0).(func(*streamingpb.BroadcastWatchRequest) error); ok {
r0 = rf(_a0)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchClient_Send_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Send'
type MockStreamingCoordBroadcastService_WatchClient_Send_Call struct {
*mock.Call
}
// Send is a helper method to define mock.On call
// - _a0 *streamingpb.BroadcastWatchRequest
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Send(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
return &MockStreamingCoordBroadcastService_WatchClient_Send_Call{Call: _e.mock.On("Send", _a0)}
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) Run(run func(_a0 *streamingpb.BroadcastWatchRequest)) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(*streamingpb.BroadcastWatchRequest))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) RunAndReturn(run func(*streamingpb.BroadcastWatchRequest) error) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
_c.Call.Return(run)
return _c
}
// SendMsg provides a mock function with given fields: m
func (_m *MockStreamingCoordBroadcastService_WatchClient) SendMsg(m interface{}) error {
ret := _m.Called(m)
if len(ret) == 0 {
panic("no return value specified for SendMsg")
}
var r0 error
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
r0 = rf(m)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendMsg'
type MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call struct {
*mock.Call
}
// SendMsg is a helper method to define mock.On call
// - m interface{}
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) SendMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
return &MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call{Call: _e.mock.On("SendMsg", m)}
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(interface{}))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
_c.Call.Return(run)
return _c
}
// Trailer provides a mock function with given fields:
func (_m *MockStreamingCoordBroadcastService_WatchClient) Trailer() metadata.MD {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Trailer")
}
var r0 metadata.MD
if rf, ok := ret.Get(0).(func() metadata.MD); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(metadata.MD)
}
}
return r0
}
// MockStreamingCoordBroadcastService_WatchClient_Trailer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Trailer'
type MockStreamingCoordBroadcastService_WatchClient_Trailer_Call struct {
*mock.Call
}
// Trailer is a helper method to define mock.On call
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Trailer() *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
return &MockStreamingCoordBroadcastService_WatchClient_Trailer_Call{Call: _e.mock.On("Trailer")}
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) Return(_a0 metadata.MD) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) RunAndReturn(run func() metadata.MD) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
_c.Call.Return(run)
return _c
}
// NewMockStreamingCoordBroadcastService_WatchClient creates a new instance of MockStreamingCoordBroadcastService_WatchClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockStreamingCoordBroadcastService_WatchClient(t interface {
mock.TestingT
Cleanup(func())
}) *MockStreamingCoordBroadcastService_WatchClient {
mock := &MockStreamingCoordBroadcastService_WatchClient{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -0,0 +1,406 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_streamingpb
import (
context "context"
mock "github.com/stretchr/testify/mock"
metadata "google.golang.org/grpc/metadata"
streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb"
)
// MockStreamingCoordBroadcastService_WatchServer is an autogenerated mock type for the StreamingCoordBroadcastService_WatchServer type
type MockStreamingCoordBroadcastService_WatchServer struct {
mock.Mock
}
type MockStreamingCoordBroadcastService_WatchServer_Expecter struct {
mock *mock.Mock
}
func (_m *MockStreamingCoordBroadcastService_WatchServer) EXPECT() *MockStreamingCoordBroadcastService_WatchServer_Expecter {
return &MockStreamingCoordBroadcastService_WatchServer_Expecter{mock: &_m.Mock}
}
// Context provides a mock function with given fields:
func (_m *MockStreamingCoordBroadcastService_WatchServer) Context() context.Context {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Context")
}
var r0 context.Context
if rf, ok := ret.Get(0).(func() context.Context); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(context.Context)
}
}
return r0
}
// MockStreamingCoordBroadcastService_WatchServer_Context_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Context'
type MockStreamingCoordBroadcastService_WatchServer_Context_Call struct {
*mock.Call
}
// Context is a helper method to define mock.On call
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Context() *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
return &MockStreamingCoordBroadcastService_WatchServer_Context_Call{Call: _e.mock.On("Context")}
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) Return(_a0 context.Context) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) RunAndReturn(run func() context.Context) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
_c.Call.Return(run)
return _c
}
// Recv provides a mock function with given fields:
func (_m *MockStreamingCoordBroadcastService_WatchServer) Recv() (*streamingpb.BroadcastWatchRequest, error) {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Recv")
}
var r0 *streamingpb.BroadcastWatchRequest
var r1 error
if rf, ok := ret.Get(0).(func() (*streamingpb.BroadcastWatchRequest, error)); ok {
return rf()
}
if rf, ok := ret.Get(0).(func() *streamingpb.BroadcastWatchRequest); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*streamingpb.BroadcastWatchRequest)
}
}
if rf, ok := ret.Get(1).(func() error); ok {
r1 = rf()
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockStreamingCoordBroadcastService_WatchServer_Recv_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Recv'
type MockStreamingCoordBroadcastService_WatchServer_Recv_Call struct {
*mock.Call
}
// Recv is a helper method to define mock.On call
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Recv() *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
return &MockStreamingCoordBroadcastService_WatchServer_Recv_Call{Call: _e.mock.On("Recv")}
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) Return(_a0 *streamingpb.BroadcastWatchRequest, _a1 error) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) RunAndReturn(run func() (*streamingpb.BroadcastWatchRequest, error)) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
_c.Call.Return(run)
return _c
}
// RecvMsg provides a mock function with given fields: m
func (_m *MockStreamingCoordBroadcastService_WatchServer) RecvMsg(m interface{}) error {
ret := _m.Called(m)
if len(ret) == 0 {
panic("no return value specified for RecvMsg")
}
var r0 error
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
r0 = rf(m)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RecvMsg'
type MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call struct {
*mock.Call
}
// RecvMsg is a helper method to define mock.On call
// - m interface{}
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) RecvMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
return &MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call{Call: _e.mock.On("RecvMsg", m)}
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(interface{}))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
_c.Call.Return(run)
return _c
}
// Send provides a mock function with given fields: _a0
func (_m *MockStreamingCoordBroadcastService_WatchServer) Send(_a0 *streamingpb.BroadcastWatchResponse) error {
ret := _m.Called(_a0)
if len(ret) == 0 {
panic("no return value specified for Send")
}
var r0 error
if rf, ok := ret.Get(0).(func(*streamingpb.BroadcastWatchResponse) error); ok {
r0 = rf(_a0)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchServer_Send_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Send'
type MockStreamingCoordBroadcastService_WatchServer_Send_Call struct {
*mock.Call
}
// Send is a helper method to define mock.On call
// - _a0 *streamingpb.BroadcastWatchResponse
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Send(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
return &MockStreamingCoordBroadcastService_WatchServer_Send_Call{Call: _e.mock.On("Send", _a0)}
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) Run(run func(_a0 *streamingpb.BroadcastWatchResponse)) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(*streamingpb.BroadcastWatchResponse))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) RunAndReturn(run func(*streamingpb.BroadcastWatchResponse) error) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
_c.Call.Return(run)
return _c
}
// SendHeader provides a mock function with given fields: _a0
func (_m *MockStreamingCoordBroadcastService_WatchServer) SendHeader(_a0 metadata.MD) error {
ret := _m.Called(_a0)
if len(ret) == 0 {
panic("no return value specified for SendHeader")
}
var r0 error
if rf, ok := ret.Get(0).(func(metadata.MD) error); ok {
r0 = rf(_a0)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendHeader'
type MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call struct {
*mock.Call
}
// SendHeader is a helper method to define mock.On call
// - _a0 metadata.MD
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SendHeader(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
return &MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call{Call: _e.mock.On("SendHeader", _a0)}
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(metadata.MD))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) RunAndReturn(run func(metadata.MD) error) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
_c.Call.Return(run)
return _c
}
// SendMsg provides a mock function with given fields: m
func (_m *MockStreamingCoordBroadcastService_WatchServer) SendMsg(m interface{}) error {
ret := _m.Called(m)
if len(ret) == 0 {
panic("no return value specified for SendMsg")
}
var r0 error
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
r0 = rf(m)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendMsg'
type MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call struct {
*mock.Call
}
// SendMsg is a helper method to define mock.On call
// - m interface{}
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SendMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
return &MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call{Call: _e.mock.On("SendMsg", m)}
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(interface{}))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
_c.Call.Return(run)
return _c
}
// SetHeader provides a mock function with given fields: _a0
func (_m *MockStreamingCoordBroadcastService_WatchServer) SetHeader(_a0 metadata.MD) error {
ret := _m.Called(_a0)
if len(ret) == 0 {
panic("no return value specified for SetHeader")
}
var r0 error
if rf, ok := ret.Get(0).(func(metadata.MD) error); ok {
r0 = rf(_a0)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetHeader'
type MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call struct {
*mock.Call
}
// SetHeader is a helper method to define mock.On call
// - _a0 metadata.MD
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SetHeader(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
return &MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call{Call: _e.mock.On("SetHeader", _a0)}
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(metadata.MD))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) RunAndReturn(run func(metadata.MD) error) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
_c.Call.Return(run)
return _c
}
// SetTrailer provides a mock function with given fields: _a0
func (_m *MockStreamingCoordBroadcastService_WatchServer) SetTrailer(_a0 metadata.MD) {
_m.Called(_a0)
}
// MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetTrailer'
type MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call struct {
*mock.Call
}
// SetTrailer is a helper method to define mock.On call
// - _a0 metadata.MD
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SetTrailer(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
return &MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call{Call: _e.mock.On("SetTrailer", _a0)}
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(metadata.MD))
})
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) Return() *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
_c.Call.Return()
return _c
}
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) RunAndReturn(run func(metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
_c.Call.Return(run)
return _c
}
// NewMockStreamingCoordBroadcastService_WatchServer creates a new instance of MockStreamingCoordBroadcastService_WatchServer. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockStreamingCoordBroadcastService_WatchServer(t interface {
mock.TestingT
Cleanup(func())
}) *MockStreamingCoordBroadcastService_WatchServer {
mock := &MockStreamingCoordBroadcastService_WatchServer{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -65,6 +65,53 @@ func (_c *MockImmutableMessage_BarrierTimeTick_Call) RunAndReturn(run func() uin
return _c return _c
} }
// BroadcastHeader provides a mock function with given fields:
func (_m *MockImmutableMessage) BroadcastHeader() *message.BroadcastHeader {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for BroadcastHeader")
}
var r0 *message.BroadcastHeader
if rf, ok := ret.Get(0).(func() *message.BroadcastHeader); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*message.BroadcastHeader)
}
}
return r0
}
// MockImmutableMessage_BroadcastHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastHeader'
type MockImmutableMessage_BroadcastHeader_Call struct {
*mock.Call
}
// BroadcastHeader is a helper method to define mock.On call
func (_e *MockImmutableMessage_Expecter) BroadcastHeader() *MockImmutableMessage_BroadcastHeader_Call {
return &MockImmutableMessage_BroadcastHeader_Call{Call: _e.mock.On("BroadcastHeader")}
}
func (_c *MockImmutableMessage_BroadcastHeader_Call) Run(run func()) *MockImmutableMessage_BroadcastHeader_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockImmutableMessage_BroadcastHeader_Call) Return(_a0 *message.BroadcastHeader) *MockImmutableMessage_BroadcastHeader_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockImmutableMessage_BroadcastHeader_Call) RunAndReturn(run func() *message.BroadcastHeader) *MockImmutableMessage_BroadcastHeader_Call {
_c.Call.Return(run)
return _c
}
// EstimateSize provides a mock function with given fields: // EstimateSize provides a mock function with given fields:
func (_m *MockImmutableMessage) EstimateSize() int { func (_m *MockImmutableMessage) EstimateSize() int {
ret := _m.Called() ret := _m.Called()

View File

@ -112,6 +112,53 @@ func (_c *MockImmutableTxnMessage_Begin_Call) RunAndReturn(run func() message.Im
return _c return _c
} }
// BroadcastHeader provides a mock function with given fields:
func (_m *MockImmutableTxnMessage) BroadcastHeader() *message.BroadcastHeader {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for BroadcastHeader")
}
var r0 *message.BroadcastHeader
if rf, ok := ret.Get(0).(func() *message.BroadcastHeader); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*message.BroadcastHeader)
}
}
return r0
}
// MockImmutableTxnMessage_BroadcastHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastHeader'
type MockImmutableTxnMessage_BroadcastHeader_Call struct {
*mock.Call
}
// BroadcastHeader is a helper method to define mock.On call
func (_e *MockImmutableTxnMessage_Expecter) BroadcastHeader() *MockImmutableTxnMessage_BroadcastHeader_Call {
return &MockImmutableTxnMessage_BroadcastHeader_Call{Call: _e.mock.On("BroadcastHeader")}
}
func (_c *MockImmutableTxnMessage_BroadcastHeader_Call) Run(run func()) *MockImmutableTxnMessage_BroadcastHeader_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockImmutableTxnMessage_BroadcastHeader_Call) Return(_a0 *message.BroadcastHeader) *MockImmutableTxnMessage_BroadcastHeader_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockImmutableTxnMessage_BroadcastHeader_Call) RunAndReturn(run func() *message.BroadcastHeader) *MockImmutableTxnMessage_BroadcastHeader_Call {
_c.Call.Return(run)
return _c
}
// Commit provides a mock function with given fields: // Commit provides a mock function with given fields:
func (_m *MockImmutableTxnMessage) Commit() message.ImmutableMessage { func (_m *MockImmutableTxnMessage) Commit() message.ImmutableMessage {
ret := _m.Called() ret := _m.Called()

View File

@ -65,6 +65,53 @@ func (_c *MockMutableMessage_BarrierTimeTick_Call) RunAndReturn(run func() uint6
return _c return _c
} }
// BroadcastHeader provides a mock function with given fields:
func (_m *MockMutableMessage) BroadcastHeader() *message.BroadcastHeader {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for BroadcastHeader")
}
var r0 *message.BroadcastHeader
if rf, ok := ret.Get(0).(func() *message.BroadcastHeader); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*message.BroadcastHeader)
}
}
return r0
}
// MockMutableMessage_BroadcastHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastHeader'
type MockMutableMessage_BroadcastHeader_Call struct {
*mock.Call
}
// BroadcastHeader is a helper method to define mock.On call
func (_e *MockMutableMessage_Expecter) BroadcastHeader() *MockMutableMessage_BroadcastHeader_Call {
return &MockMutableMessage_BroadcastHeader_Call{Call: _e.mock.On("BroadcastHeader")}
}
func (_c *MockMutableMessage_BroadcastHeader_Call) Run(run func()) *MockMutableMessage_BroadcastHeader_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockMutableMessage_BroadcastHeader_Call) Return(_a0 *message.BroadcastHeader) *MockMutableMessage_BroadcastHeader_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockMutableMessage_BroadcastHeader_Call) RunAndReturn(run func() *message.BroadcastHeader) *MockMutableMessage_BroadcastHeader_Call {
_c.Call.Return(run)
return _c
}
// EstimateSize provides a mock function with given fields: // EstimateSize provides a mock function with given fields:
func (_m *MockMutableMessage) EstimateSize() int { func (_m *MockMutableMessage) EstimateSize() int {
ret := _m.Called() ret := _m.Called()

View File

@ -243,7 +243,51 @@ enum TxnState {
TxnRollbacked = 6; TxnRollbacked = 6;
} }
// VChannels is a layout to represent the virtual channels for broadcast. // RMQMessageLayout is the layout of message for RMQ.
message VChannels { message RMQMessageLayout {
repeated string vchannels = 1; bytes payload = 1; // message body
map<string, string> properties = 2; // message properties
}
// BroadcastHeader is the common header of broadcast message.
message BroadcastHeader {
uint64 broadcast_id = 1;
repeated string vchannels = 2;
repeated ResourceKey Resource_keys = 3; // the resource key of the broadcast message.
// Once the broadcast is sent, the resource of resource key will be hold.
// New broadcast message with the same resource key will be rejected.
// And the user can watch the resource key to known when the resource is released.
}
// ResourceDomain is the domain of resource hold.
enum ResourceDomain {
ResourceDomainUnknown = 0; // should never be used.
ResourceDomainImportJobID = 1; // the domain of import job id.
ResourceDomainCollectionName = 2; // the domain of collection name.
}
// ResourceKey is the key for resource hold.
// It's used to implement the resource acquirition mechanism for broadcast message.
// The key should be a unique identifier of the resource for different domain.
message ResourceKey {
ResourceDomain domain = 1;
string key = 2;
}
// BroadcastEvent is the event of broadcast message.
message BroadcastEvent {
oneof event {
BroadcastResourceKeyAckAll resource_key_ack_all = 1;
BroadcastResourceKeyAckOne resource_key_ack_one = 2;
}
}
// The resource key is acked by all vchannels.
message BroadcastResourceKeyAckAll {
messages.ResourceKey resource_key = 1;
}
// The resource key is acked by any vchannel.
message BroadcastResourceKeyAckOne {
messages.ResourceKey resource_key = 1;
} }

View File

@ -191,6 +191,56 @@ func (TxnState) EnumDescriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{1} return file_messages_proto_rawDescGZIP(), []int{1}
} }
// ResourceDomain is the domain of resource hold.
type ResourceDomain int32
const (
ResourceDomain_ResourceDomainUnknown ResourceDomain = 0 // should never be used.
ResourceDomain_ResourceDomainImportJobID ResourceDomain = 1 // the domain of import job id.
ResourceDomain_ResourceDomainCollectionName ResourceDomain = 2 // the domain of collection name.
)
// Enum value maps for ResourceDomain.
var (
ResourceDomain_name = map[int32]string{
0: "ResourceDomainUnknown",
1: "ResourceDomainImportJobID",
2: "ResourceDomainCollectionName",
}
ResourceDomain_value = map[string]int32{
"ResourceDomainUnknown": 0,
"ResourceDomainImportJobID": 1,
"ResourceDomainCollectionName": 2,
}
)
func (x ResourceDomain) Enum() *ResourceDomain {
p := new(ResourceDomain)
*p = x
return p
}
func (x ResourceDomain) String() string {
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
}
func (ResourceDomain) Descriptor() protoreflect.EnumDescriptor {
return file_messages_proto_enumTypes[2].Descriptor()
}
func (ResourceDomain) Type() protoreflect.EnumType {
return &file_messages_proto_enumTypes[2]
}
func (x ResourceDomain) Number() protoreflect.EnumNumber {
return protoreflect.EnumNumber(x)
}
// Deprecated: Use ResourceDomain.Descriptor instead.
func (ResourceDomain) EnumDescriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{2}
}
// MessageID is the unique identifier of a message. // MessageID is the unique identifier of a message.
type MessageID struct { type MessageID struct {
state protoimpl.MessageState state protoimpl.MessageState
@ -1633,17 +1683,18 @@ func (x *TxnContext) GetKeepaliveMilliseconds() int64 {
return 0 return 0
} }
// VChannels is a layout to represent the virtual channels for broadcast. // RMQMessageLayout is the layout of message for RMQ.
type VChannels struct { type RMQMessageLayout struct {
state protoimpl.MessageState state protoimpl.MessageState
sizeCache protoimpl.SizeCache sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields unknownFields protoimpl.UnknownFields
Vchannels []string `protobuf:"bytes,1,rep,name=vchannels,proto3" json:"vchannels,omitempty"` Payload []byte `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"` // message body
Properties map[string]string `protobuf:"bytes,2,rep,name=properties,proto3" json:"properties,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // message properties
} }
func (x *VChannels) Reset() { func (x *RMQMessageLayout) Reset() {
*x = VChannels{} *x = RMQMessageLayout{}
if protoimpl.UnsafeEnabled { if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[29] mi := &file_messages_proto_msgTypes[29]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
@ -1651,13 +1702,13 @@ func (x *VChannels) Reset() {
} }
} }
func (x *VChannels) String() string { func (x *RMQMessageLayout) String() string {
return protoimpl.X.MessageStringOf(x) return protoimpl.X.MessageStringOf(x)
} }
func (*VChannels) ProtoMessage() {} func (*RMQMessageLayout) ProtoMessage() {}
func (x *VChannels) ProtoReflect() protoreflect.Message { func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[29] mi := &file_messages_proto_msgTypes[29]
if protoimpl.UnsafeEnabled && x != nil { if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
@ -1669,18 +1720,325 @@ func (x *VChannels) ProtoReflect() protoreflect.Message {
return mi.MessageOf(x) return mi.MessageOf(x)
} }
// Deprecated: Use VChannels.ProtoReflect.Descriptor instead. // Deprecated: Use RMQMessageLayout.ProtoReflect.Descriptor instead.
func (*VChannels) Descriptor() ([]byte, []int) { func (*RMQMessageLayout) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{29} return file_messages_proto_rawDescGZIP(), []int{29}
} }
func (x *VChannels) GetVchannels() []string { func (x *RMQMessageLayout) GetPayload() []byte {
if x != nil {
return x.Payload
}
return nil
}
func (x *RMQMessageLayout) GetProperties() map[string]string {
if x != nil {
return x.Properties
}
return nil
}
// BroadcastHeader is the common header of broadcast message.
type BroadcastHeader struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
BroadcastId uint64 `protobuf:"varint,1,opt,name=broadcast_id,json=broadcastId,proto3" json:"broadcast_id,omitempty"`
Vchannels []string `protobuf:"bytes,2,rep,name=vchannels,proto3" json:"vchannels,omitempty"`
ResourceKeys []*ResourceKey `protobuf:"bytes,3,rep,name=Resource_keys,json=ResourceKeys,proto3" json:"Resource_keys,omitempty"` // the resource key of the broadcast message.
}
func (x *BroadcastHeader) Reset() {
*x = BroadcastHeader{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[30]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BroadcastHeader) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BroadcastHeader) ProtoMessage() {}
func (x *BroadcastHeader) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[30]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BroadcastHeader.ProtoReflect.Descriptor instead.
func (*BroadcastHeader) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{30}
}
func (x *BroadcastHeader) GetBroadcastId() uint64 {
if x != nil {
return x.BroadcastId
}
return 0
}
func (x *BroadcastHeader) GetVchannels() []string {
if x != nil { if x != nil {
return x.Vchannels return x.Vchannels
} }
return nil return nil
} }
func (x *BroadcastHeader) GetResourceKeys() []*ResourceKey {
if x != nil {
return x.ResourceKeys
}
return nil
}
// ResourceKey is the key for resource hold.
// It's used to implement the resource acquirition mechanism for broadcast message.
// The key should be a unique identifier of the resource for different domain.
type ResourceKey struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Domain ResourceDomain `protobuf:"varint,1,opt,name=domain,proto3,enum=milvus.proto.messages.ResourceDomain" json:"domain,omitempty"`
Key string `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
}
func (x *ResourceKey) Reset() {
*x = ResourceKey{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[31]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ResourceKey) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ResourceKey) ProtoMessage() {}
func (x *ResourceKey) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[31]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ResourceKey.ProtoReflect.Descriptor instead.
func (*ResourceKey) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{31}
}
func (x *ResourceKey) GetDomain() ResourceDomain {
if x != nil {
return x.Domain
}
return ResourceDomain_ResourceDomainUnknown
}
func (x *ResourceKey) GetKey() string {
if x != nil {
return x.Key
}
return ""
}
// BroadcastEvent is the event of broadcast message.
type BroadcastEvent struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Types that are assignable to Event:
//
// *BroadcastEvent_ResourceKeyAckAll
// *BroadcastEvent_ResourceKeyAckOne
Event isBroadcastEvent_Event `protobuf_oneof:"event"`
}
func (x *BroadcastEvent) Reset() {
*x = BroadcastEvent{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BroadcastEvent) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BroadcastEvent) ProtoMessage() {}
func (x *BroadcastEvent) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[32]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BroadcastEvent.ProtoReflect.Descriptor instead.
func (*BroadcastEvent) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{32}
}
func (m *BroadcastEvent) GetEvent() isBroadcastEvent_Event {
if m != nil {
return m.Event
}
return nil
}
func (x *BroadcastEvent) GetResourceKeyAckAll() *BroadcastResourceKeyAckAll {
if x, ok := x.GetEvent().(*BroadcastEvent_ResourceKeyAckAll); ok {
return x.ResourceKeyAckAll
}
return nil
}
func (x *BroadcastEvent) GetResourceKeyAckOne() *BroadcastResourceKeyAckOne {
if x, ok := x.GetEvent().(*BroadcastEvent_ResourceKeyAckOne); ok {
return x.ResourceKeyAckOne
}
return nil
}
type isBroadcastEvent_Event interface {
isBroadcastEvent_Event()
}
type BroadcastEvent_ResourceKeyAckAll struct {
ResourceKeyAckAll *BroadcastResourceKeyAckAll `protobuf:"bytes,1,opt,name=resource_key_ack_all,json=resourceKeyAckAll,proto3,oneof"`
}
type BroadcastEvent_ResourceKeyAckOne struct {
ResourceKeyAckOne *BroadcastResourceKeyAckOne `protobuf:"bytes,2,opt,name=resource_key_ack_one,json=resourceKeyAckOne,proto3,oneof"`
}
func (*BroadcastEvent_ResourceKeyAckAll) isBroadcastEvent_Event() {}
func (*BroadcastEvent_ResourceKeyAckOne) isBroadcastEvent_Event() {}
// The resource key is acked by all vchannels.
type BroadcastResourceKeyAckAll struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
ResourceKey *ResourceKey `protobuf:"bytes,1,opt,name=resource_key,json=resourceKey,proto3" json:"resource_key,omitempty"`
}
func (x *BroadcastResourceKeyAckAll) Reset() {
*x = BroadcastResourceKeyAckAll{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BroadcastResourceKeyAckAll) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BroadcastResourceKeyAckAll) ProtoMessage() {}
func (x *BroadcastResourceKeyAckAll) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[33]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BroadcastResourceKeyAckAll.ProtoReflect.Descriptor instead.
func (*BroadcastResourceKeyAckAll) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{33}
}
func (x *BroadcastResourceKeyAckAll) GetResourceKey() *ResourceKey {
if x != nil {
return x.ResourceKey
}
return nil
}
// The resource key is acked by any vchannel.
type BroadcastResourceKeyAckOne struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
ResourceKey *ResourceKey `protobuf:"bytes,1,opt,name=resource_key,json=resourceKey,proto3" json:"resource_key,omitempty"`
}
func (x *BroadcastResourceKeyAckOne) Reset() {
*x = BroadcastResourceKeyAckOne{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[34]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BroadcastResourceKeyAckOne) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BroadcastResourceKeyAckOne) ProtoMessage() {}
func (x *BroadcastResourceKeyAckOne) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[34]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BroadcastResourceKeyAckOne.ProtoReflect.Descriptor instead.
func (*BroadcastResourceKeyAckOne) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{34}
}
func (x *BroadcastResourceKeyAckOne) GetResourceKey() *ResourceKey {
if x != nil {
return x.ResourceKey
}
return nil
}
var File_messages_proto protoreflect.FileDescriptor var File_messages_proto protoreflect.FileDescriptor
var file_messages_proto_rawDesc = []byte{ var file_messages_proto_rawDesc = []byte{
@ -1828,37 +2186,97 @@ var file_messages_proto_rawDesc = []byte{
0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73,
0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65,
0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f,
0x6e, 0x64, 0x73, 0x22, 0x29, 0x0a, 0x09, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61,
0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c,
0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x2a, 0xfc, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f,
0x01, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73,
0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52,
0x65, 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e,
0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52,
0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50,
0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10,
0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79,
0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42,
0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21,
0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01,
0x10, 0x0a, 0x12, 0x0d, 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49,
0x07, 0x12, 0x0e, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02,
0x07, 0x12, 0x10, 0x0a, 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12,
0x10, 0x86, 0x07, 0x12, 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73,
0x0a, 0x08, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52,
0x6e, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f,
0x6e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f,
0x4f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69,
0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e,
0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06,
0x10, 0x06, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20,
0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f,
0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72,
0x61, 0x67, 0x65, 0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f,
0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c,
0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65,
0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73,
0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f,
0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65,
0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74,
0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45,
0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61,
0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b,
0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f,
0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x2a, 0xfc, 0x01, 0x0a, 0x0b, 0x4d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e,
0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54,
0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10,
0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a,
0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61,
0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12,
0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74,
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61,
0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43,
0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0d,
0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a,
0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a,
0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12,
0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78,
0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b,
0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67,
0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69,
0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f,
0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d,
0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f,
0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54,
0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c,
0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e,
0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61,
0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70,
0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 0x42, 0x32, 0x5a, 0x30,
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62,
0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
} }
var ( var (
@ -1873,57 +2291,71 @@ func file_messages_proto_rawDescGZIP() []byte {
return file_messages_proto_rawDescData return file_messages_proto_rawDescData
} }
var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 2) var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 32) var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 38)
var file_messages_proto_goTypes = []interface{}{ var file_messages_proto_goTypes = []interface{}{
(MessageType)(0), // 0: milvus.proto.messages.MessageType (MessageType)(0), // 0: milvus.proto.messages.MessageType
(TxnState)(0), // 1: milvus.proto.messages.TxnState (TxnState)(0), // 1: milvus.proto.messages.TxnState
(*MessageID)(nil), // 2: milvus.proto.messages.MessageID (ResourceDomain)(0), // 2: milvus.proto.messages.ResourceDomain
(*Message)(nil), // 3: milvus.proto.messages.Message (*MessageID)(nil), // 3: milvus.proto.messages.MessageID
(*ImmutableMessage)(nil), // 4: milvus.proto.messages.ImmutableMessage (*Message)(nil), // 4: milvus.proto.messages.Message
(*FlushMessageBody)(nil), // 5: milvus.proto.messages.FlushMessageBody (*ImmutableMessage)(nil), // 5: milvus.proto.messages.ImmutableMessage
(*ManualFlushMessageBody)(nil), // 6: milvus.proto.messages.ManualFlushMessageBody (*FlushMessageBody)(nil), // 6: milvus.proto.messages.FlushMessageBody
(*CreateSegmentMessageBody)(nil), // 7: milvus.proto.messages.CreateSegmentMessageBody (*ManualFlushMessageBody)(nil), // 7: milvus.proto.messages.ManualFlushMessageBody
(*CreateSegmentInfo)(nil), // 8: milvus.proto.messages.CreateSegmentInfo (*CreateSegmentMessageBody)(nil), // 8: milvus.proto.messages.CreateSegmentMessageBody
(*BeginTxnMessageBody)(nil), // 9: milvus.proto.messages.BeginTxnMessageBody (*CreateSegmentInfo)(nil), // 9: milvus.proto.messages.CreateSegmentInfo
(*CommitTxnMessageBody)(nil), // 10: milvus.proto.messages.CommitTxnMessageBody (*BeginTxnMessageBody)(nil), // 10: milvus.proto.messages.BeginTxnMessageBody
(*RollbackTxnMessageBody)(nil), // 11: milvus.proto.messages.RollbackTxnMessageBody (*CommitTxnMessageBody)(nil), // 11: milvus.proto.messages.CommitTxnMessageBody
(*TxnMessageBody)(nil), // 12: milvus.proto.messages.TxnMessageBody (*RollbackTxnMessageBody)(nil), // 12: milvus.proto.messages.RollbackTxnMessageBody
(*TimeTickMessageHeader)(nil), // 13: milvus.proto.messages.TimeTickMessageHeader (*TxnMessageBody)(nil), // 13: milvus.proto.messages.TxnMessageBody
(*InsertMessageHeader)(nil), // 14: milvus.proto.messages.InsertMessageHeader (*TimeTickMessageHeader)(nil), // 14: milvus.proto.messages.TimeTickMessageHeader
(*PartitionSegmentAssignment)(nil), // 15: milvus.proto.messages.PartitionSegmentAssignment (*InsertMessageHeader)(nil), // 15: milvus.proto.messages.InsertMessageHeader
(*SegmentAssignment)(nil), // 16: milvus.proto.messages.SegmentAssignment (*PartitionSegmentAssignment)(nil), // 16: milvus.proto.messages.PartitionSegmentAssignment
(*DeleteMessageHeader)(nil), // 17: milvus.proto.messages.DeleteMessageHeader (*SegmentAssignment)(nil), // 17: milvus.proto.messages.SegmentAssignment
(*FlushMessageHeader)(nil), // 18: milvus.proto.messages.FlushMessageHeader (*DeleteMessageHeader)(nil), // 18: milvus.proto.messages.DeleteMessageHeader
(*CreateSegmentMessageHeader)(nil), // 19: milvus.proto.messages.CreateSegmentMessageHeader (*FlushMessageHeader)(nil), // 19: milvus.proto.messages.FlushMessageHeader
(*ManualFlushMessageHeader)(nil), // 20: milvus.proto.messages.ManualFlushMessageHeader (*CreateSegmentMessageHeader)(nil), // 20: milvus.proto.messages.CreateSegmentMessageHeader
(*CreateCollectionMessageHeader)(nil), // 21: milvus.proto.messages.CreateCollectionMessageHeader (*ManualFlushMessageHeader)(nil), // 21: milvus.proto.messages.ManualFlushMessageHeader
(*DropCollectionMessageHeader)(nil), // 22: milvus.proto.messages.DropCollectionMessageHeader (*CreateCollectionMessageHeader)(nil), // 22: milvus.proto.messages.CreateCollectionMessageHeader
(*CreatePartitionMessageHeader)(nil), // 23: milvus.proto.messages.CreatePartitionMessageHeader (*DropCollectionMessageHeader)(nil), // 23: milvus.proto.messages.DropCollectionMessageHeader
(*DropPartitionMessageHeader)(nil), // 24: milvus.proto.messages.DropPartitionMessageHeader (*CreatePartitionMessageHeader)(nil), // 24: milvus.proto.messages.CreatePartitionMessageHeader
(*BeginTxnMessageHeader)(nil), // 25: milvus.proto.messages.BeginTxnMessageHeader (*DropPartitionMessageHeader)(nil), // 25: milvus.proto.messages.DropPartitionMessageHeader
(*CommitTxnMessageHeader)(nil), // 26: milvus.proto.messages.CommitTxnMessageHeader (*BeginTxnMessageHeader)(nil), // 26: milvus.proto.messages.BeginTxnMessageHeader
(*RollbackTxnMessageHeader)(nil), // 27: milvus.proto.messages.RollbackTxnMessageHeader (*CommitTxnMessageHeader)(nil), // 27: milvus.proto.messages.CommitTxnMessageHeader
(*TxnMessageHeader)(nil), // 28: milvus.proto.messages.TxnMessageHeader (*RollbackTxnMessageHeader)(nil), // 28: milvus.proto.messages.RollbackTxnMessageHeader
(*ManualFlushExtraResponse)(nil), // 29: milvus.proto.messages.ManualFlushExtraResponse (*TxnMessageHeader)(nil), // 29: milvus.proto.messages.TxnMessageHeader
(*TxnContext)(nil), // 30: milvus.proto.messages.TxnContext (*ManualFlushExtraResponse)(nil), // 30: milvus.proto.messages.ManualFlushExtraResponse
(*VChannels)(nil), // 31: milvus.proto.messages.VChannels (*TxnContext)(nil), // 31: milvus.proto.messages.TxnContext
nil, // 32: milvus.proto.messages.Message.PropertiesEntry (*RMQMessageLayout)(nil), // 32: milvus.proto.messages.RMQMessageLayout
nil, // 33: milvus.proto.messages.ImmutableMessage.PropertiesEntry (*BroadcastHeader)(nil), // 33: milvus.proto.messages.BroadcastHeader
(*ResourceKey)(nil), // 34: milvus.proto.messages.ResourceKey
(*BroadcastEvent)(nil), // 35: milvus.proto.messages.BroadcastEvent
(*BroadcastResourceKeyAckAll)(nil), // 36: milvus.proto.messages.BroadcastResourceKeyAckAll
(*BroadcastResourceKeyAckOne)(nil), // 37: milvus.proto.messages.BroadcastResourceKeyAckOne
nil, // 38: milvus.proto.messages.Message.PropertiesEntry
nil, // 39: milvus.proto.messages.ImmutableMessage.PropertiesEntry
nil, // 40: milvus.proto.messages.RMQMessageLayout.PropertiesEntry
} }
var file_messages_proto_depIdxs = []int32{ var file_messages_proto_depIdxs = []int32{
32, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry 38, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry
2, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID 3, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID
33, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry 39, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry
8, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo 9, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo
3, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message 4, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message
15, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment 16, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment
16, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment 17, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment
7, // [7:7] is the sub-list for method output_type 40, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry
7, // [7:7] is the sub-list for method input_type 34, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey
7, // [7:7] is the sub-list for extension type_name 2, // 9: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain
7, // [7:7] is the sub-list for extension extendee 36, // 10: milvus.proto.messages.BroadcastEvent.resource_key_ack_all:type_name -> milvus.proto.messages.BroadcastResourceKeyAckAll
0, // [0:7] is the sub-list for field type_name 37, // 11: milvus.proto.messages.BroadcastEvent.resource_key_ack_one:type_name -> milvus.proto.messages.BroadcastResourceKeyAckOne
34, // 12: milvus.proto.messages.BroadcastResourceKeyAckAll.resource_key:type_name -> milvus.proto.messages.ResourceKey
34, // 13: milvus.proto.messages.BroadcastResourceKeyAckOne.resource_key:type_name -> milvus.proto.messages.ResourceKey
14, // [14:14] is the sub-list for method output_type
14, // [14:14] is the sub-list for method input_type
14, // [14:14] is the sub-list for extension type_name
14, // [14:14] is the sub-list for extension extendee
0, // [0:14] is the sub-list for field type_name
} }
func init() { file_messages_proto_init() } func init() { file_messages_proto_init() }
@ -2281,7 +2713,67 @@ func file_messages_proto_init() {
} }
} }
file_messages_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { file_messages_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*VChannels); i { switch v := v.(*RMQMessageLayout); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_messages_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastHeader); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_messages_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ResourceKey); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_messages_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastEvent); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_messages_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastResourceKeyAckAll); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_messages_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastResourceKeyAckOne); i {
case 0: case 0:
return &v.state return &v.state
case 1: case 1:
@ -2293,13 +2785,17 @@ func file_messages_proto_init() {
} }
} }
} }
file_messages_proto_msgTypes[32].OneofWrappers = []interface{}{
(*BroadcastEvent_ResourceKeyAckAll)(nil),
(*BroadcastEvent_ResourceKeyAckOne)(nil),
}
type x struct{} type x struct{}
out := protoimpl.TypeBuilder{ out := protoimpl.TypeBuilder{
File: protoimpl.DescBuilder{ File: protoimpl.DescBuilder{
GoPackagePath: reflect.TypeOf(x{}).PkgPath(), GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_messages_proto_rawDesc, RawDescriptor: file_messages_proto_rawDesc,
NumEnums: 2, NumEnums: 3,
NumMessages: 32, NumMessages: 38,
NumExtensions: 0, NumExtensions: 0,
NumServices: 0, NumServices: 0,
}, },

View File

@ -64,14 +64,15 @@ message VersionPair {
enum BroadcastTaskState { enum BroadcastTaskState {
BROADCAST_TASK_STATE_UNKNOWN = 0; // should never used. BROADCAST_TASK_STATE_UNKNOWN = 0; // should never used.
BROADCAST_TASK_STATE_PENDING = 1; // task is pending. BROADCAST_TASK_STATE_PENDING = 1; // task is pending.
BROADCAST_TASK_STATE_DONE = 2; // task is done, the message is broadcasted, and the persisted task can be cleared. BROADCAST_TASK_STATE_DONE = 2; // task has been broadcasted and acknowledged, the resource lock is released, and the persisted task can be cleared.
BROADCAST_TASK_STATE_WAIT_ACK = 3; // task has been broadcasted, waiting for ack, the resource lock is still acquired by some vchannels.
} }
// BroadcastTask is the task to broadcast the message. // BroadcastTask is the task to broadcast the messake.
message BroadcastTask { message BroadcastTask {
int64 task_id = 1; // task id. messages.Message message = 1; // message to be broadcast.
messages.Message message = 2; // message to be broadcast. BroadcastTaskState state = 2; // state of the task.
BroadcastTaskState state = 3; // state of the task. bytes acked_vchannel_bitmap = 3; // given vchannels that have been acked, the size of bitmap is same with message.BroadcastHeader().VChannels.
} }
// //
@ -92,6 +93,12 @@ service StreamingCoordBroadcastService {
// Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal.
// It performs an atomic broadcast to all wal, achieve eventual consistency. // It performs an atomic broadcast to all wal, achieve eventual consistency.
rpc Broadcast(BroadcastRequest) returns (BroadcastResponse) {} rpc Broadcast(BroadcastRequest) returns (BroadcastResponse) {}
// Ack acknowledge broadcast message is consumed.
rpc Ack(BroadcastAckRequest) returns (BroadcastAckResponse) {}
// Watch watch the broadcast events.
rpc Watch(stream BroadcastWatchRequest) returns (stream BroadcastWatchResponse) {}
} }
// BroadcastRequest is the request of the Broadcast RPC. // BroadcastRequest is the request of the Broadcast RPC.
@ -102,6 +109,43 @@ message BroadcastRequest {
// BroadcastResponse is the response of the Broadcast RPC. // BroadcastResponse is the response of the Broadcast RPC.
message BroadcastResponse { message BroadcastResponse {
map<string,ProduceMessageResponseResult> results = 1; map<string,ProduceMessageResponseResult> results = 1;
uint64 broadcast_id = 2;
}
message BroadcastAckRequest {
uint64 broadcast_id = 1; // broadcast id.
string vchannel = 2; // the vchannel that acked the message.
}
message BroadcastAckResponse {
}
message BroadcastWatchRequest {
oneof command {
BroadcastCreateEventWatchRequest create_event_watch = 1;
CloseBroadcastWatchRequest close = 2;
}
}
message BroadcastCreateEventWatchRequest {
messages.BroadcastEvent event = 1;
}
message CloseBroadcastWatchRequest {
}
message BroadcastWatchResponse {
oneof response {
BroadcastEventWatchResponse event_done = 1;
CloseBroadcastWatchResponse close = 2;
}
}
message BroadcastEventWatchResponse {
messages.BroadcastEvent event = 1;
}
message CloseBroadcastWatchResponse {
} }
// //
@ -223,6 +267,7 @@ enum StreamingCode {
STREAMING_CODE_TRANSACTION_EXPIRED = 9; // transaction expired STREAMING_CODE_TRANSACTION_EXPIRED = 9; // transaction expired
STREAMING_CODE_INVALID_TRANSACTION_STATE = 10; // invalid transaction state STREAMING_CODE_INVALID_TRANSACTION_STATE = 10; // invalid transaction state
STREAMING_CODE_UNRECOVERABLE = 11; // unrecoverable error STREAMING_CODE_UNRECOVERABLE = 11; // unrecoverable error
STREAMING_CODE_RESOURCE_ACQUIRED = 12; // resource is acquired by other operation
STREAMING_CODE_UNKNOWN = 999; // unknown error STREAMING_CODE_UNKNOWN = 999; // unknown error
} }

File diff suppressed because it is too large Load Diff

View File

@ -109,6 +109,8 @@ var StreamingNodeStateService_ServiceDesc = grpc.ServiceDesc{
const ( const (
StreamingCoordBroadcastService_Broadcast_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Broadcast" StreamingCoordBroadcastService_Broadcast_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Broadcast"
StreamingCoordBroadcastService_Ack_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Ack"
StreamingCoordBroadcastService_Watch_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Watch"
) )
// StreamingCoordBroadcastServiceClient is the client API for StreamingCoordBroadcastService service. // StreamingCoordBroadcastServiceClient is the client API for StreamingCoordBroadcastService service.
@ -118,6 +120,10 @@ type StreamingCoordBroadcastServiceClient interface {
// Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal.
// It performs an atomic broadcast to all wal, achieve eventual consistency. // It performs an atomic broadcast to all wal, achieve eventual consistency.
Broadcast(ctx context.Context, in *BroadcastRequest, opts ...grpc.CallOption) (*BroadcastResponse, error) Broadcast(ctx context.Context, in *BroadcastRequest, opts ...grpc.CallOption) (*BroadcastResponse, error)
// Ack acknowledge broadcast message is consumed.
Ack(ctx context.Context, in *BroadcastAckRequest, opts ...grpc.CallOption) (*BroadcastAckResponse, error)
// Watch watch the broadcast events.
Watch(ctx context.Context, opts ...grpc.CallOption) (StreamingCoordBroadcastService_WatchClient, error)
} }
type streamingCoordBroadcastServiceClient struct { type streamingCoordBroadcastServiceClient struct {
@ -137,6 +143,46 @@ func (c *streamingCoordBroadcastServiceClient) Broadcast(ctx context.Context, in
return out, nil return out, nil
} }
func (c *streamingCoordBroadcastServiceClient) Ack(ctx context.Context, in *BroadcastAckRequest, opts ...grpc.CallOption) (*BroadcastAckResponse, error) {
out := new(BroadcastAckResponse)
err := c.cc.Invoke(ctx, StreamingCoordBroadcastService_Ack_FullMethodName, in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *streamingCoordBroadcastServiceClient) Watch(ctx context.Context, opts ...grpc.CallOption) (StreamingCoordBroadcastService_WatchClient, error) {
stream, err := c.cc.NewStream(ctx, &StreamingCoordBroadcastService_ServiceDesc.Streams[0], StreamingCoordBroadcastService_Watch_FullMethodName, opts...)
if err != nil {
return nil, err
}
x := &streamingCoordBroadcastServiceWatchClient{stream}
return x, nil
}
type StreamingCoordBroadcastService_WatchClient interface {
Send(*BroadcastWatchRequest) error
Recv() (*BroadcastWatchResponse, error)
grpc.ClientStream
}
type streamingCoordBroadcastServiceWatchClient struct {
grpc.ClientStream
}
func (x *streamingCoordBroadcastServiceWatchClient) Send(m *BroadcastWatchRequest) error {
return x.ClientStream.SendMsg(m)
}
func (x *streamingCoordBroadcastServiceWatchClient) Recv() (*BroadcastWatchResponse, error) {
m := new(BroadcastWatchResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
// StreamingCoordBroadcastServiceServer is the server API for StreamingCoordBroadcastService service. // StreamingCoordBroadcastServiceServer is the server API for StreamingCoordBroadcastService service.
// All implementations should embed UnimplementedStreamingCoordBroadcastServiceServer // All implementations should embed UnimplementedStreamingCoordBroadcastServiceServer
// for forward compatibility // for forward compatibility
@ -144,6 +190,10 @@ type StreamingCoordBroadcastServiceServer interface {
// Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal.
// It performs an atomic broadcast to all wal, achieve eventual consistency. // It performs an atomic broadcast to all wal, achieve eventual consistency.
Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error) Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error)
// Ack acknowledge broadcast message is consumed.
Ack(context.Context, *BroadcastAckRequest) (*BroadcastAckResponse, error)
// Watch watch the broadcast events.
Watch(StreamingCoordBroadcastService_WatchServer) error
} }
// UnimplementedStreamingCoordBroadcastServiceServer should be embedded to have forward compatible implementations. // UnimplementedStreamingCoordBroadcastServiceServer should be embedded to have forward compatible implementations.
@ -153,6 +203,12 @@ type UnimplementedStreamingCoordBroadcastServiceServer struct {
func (UnimplementedStreamingCoordBroadcastServiceServer) Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error) { func (UnimplementedStreamingCoordBroadcastServiceServer) Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method Broadcast not implemented") return nil, status.Errorf(codes.Unimplemented, "method Broadcast not implemented")
} }
func (UnimplementedStreamingCoordBroadcastServiceServer) Ack(context.Context, *BroadcastAckRequest) (*BroadcastAckResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method Ack not implemented")
}
func (UnimplementedStreamingCoordBroadcastServiceServer) Watch(StreamingCoordBroadcastService_WatchServer) error {
return status.Errorf(codes.Unimplemented, "method Watch not implemented")
}
// UnsafeStreamingCoordBroadcastServiceServer may be embedded to opt out of forward compatibility for this service. // UnsafeStreamingCoordBroadcastServiceServer may be embedded to opt out of forward compatibility for this service.
// Use of this interface is not recommended, as added methods to StreamingCoordBroadcastServiceServer will // Use of this interface is not recommended, as added methods to StreamingCoordBroadcastServiceServer will
@ -183,6 +239,50 @@ func _StreamingCoordBroadcastService_Broadcast_Handler(srv interface{}, ctx cont
return interceptor(ctx, in, info, handler) return interceptor(ctx, in, info, handler)
} }
func _StreamingCoordBroadcastService_Ack_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(BroadcastAckRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(StreamingCoordBroadcastServiceServer).Ack(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: StreamingCoordBroadcastService_Ack_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(StreamingCoordBroadcastServiceServer).Ack(ctx, req.(*BroadcastAckRequest))
}
return interceptor(ctx, in, info, handler)
}
func _StreamingCoordBroadcastService_Watch_Handler(srv interface{}, stream grpc.ServerStream) error {
return srv.(StreamingCoordBroadcastServiceServer).Watch(&streamingCoordBroadcastServiceWatchServer{stream})
}
type StreamingCoordBroadcastService_WatchServer interface {
Send(*BroadcastWatchResponse) error
Recv() (*BroadcastWatchRequest, error)
grpc.ServerStream
}
type streamingCoordBroadcastServiceWatchServer struct {
grpc.ServerStream
}
func (x *streamingCoordBroadcastServiceWatchServer) Send(m *BroadcastWatchResponse) error {
return x.ServerStream.SendMsg(m)
}
func (x *streamingCoordBroadcastServiceWatchServer) Recv() (*BroadcastWatchRequest, error) {
m := new(BroadcastWatchRequest)
if err := x.ServerStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
// StreamingCoordBroadcastService_ServiceDesc is the grpc.ServiceDesc for StreamingCoordBroadcastService service. // StreamingCoordBroadcastService_ServiceDesc is the grpc.ServiceDesc for StreamingCoordBroadcastService service.
// It's only intended for direct use with grpc.RegisterService, // It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy) // and not to be introspected or modified (even as a copy)
@ -194,8 +294,19 @@ var StreamingCoordBroadcastService_ServiceDesc = grpc.ServiceDesc{
MethodName: "Broadcast", MethodName: "Broadcast",
Handler: _StreamingCoordBroadcastService_Broadcast_Handler, Handler: _StreamingCoordBroadcastService_Broadcast_Handler,
}, },
{
MethodName: "Ack",
Handler: _StreamingCoordBroadcastService_Ack_Handler,
},
},
Streams: []grpc.StreamDesc{
{
StreamName: "Watch",
Handler: _StreamingCoordBroadcastService_Watch_Handler,
ServerStreams: true,
ClientStreams: true,
},
}, },
Streams: []grpc.StreamDesc{},
Metadata: "streaming.proto", Metadata: "streaming.proto",
} }

View File

@ -0,0 +1,111 @@
package message
import (
"strconv"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/pkg/proto/messagespb"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
// newBroadcastHeaderFromProto creates a BroadcastHeader from proto.
func newBroadcastHeaderFromProto(proto *messagespb.BroadcastHeader) *BroadcastHeader {
rks := make(typeutil.Set[ResourceKey], len(proto.ResourceKeys))
for _, key := range proto.ResourceKeys {
rks.Insert(NewResourceKeyFromProto(key))
}
return &BroadcastHeader{
BroadcastID: proto.BroadcastId,
VChannels: proto.Vchannels,
ResourceKeys: rks,
}
}
type BroadcastHeader struct {
BroadcastID uint64
VChannels []string
ResourceKeys typeutil.Set[ResourceKey]
}
// NewResourceKeyFromProto creates a ResourceKey from proto.
func NewResourceKeyFromProto(proto *messagespb.ResourceKey) ResourceKey {
return ResourceKey{
Domain: proto.Domain,
Key: proto.Key,
}
}
// newProtoFromResourceKey creates a set of proto from ResourceKey.
func newProtoFromResourceKey(keys ...ResourceKey) []*messagespb.ResourceKey {
deduplicated := typeutil.NewSet(keys...)
protos := make([]*messagespb.ResourceKey, 0, len(keys))
for key := range deduplicated {
protos = append(protos, &messagespb.ResourceKey{
Domain: key.Domain,
Key: key.Key,
})
}
return protos
}
type ResourceKey struct {
Domain messagespb.ResourceDomain
Key string
}
func (rk *ResourceKey) IntoResourceKey() *messagespb.ResourceKey {
return &messagespb.ResourceKey{
Domain: rk.Domain,
Key: rk.Key,
}
}
// NewImportJobIDResourceKey creates a key for import job resource.
func NewImportJobIDResourceKey(importJobID int64) ResourceKey {
return ResourceKey{
Domain: messagespb.ResourceDomain_ResourceDomainImportJobID,
Key: strconv.FormatInt(importJobID, 10),
}
}
// NewCollectionNameResourceKey creates a key for collection name resource.
func NewCollectionNameResourceKey(collectionName string) ResourceKey {
return ResourceKey{
Domain: messagespb.ResourceDomain_ResourceDomainCollectionName,
Key: collectionName,
}
}
type BroadcastEvent = messagespb.BroadcastEvent
// UniqueKeyOfBroadcastEvent returns a unique key for a broadcast event.
func UniqueKeyOfBroadcastEvent(ev *BroadcastEvent) string {
s, err := proto.Marshal(ev)
if err != nil {
panic(err)
}
return string(s)
}
// NewResourceKeyAckOneBroadcastEvent creates a broadcast event for acking one key.
func NewResourceKeyAckOneBroadcastEvent(rk ResourceKey) *BroadcastEvent {
return &BroadcastEvent{
Event: &messagespb.BroadcastEvent_ResourceKeyAckOne{
ResourceKeyAckOne: &messagespb.BroadcastResourceKeyAckOne{
ResourceKey: rk.IntoResourceKey(),
},
},
}
}
// NewResourceKeyAckAllBroadcastEvent creates a broadcast event for ack all vchannel.
func NewResourceKeyAckAllBroadcastEvent(rk ResourceKey) *BroadcastEvent {
return &BroadcastEvent{
Event: &messagespb.BroadcastEvent_ResourceKeyAckAll{
ResourceKeyAckAll: &messagespb.BroadcastResourceKeyAckAll{
ResourceKey: rk.IntoResourceKey(),
},
},
}
}

View File

@ -12,26 +12,26 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil" "github.com/milvus-io/milvus/pkg/util/typeutil"
) )
// NewMutableMessage creates a new mutable message. // NewMutableMessageBeforeAppend creates a new mutable message.
// !!! Only used at server side for streamingnode internal service, don't use it at client side. // !!! Only used at server side for streamingnode internal service, don't use it at client side.
func NewMutableMessage(payload []byte, properties map[string]string) MutableMessage { func NewMutableMessageBeforeAppend(payload []byte, properties map[string]string) MutableMessage {
m := &messageImpl{ m := &messageImpl{
payload: payload, payload: payload,
properties: properties, properties: properties,
} }
// make a assertion by vchannel function.
m.assertNotBroadcast()
return m return m
} }
// NewBroadcastMutableMessage creates a new broadcast mutable message. // NewBroadcastMutableMessageBeforeAppend creates a new broadcast mutable message.
// !!! Only used at server side for streamingcoord internal service, don't use it at client side. // !!! Only used at server side for streamingcoord internal service, don't use it at client side.
func NewBroadcastMutableMessage(payload []byte, properties map[string]string) BroadcastMutableMessage { func NewBroadcastMutableMessageBeforeAppend(payload []byte, properties map[string]string) BroadcastMutableMessage {
m := &messageImpl{ m := &messageImpl{
payload: payload, payload: payload,
properties: properties, properties: properties,
} }
m.assertBroadcast() if !m.properties.Exist(messageBroadcastHeader) {
panic("current message is not a broadcast message")
}
return m return m
} }
@ -126,7 +126,7 @@ func (b *mutableMesasgeBuilder[H, B]) WithVChannel(vchannel string) *mutableMesa
} }
// WithBroadcast creates a new builder with broadcast property. // WithBroadcast creates a new builder with broadcast property.
func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string) *mutableMesasgeBuilder[H, B] { func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string, resourceKeys ...ResourceKey) *mutableMesasgeBuilder[H, B] {
if len(vchannels) < 1 { if len(vchannels) < 1 {
panic("broadcast message must have at least one vchannel") panic("broadcast message must have at least one vchannel")
} }
@ -137,19 +137,21 @@ func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string) *mutable
panic("a broadcast message cannot set up vchannel property") panic("a broadcast message cannot set up vchannel property")
} }
deduplicated := typeutil.NewSet(vchannels...) deduplicated := typeutil.NewSet(vchannels...)
vcs, err := EncodeProto(&messagespb.VChannels{
bh, err := EncodeProto(&messagespb.BroadcastHeader{
Vchannels: deduplicated.Collect(), Vchannels: deduplicated.Collect(),
ResourceKeys: newProtoFromResourceKey(resourceKeys...),
}) })
if err != nil { if err != nil {
panic("failed to encode vchannels") panic("failed to encode vchannels")
} }
b.properties.Set(messageVChannels, vcs) b.properties.Set(messageBroadcastHeader, bh)
return b return b
} }
// WithAllVChannel creates a new builder with all vchannel property. // WithAllVChannel creates a new builder with all vchannel property.
func (b *mutableMesasgeBuilder[H, B]) WithAllVChannel() *mutableMesasgeBuilder[H, B] { func (b *mutableMesasgeBuilder[H, B]) WithAllVChannel() *mutableMesasgeBuilder[H, B] {
if b.properties.Exist(messageVChannel) || b.properties.Exist(messageVChannels) { if b.properties.Exist(messageVChannel) || b.properties.Exist(messageBroadcastHeader) {
panic("a vchannel or broadcast message cannot set up all vchannel property") panic("a vchannel or broadcast message cannot set up all vchannel property")
} }
b.allVChannel = true b.allVChannel = true
@ -191,7 +193,7 @@ func (b *mutableMesasgeBuilder[H, B]) BuildMutable() (MutableMessage, error) {
// Panic if not set payload and message type. // Panic if not set payload and message type.
// should only used at client side. // should only used at client side.
func (b *mutableMesasgeBuilder[H, B]) BuildBroadcast() (BroadcastMutableMessage, error) { func (b *mutableMesasgeBuilder[H, B]) BuildBroadcast() (BroadcastMutableMessage, error) {
if !b.properties.Exist(messageVChannels) { if !b.properties.Exist(messageBroadcastHeader) {
panic("a broadcast message builder not ready for vchannel field") panic("a broadcast message builder not ready for vchannel field")
} }

View File

@ -1,6 +1,8 @@
package message package message
import "google.golang.org/protobuf/proto" import (
"google.golang.org/protobuf/proto"
)
var ( var (
_ BasicMessage = (*messageImpl)(nil) _ BasicMessage = (*messageImpl)(nil)
@ -39,7 +41,12 @@ type BasicMessage interface {
BarrierTimeTick() uint64 BarrierTimeTick() uint64
// TxnContext returns the transaction context of current message. // TxnContext returns the transaction context of current message.
// If the message is not a transaction message, it will return nil.
TxnContext() *TxnContext TxnContext() *TxnContext
// BroadcastHeader returns the broadcast common header of the message.
// If the message is not a broadcast message, it will return 0.
BroadcastHeader() *BroadcastHeader
} }
// MutableMessage is the mutable message interface. // MutableMessage is the mutable message interface.
@ -87,11 +94,11 @@ type MutableMessage interface {
type BroadcastMutableMessage interface { type BroadcastMutableMessage interface {
BasicMessage BasicMessage
// BroadcastVChannels returns the target vchannels of the message broadcast. // WithBroadcastID sets the broadcast id of the message.
// Those vchannels can be on multi pchannels. WithBroadcastID(broadcastID uint64) BroadcastMutableMessage
BroadcastVChannels() []string
// SplitIntoMutableMessage splits the broadcast message into multiple mutable messages. // SplitIntoMutableMessage splits the broadcast message into multiple mutable messages.
// The broadcast id will be set into the properties of each message.
SplitIntoMutableMessage() []MutableMessage SplitIntoMutableMessage() []MutableMessage
} }

View File

@ -93,6 +93,24 @@ func (m *messageImpl) WithTxnContext(txnCtx TxnContext) MutableMessage {
return m return m
} }
// WithBroadcastID sets the broadcast id of current message.
func (m *messageImpl) WithBroadcastID(id uint64) BroadcastMutableMessage {
bh := m.broadcastHeader()
if bh == nil {
panic("there's a bug in the message codes, broadcast header lost in properties of broadcast message")
}
if bh.BroadcastId != 0 {
panic("broadcast id already set in properties of broadcast message")
}
bh.BroadcastId = id
bhVal, err := EncodeProto(bh)
if err != nil {
panic("should not happen on broadcast header proto")
}
m.properties.Set(messageBroadcastHeader, bhVal)
return m
}
// IntoImmutableMessage converts current message to immutable message. // IntoImmutableMessage converts current message to immutable message.
func (m *messageImpl) IntoImmutableMessage(id MessageID) ImmutableMessage { func (m *messageImpl) IntoImmutableMessage(id MessageID) ImmutableMessage {
return &immutableMessageImpl{ return &immutableMessageImpl{
@ -144,8 +162,10 @@ func (m *messageImpl) BarrierTimeTick() uint64 {
// If the message is a all channel message, it will return "". // If the message is a all channel message, it will return "".
// If the message is a broadcast message, it will panic. // If the message is a broadcast message, it will panic.
func (m *messageImpl) VChannel() string { func (m *messageImpl) VChannel() string {
m.assertNotBroadcast() if m.properties.Exist(messageBroadcastHeader) && !m.properties.Exist(messageVChannel) {
// If a message is a broadcast message, it must have a vchannel properties in it after split.
panic("there's a bug in the message codes, vchannel lost in properties of broadcast message")
}
value, ok := m.properties.Get(messageVChannel) value, ok := m.properties.Get(messageVChannel)
if !ok { if !ok {
return "" return ""
@ -153,22 +173,38 @@ func (m *messageImpl) VChannel() string {
return value return value
} }
// BroadcastVChannels returns the vchannels of current message that want to broadcast. // BroadcastHeader returns the broadcast header of current message.
// If the message is not a broadcast message, it will panic. func (m *messageImpl) BroadcastHeader() *BroadcastHeader {
func (m *messageImpl) BroadcastVChannels() []string { header := m.broadcastHeader()
m.assertBroadcast() return newBroadcastHeaderFromProto(header)
}
value, _ := m.properties.Get(messageVChannels) // broadcastHeader returns the broadcast header of current message.
vcs := &messagespb.VChannels{} func (m *messageImpl) broadcastHeader() *messagespb.BroadcastHeader {
if err := DecodeProto(value, vcs); err != nil { value, ok := m.properties.Get(messageBroadcastHeader)
panic("can not decode vchannels") if !ok {
return nil
} }
return vcs.Vchannels header := &messagespb.BroadcastHeader{}
if err := DecodeProto(value, header); err != nil {
panic("can not decode broadcast header")
}
return header
} }
// SplitIntoMutableMessage splits the current broadcast message into multiple messages. // SplitIntoMutableMessage splits the current broadcast message into multiple messages.
func (m *messageImpl) SplitIntoMutableMessage() []MutableMessage { func (m *messageImpl) SplitIntoMutableMessage() []MutableMessage {
vchannels := m.BroadcastVChannels() bh := m.broadcastHeader()
if bh == nil {
panic("there's a bug in the message codes, broadcast header lost in properties of broadcast message")
}
if len(bh.Vchannels) == 0 {
panic("there's a bug in the message codes, no vchannel in broadcast message")
}
if bh.BroadcastId == 0 {
panic("there's a bug in the message codes, no broadcast id in broadcast message")
}
vchannels := bh.Vchannels
vchannelExist := make(map[string]struct{}, len(vchannels)) vchannelExist := make(map[string]struct{}, len(vchannels))
msgs := make([]MutableMessage, 0, len(vchannels)) msgs := make([]MutableMessage, 0, len(vchannels))
@ -178,10 +214,8 @@ func (m *messageImpl) SplitIntoMutableMessage() []MutableMessage {
newProperties := make(propertiesImpl, len(m.properties)) newProperties := make(propertiesImpl, len(m.properties))
for key, val := range m.properties { for key, val := range m.properties {
if key != messageVChannels {
newProperties.Set(key, val) newProperties.Set(key, val)
} }
}
newProperties.Set(messageVChannel, vchannel) newProperties.Set(messageVChannel, vchannel)
if _, ok := vchannelExist[vchannel]; ok { if _, ok := vchannelExist[vchannel]; ok {
panic("there's a bug in the message codes, duplicate vchannel in broadcast message") panic("there's a bug in the message codes, duplicate vchannel in broadcast message")
@ -195,18 +229,6 @@ func (m *messageImpl) SplitIntoMutableMessage() []MutableMessage {
return msgs return msgs
} }
func (m *messageImpl) assertNotBroadcast() {
if m.properties.Exist(messageVChannels) {
panic("current message is a broadcast message")
}
}
func (m *messageImpl) assertBroadcast() {
if !m.properties.Exist(messageVChannels) {
panic("current message is not a broadcast message")
}
}
type immutableMessageImpl struct { type immutableMessageImpl struct {
messageImpl messageImpl
id MessageID id MessageID

View File

@ -10,7 +10,7 @@ const (
messageLastConfirmed = "_lc" // message last confirmed message id. messageLastConfirmed = "_lc" // message last confirmed message id.
messageLastConfirmedIDSameWithMessageID = "_lcs" // message last confirmed message id is the same with message id. messageLastConfirmedIDSameWithMessageID = "_lcs" // message last confirmed message id is the same with message id.
messageVChannel = "_vc" // message virtual channel. messageVChannel = "_vc" // message virtual channel.
messageVChannels = "_vcs" // message virtual channels for broadcast message. messageBroadcastHeader = "_bh" // message broadcast header.
messageHeader = "_h" // specialized message header. messageHeader = "_h" // specialized message header.
messageTxnContext = "_tx" // transaction context. messageTxnContext = "_tx" // transaction context.
) )

View File

@ -9,8 +9,15 @@ import (
"github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message"
) )
type BroadcastAckRequest struct {
// BroadcastID is the broadcast id of the ack request.
BroadcastID uint64
VChannel string
}
// BroadcastAppendResult is the result of broadcast append operation. // BroadcastAppendResult is the result of broadcast append operation.
type BroadcastAppendResult struct { type BroadcastAppendResult struct {
BroadcastID uint64 // the broadcast id of the append operation.
AppendResults map[string]*AppendResult // make the channel name to the append result. AppendResults map[string]*AppendResult // make the channel name to the append result.
} }

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
"github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
@ -304,6 +305,7 @@ func (s *CoordDownSearch) searchAfterCoordDown() float64 {
log.Info(fmt.Sprintf("=========================Failed search cost: %fs=========================", time.Since(failedStart).Seconds())) log.Info(fmt.Sprintf("=========================Failed search cost: %fs=========================", time.Since(failedStart).Seconds()))
registry.ResetRegistration() registry.ResetRegistration()
coordclient.ResetRegistration()
log.Info("=========================restart Root Coordinators=========================") log.Info("=========================restart Root Coordinators=========================")
c.StartRootCoord() c.StartRootCoord()

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
"github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
@ -244,6 +245,7 @@ func (s *CoordSwitchSuite) switchCoord() float64 {
start = time.Now() start = time.Now()
registry.ResetRegistration() registry.ResetRegistration()
coordclient.ResetRegistration()
c.StartRootCoord() c.StartRootCoord()
log.Info("=========================RootCoord restarted=========================") log.Info("=========================RootCoord restarted=========================")

View File

@ -37,6 +37,7 @@ import (
"google.golang.org/grpc/keepalive" "google.golang.org/grpc/keepalive"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
grpcdatacoord "github.com/milvus-io/milvus/internal/distributed/datacoord" grpcdatacoord "github.com/milvus-io/milvus/internal/distributed/datacoord"
grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord/client" grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode" grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode"
@ -54,6 +55,7 @@ import (
"github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/distributed/streamingnode" "github.com/milvus-io/milvus/internal/distributed/streamingnode"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/hookutil"
@ -77,6 +79,7 @@ func DefaultParams() map[string]string {
// Notice: don't use ParamItem.Key here, the config key will be empty before param table init // Notice: don't use ParamItem.Key here, the config key will be empty before param table init
configMap = map[string]string{ configMap = map[string]string{
"mq.type": "rocksmq",
"etcd.rootPath": testPath, "etcd.rootPath": testPath,
"minio.rootPath": testPath, "minio.rootPath": testPath,
"localStorage.path": path.Join("/tmp", testPath), "localStorage.path": path.Join("/tmp", testPath),
@ -135,6 +138,12 @@ type MiniClusterV2 struct {
type OptionV2 func(cluster *MiniClusterV2) type OptionV2 func(cluster *MiniClusterV2)
func OptionSetParam(k string, v string) OptionV2 {
return func(cluster *MiniClusterV2) {
cluster.params[k] = v
}
}
func StartMiniClusterV2(ctx context.Context, opts ...OptionV2) (*MiniClusterV2, error) { func StartMiniClusterV2(ctx context.Context, opts ...OptionV2) (*MiniClusterV2, error) {
cluster := &MiniClusterV2{ cluster := &MiniClusterV2{
ctx: ctx, ctx: ctx,
@ -151,6 +160,8 @@ func StartMiniClusterV2(ctx context.Context, opts ...OptionV2) (*MiniClusterV2,
for k, v := range cluster.params { for k, v := range cluster.params {
params.Save(k, v) params.Save(k, v)
} }
paramtable.SetRole(typeutil.StandaloneRole)
// setup etcd client // setup etcd client
etcdConfig := &paramtable.Get().EtcdCfg etcdConfig := &paramtable.Get().EtcdCfg
etcdCli, err := etcd.GetEtcdClient( etcdCli, err := etcd.GetEtcdClient(
@ -166,9 +177,9 @@ func StartMiniClusterV2(ctx context.Context, opts ...OptionV2) (*MiniClusterV2,
} }
cluster.EtcdCli = etcdCli cluster.EtcdCli = etcdCli
if streamingutil.IsStreamingServiceEnabled() { coordclient.ResetRegistration()
registry.ResetRegistration()
streaming.Init() streaming.Init()
}
cluster.MetaWatcher = &EtcdMetaWatcher{ cluster.MetaWatcher = &EtcdMetaWatcher{
rootPath: etcdConfig.RootPath.GetValue(), rootPath: etcdConfig.RootPath.GetValue(),
@ -389,6 +400,7 @@ func (cluster *MiniClusterV2) StopRootCoord() {
func (cluster *MiniClusterV2) StartRootCoord() { func (cluster *MiniClusterV2) StartRootCoord() {
if cluster.RootCoord == nil { if cluster.RootCoord == nil {
coordclient.ResetRootCoordRegistration()
var err error var err error
if cluster.RootCoord, err = grpcrootcoord.NewServer(cluster.ctx, cluster.factory); err != nil { if cluster.RootCoord, err = grpcrootcoord.NewServer(cluster.ctx, cluster.factory); err != nil {
panic(err) panic(err)
@ -406,6 +418,7 @@ func (cluster *MiniClusterV2) StopDataCoord() {
func (cluster *MiniClusterV2) StartDataCoord() { func (cluster *MiniClusterV2) StartDataCoord() {
if cluster.DataCoord == nil { if cluster.DataCoord == nil {
coordclient.ResetRootCoordRegistration()
var err error var err error
if cluster.DataCoord, err = grpcdatacoord.NewServer(cluster.ctx, cluster.factory); err != nil { if cluster.DataCoord, err = grpcdatacoord.NewServer(cluster.ctx, cluster.factory); err != nil {
panic(err) panic(err)
@ -423,6 +436,7 @@ func (cluster *MiniClusterV2) StopQueryCoord() {
func (cluster *MiniClusterV2) StartQueryCoord() { func (cluster *MiniClusterV2) StartQueryCoord() {
if cluster.QueryCoord == nil { if cluster.QueryCoord == nil {
coordclient.ResetQueryCoordRegistration()
var err error var err error
if cluster.QueryCoord, err = grpcquerycoord.NewServer(cluster.ctx, cluster.factory); err != nil { if cluster.QueryCoord, err = grpcquerycoord.NewServer(cluster.ctx, cluster.factory); err != nil {
panic(err) panic(err)
@ -486,10 +500,6 @@ func (cluster *MiniClusterV2) Stop() error {
cluster.StopAllStreamingNodes() cluster.StopAllStreamingNodes()
cluster.StopAllQueryNodes() cluster.StopAllQueryNodes()
if streamingutil.IsStreamingServiceEnabled() {
streaming.Release()
}
cluster.IndexNode.Stop() cluster.IndexNode.Stop()
log.Info("mini cluster indexNode stopped") log.Info("mini cluster indexNode stopped")
@ -505,6 +515,7 @@ func (cluster *MiniClusterV2) Stop() error {
} }
} }
cluster.ChunkManager.RemoveWithPrefix(cluster.ctx, cluster.ChunkManager.RootPath()) cluster.ChunkManager.RemoveWithPrefix(cluster.ctx, cluster.ChunkManager.RootPath())
streaming.Release()
return nil return nil
} }

View File

@ -46,6 +46,9 @@ type HelloStreamingSuite struct {
func (s *HelloStreamingSuite) SetupSuite() { func (s *HelloStreamingSuite) SetupSuite() {
streamingutil.SetStreamingServiceEnabled() streamingutil.SetStreamingServiceEnabled()
s.MiniClusterSuite.Opt = []integration.OptionV2{
integration.OptionSetParam("mq.type", "pulsar"),
}
s.MiniClusterSuite.SetupSuite() s.MiniClusterSuite.SetupSuite()
} }

View File

@ -30,12 +30,10 @@ import (
"go.uber.org/zap/zapcore" "go.uber.org/zap/zapcore"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
"github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/hookutil"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/etcd"
"github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
) )
var caseTimeout time.Duration var caseTimeout time.Duration
@ -90,6 +88,7 @@ type MiniClusterSuite struct {
EmbedEtcdSuite EmbedEtcdSuite
Cluster *MiniClusterV2 Cluster *MiniClusterV2
Opt []OptionV2
cancelFunc context.CancelFunc cancelFunc context.CancelFunc
} }
@ -110,15 +109,14 @@ func (s *MiniClusterSuite) SetupTest() {
// setup env value to init etcd source // setup env value to init etcd source
s.T().Setenv("etcd.endpoints", val) s.T().Setenv("etcd.endpoints", val)
params = paramtable.Get()
s.T().Log("Setup case timeout", caseTimeout) s.T().Log("Setup case timeout", caseTimeout)
ctx, cancel := context.WithTimeout(context.Background(), caseTimeout) ctx, cancel := context.WithTimeout(context.Background(), caseTimeout)
s.cancelFunc = cancel s.cancelFunc = cancel
c, err := StartMiniClusterV2(ctx, func(c *MiniClusterV2) { opts := append(s.Opt, func(c *MiniClusterV2) {
// change config etcd endpoints // change config etcd endpoints
c.params[params.EtcdCfg.Endpoints.Key] = val c.params[params.EtcdCfg.Endpoints.Key] = val
}) })
c, err := StartMiniClusterV2(ctx, opts...)
s.Require().NoError(err) s.Require().NoError(err)
s.Cluster = c s.Cluster = c
@ -170,5 +168,4 @@ func (s *MiniClusterSuite) TearDownTest() {
if s.Cluster != nil { if s.Cluster != nil {
s.Cluster.Stop() s.Cluster.Stop()
} }
registry.ResetRegistration()
} }