From 7c575a18b046ffd055bd7da83675744914e22719 Mon Sep 17 00:00:00 2001 From: Zhen Ye Date: Wed, 17 Dec 2025 16:55:17 +0800 Subject: [PATCH] enhance: support AckSyncUp for broadcaster, and enable it in truncate api (#46313) issue: #43897 also for issue: #46166 add ack_sync_up flag into broadcast message header, which indicates that whether the broadcast operation is need to be synced up between the streaming node and the coordinator. If the ack_sync_up is false, the broadcast operation will be acked once the recovery storage see the message at current vchannel, the fast ack operation can be applied to speed up the broadcast operation. If the ack_sync_up is true, the broadcast operation will be acked after the checkpoint of current vchannel reach current message. The fast ack operation can not be applied to speed up the broadcast operation, because the ack operation need to be synced up with streaming node. e.g. if truncate collection operation want to call ack once callback after the all segment are flushed at current vchannel, it should set the ack_sync_up to be true. TODO: current implementation doesn't promise the ack sync up semantic, it only promise FastAck operation will not be applied, wait for 3.0 to implement the ack sync up semantic. only for truncate api now. --------- Signed-off-by: chyezh --- internal/datacoord/handler.go | 1 + internal/datacoord/server_test.go | 5 +- internal/datacoord/util.go | 4 + .../distributed/streaming/streaming_test.go | 2 +- internal/metastore/kv/rootcoord/kv_catalog.go | 1 + internal/metastore/model/collection.go | 254 +++++++----- internal/metastore/model/collection_test.go | 19 + internal/proxy/task_import.go | 3 +- internal/rootcoord/ddl_callbacks.go | 2 + .../ddl_callbacks_collection_test.go | 4 + .../ddl_callbacks_create_collection.go | 15 +- .../ddl_callbacks_truncate_collection.go | 52 ++- internal/rootcoord/meta_table.go | 61 +++ internal/rootcoord/meta_table_test.go | 92 +++++ internal/rootcoord/mocks/meta_table.go | 94 +++++ internal/rootcoord/root_coord_test.go | 18 +- .../client/broadcast/broadcast_test.go | 2 +- .../server/broadcaster/broadcast_task.go | 18 +- .../server/broadcaster/broadcaster_test.go | 6 +- .../registry/ack_message_callback.go | 1 + .../registry/ack_once_message_callback.go | 83 ++++ .../registry/specialized_ack_once_callback.go | 31 ++ .../registry/specialized_callback.go | 1 + .../broadcaster/registry/test_utility.go | 1 + .../server/service/broadcast_test.go | 2 +- .../server/flusher/flusherimpl/wal_flusher.go | 19 +- pkg/common/common.go | 3 + pkg/proto/etcd_meta.proto | 6 +- pkg/proto/etcdpb/etcd_meta.pb.go | 388 ++++++++++-------- pkg/proto/messages.proto | 7 + pkg/proto/messagespb/messages.pb.go | 17 +- pkg/streaming/util/message/broadcast.go | 7 + pkg/streaming/util/message/builder.go | 29 +- pkg/streaming/util/message/builder_test.go | 10 +- pkg/streaming/util/message/codegen/main.go | 1 + .../util/message/marshal_log_object.go | 7 + pkg/streaming/util/message/message_test.go | 6 +- pkg/streaming/util/message/reflect_info.go | 42 ++ 38 files changed, 1000 insertions(+), 314 deletions(-) create mode 100644 internal/streamingcoord/server/broadcaster/registry/ack_once_message_callback.go create mode 100644 internal/streamingcoord/server/broadcaster/registry/specialized_ack_once_callback.go diff --git a/internal/datacoord/handler.go b/internal/datacoord/handler.go index fd105d51b0..9325625a34 100644 --- a/internal/datacoord/handler.go +++ b/internal/datacoord/handler.go @@ -565,6 +565,7 @@ func (h *ServerHandler) GetCollection(ctx context.Context, collectionID UniqueID return nil, err } + // TODO: the cache should be removed in next step. return h.s.meta.GetCollection(collectionID), nil } diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 3b71ff43fd..9015cbd4c0 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -2582,7 +2582,6 @@ func TestServer_InitMessageCallback(t *testing.T) { server.initMessageCallback() // Test Import message check callback - resourceKey := message.NewImportJobIDResourceKey(1) msg, err := message.NewImportMessageBuilderV1(). WithHeader(&message.ImportMessageHeader{}). WithBody(&msgpb.ImportMsg{ @@ -2591,7 +2590,7 @@ func TestServer_InitMessageCallback(t *testing.T) { }, Schema: &schemapb.CollectionSchema{}, }). - WithBroadcast([]string{"ch-0"}, resourceKey). + WithBroadcast([]string{"ch-0"}). BuildBroadcast() err = registry.CallMessageCheckCallback(ctx, msg) assert.NoError(t, err) @@ -2605,7 +2604,7 @@ func TestServer_InitMessageCallback(t *testing.T) { }, Schema: &schemapb.CollectionSchema{}, }). - WithBroadcast([]string{"test_channel"}, resourceKey). + WithBroadcast([]string{"test_channel"}). MustBuildBroadcast() err = registry.CallMessageAckCallback(ctx, importMsg, map[string]*message.AppendResult{ "test_channel": { diff --git a/internal/datacoord/util.go b/internal/datacoord/util.go index f9f359c6b9..4a9bf35898 100644 --- a/internal/datacoord/util.go +++ b/internal/datacoord/util.go @@ -197,6 +197,10 @@ func getCompactedSegmentSize(s *datapb.CompactionSegment) int64 { // getCollectionAutoCompactionEnabled returns whether auto compaction for collection is enabled. // if not set, returns global auto compaction config. func getCollectionAutoCompactionEnabled(properties map[string]string) (bool, error) { + // when collection is on truncating, disable auto compaction. + if _, ok := properties[common.CollectionOnTruncatingKey]; ok { + return false, nil + } v, ok := properties[common.CollectionAutoCompactionKey] if ok { enabled, err := strconv.ParseBool(v) diff --git a/internal/distributed/streaming/streaming_test.go b/internal/distributed/streaming/streaming_test.go index 0ecfb30c73..702c0a1f12 100644 --- a/internal/distributed/streaming/streaming_test.go +++ b/internal/distributed/streaming/streaming_test.go @@ -147,7 +147,7 @@ func TestStreamingBroadcast(t *testing.T) { CollectionID: 1, CollectionName: collectionName, }). - WithBroadcast(vChannels, message.NewExclusiveCollectionNameResourceKey("db", collectionName)). + WithBroadcast(vChannels). BuildBroadcast() resp, err := streaming.WAL().Broadcast().Append(context.Background(), msg) diff --git a/internal/metastore/kv/rootcoord/kv_catalog.go b/internal/metastore/kv/rootcoord/kv_catalog.go index c4eb207780..3bd337f68e 100644 --- a/internal/metastore/kv/rootcoord/kv_catalog.go +++ b/internal/metastore/kv/rootcoord/kv_catalog.go @@ -698,6 +698,7 @@ func (kc *Catalog) alterModifyCollection(ctx context.Context, oldColl *model.Col oldCollClone.UpdateTimestamp = newColl.UpdateTimestamp oldCollClone.EnableDynamicField = newColl.EnableDynamicField oldCollClone.SchemaVersion = newColl.SchemaVersion + oldCollClone.ShardInfos = newColl.ShardInfos newKey := BuildCollectionKey(newColl.DBID, oldColl.CollectionID) value, err := proto.Marshal(model.MarshalCollectionModel(oldCollClone)) diff --git a/internal/metastore/model/collection.go b/internal/metastore/model/collection.go index 4899c83b74..b3d076918f 100644 --- a/internal/metastore/model/collection.go +++ b/internal/metastore/model/collection.go @@ -28,30 +28,36 @@ import ( // TODO: These collection is dirty implementation and easy to be broken, we should drop it in the future. type Collection struct { - TenantID string - DBID int64 - CollectionID int64 - Partitions []*Partition - Name string - DBName string - Description string - AutoID bool - Fields []*Field - StructArrayFields []*StructArrayField - Functions []*Function - VirtualChannelNames []string - PhysicalChannelNames []string - ShardsNum int32 - StartPositions []*commonpb.KeyDataPair - CreateTime uint64 - ConsistencyLevel commonpb.ConsistencyLevel - Aliases []string // TODO: deprecate this. - Properties []*commonpb.KeyValuePair - State pb.CollectionState - EnableDynamicField bool - UpdateTimestamp uint64 - SchemaVersion int32 - LastTruncateTimestamps map[string]uint64 + TenantID string + DBID int64 + CollectionID int64 + Partitions []*Partition + Name string + DBName string + Description string + AutoID bool + Fields []*Field + StructArrayFields []*StructArrayField + Functions []*Function + VirtualChannelNames []string + PhysicalChannelNames []string + ShardsNum int32 + StartPositions []*commonpb.KeyDataPair + CreateTime uint64 + ConsistencyLevel commonpb.ConsistencyLevel + Aliases []string // TODO: deprecate this. + Properties []*commonpb.KeyValuePair + State pb.CollectionState + EnableDynamicField bool + UpdateTimestamp uint64 + SchemaVersion int32 + ShardInfos map[string]*ShardInfo +} + +type ShardInfo struct { + PChannelName string // the pchannel name of the shard, it is the same with the physical channel name. + VChannelName string // the vchannel name of the shard, it is the same with the virtual channel name. + LastTruncateTimeTick uint64 // the last truncate time tick of the shard, if the shard is not truncated, the value is 0. } func (c *Collection) Available() bool { @@ -60,59 +66,67 @@ func (c *Collection) Available() bool { func (c *Collection) ShallowClone() *Collection { return &Collection{ - TenantID: c.TenantID, - DBID: c.DBID, - CollectionID: c.CollectionID, - Name: c.Name, - DBName: c.DBName, - Description: c.Description, - AutoID: c.AutoID, - Fields: c.Fields, - StructArrayFields: c.StructArrayFields, - Partitions: c.Partitions, - VirtualChannelNames: c.VirtualChannelNames, - PhysicalChannelNames: c.PhysicalChannelNames, - ShardsNum: c.ShardsNum, - ConsistencyLevel: c.ConsistencyLevel, - CreateTime: c.CreateTime, - StartPositions: c.StartPositions, - Aliases: c.Aliases, - Properties: c.Properties, - State: c.State, - EnableDynamicField: c.EnableDynamicField, - Functions: c.Functions, - UpdateTimestamp: c.UpdateTimestamp, - SchemaVersion: c.SchemaVersion, - LastTruncateTimestamps: c.LastTruncateTimestamps, + TenantID: c.TenantID, + DBID: c.DBID, + CollectionID: c.CollectionID, + Name: c.Name, + DBName: c.DBName, + Description: c.Description, + AutoID: c.AutoID, + Fields: c.Fields, + StructArrayFields: c.StructArrayFields, + Partitions: c.Partitions, + VirtualChannelNames: c.VirtualChannelNames, + PhysicalChannelNames: c.PhysicalChannelNames, + ShardsNum: c.ShardsNum, + ConsistencyLevel: c.ConsistencyLevel, + CreateTime: c.CreateTime, + StartPositions: c.StartPositions, + Aliases: c.Aliases, + Properties: c.Properties, + State: c.State, + EnableDynamicField: c.EnableDynamicField, + Functions: c.Functions, + UpdateTimestamp: c.UpdateTimestamp, + SchemaVersion: c.SchemaVersion, + ShardInfos: c.ShardInfos, } } func (c *Collection) Clone() *Collection { + shardInfos := make(map[string]*ShardInfo, len(c.ShardInfos)) + for channelName, shardInfo := range c.ShardInfos { + shardInfos[channelName] = &ShardInfo{ + VChannelName: channelName, + PChannelName: shardInfo.PChannelName, + LastTruncateTimeTick: shardInfo.LastTruncateTimeTick, + } + } return &Collection{ - TenantID: c.TenantID, - DBID: c.DBID, - CollectionID: c.CollectionID, - Name: c.Name, - DBName: c.DBName, - Description: c.Description, - AutoID: c.AutoID, - Fields: CloneFields(c.Fields), - StructArrayFields: CloneStructArrayFields(c.StructArrayFields), - Partitions: ClonePartitions(c.Partitions), - VirtualChannelNames: common.CloneStringList(c.VirtualChannelNames), - PhysicalChannelNames: common.CloneStringList(c.PhysicalChannelNames), - ShardsNum: c.ShardsNum, - ConsistencyLevel: c.ConsistencyLevel, - CreateTime: c.CreateTime, - StartPositions: common.CloneKeyDataPairs(c.StartPositions), - Aliases: common.CloneStringList(c.Aliases), - Properties: common.CloneKeyValuePairs(c.Properties), - State: c.State, - EnableDynamicField: c.EnableDynamicField, - Functions: CloneFunctions(c.Functions), - UpdateTimestamp: c.UpdateTimestamp, - SchemaVersion: c.SchemaVersion, - LastTruncateTimestamps: common.CloneMap(c.LastTruncateTimestamps), + TenantID: c.TenantID, + DBID: c.DBID, + CollectionID: c.CollectionID, + Name: c.Name, + DBName: c.DBName, + Description: c.Description, + AutoID: c.AutoID, + Fields: CloneFields(c.Fields), + StructArrayFields: CloneStructArrayFields(c.StructArrayFields), + Partitions: ClonePartitions(c.Partitions), + VirtualChannelNames: common.CloneStringList(c.VirtualChannelNames), + PhysicalChannelNames: common.CloneStringList(c.PhysicalChannelNames), + ShardsNum: c.ShardsNum, + ConsistencyLevel: c.ConsistencyLevel, + CreateTime: c.CreateTime, + StartPositions: common.CloneKeyDataPairs(c.StartPositions), + Aliases: common.CloneStringList(c.Aliases), + Properties: common.CloneKeyValuePairs(c.Properties), + State: c.State, + EnableDynamicField: c.EnableDynamicField, + Functions: CloneFunctions(c.Functions), + UpdateTimestamp: c.UpdateTimestamp, + SchemaVersion: c.SchemaVersion, + ShardInfos: shardInfos, } } @@ -179,29 +193,45 @@ func UnmarshalCollectionModel(coll *pb.CollectionInfo) *Collection { PartitionCreatedTimestamp: coll.PartitionCreatedTimestamps[idx], } } + shardInfos := make(map[string]*ShardInfo, len(coll.VirtualChannelNames)) + for idx, channelName := range coll.VirtualChannelNames { + if len(coll.ShardInfos) == 0 { + shardInfos[channelName] = &ShardInfo{ + VChannelName: channelName, + PChannelName: coll.PhysicalChannelNames[idx], + LastTruncateTimeTick: 0, + } + } else { + shardInfos[channelName] = &ShardInfo{ + VChannelName: channelName, + PChannelName: coll.PhysicalChannelNames[idx], + LastTruncateTimeTick: coll.ShardInfos[idx].LastTruncateTimeTick, + } + } + } return &Collection{ - CollectionID: coll.ID, - DBID: coll.DbId, - Name: coll.Schema.Name, - DBName: coll.Schema.DbName, - Description: coll.Schema.Description, - AutoID: coll.Schema.AutoID, - Fields: UnmarshalFieldModels(coll.GetSchema().GetFields()), - StructArrayFields: UnmarshalStructArrayFieldModels(coll.GetSchema().GetStructArrayFields()), - Partitions: partitions, - VirtualChannelNames: coll.VirtualChannelNames, - PhysicalChannelNames: coll.PhysicalChannelNames, - ShardsNum: coll.ShardsNum, - ConsistencyLevel: coll.ConsistencyLevel, - CreateTime: coll.CreateTime, - StartPositions: coll.StartPositions, - State: coll.State, - Properties: coll.Properties, - EnableDynamicField: coll.Schema.EnableDynamicField, - UpdateTimestamp: coll.UpdateTimestamp, - SchemaVersion: coll.Schema.Version, - LastTruncateTimestamps: coll.LastTruncateTimestamps, + CollectionID: coll.ID, + DBID: coll.DbId, + Name: coll.Schema.Name, + DBName: coll.Schema.DbName, + Description: coll.Schema.Description, + AutoID: coll.Schema.AutoID, + Fields: UnmarshalFieldModels(coll.GetSchema().GetFields()), + StructArrayFields: UnmarshalStructArrayFieldModels(coll.GetSchema().GetStructArrayFields()), + Partitions: partitions, + VirtualChannelNames: coll.VirtualChannelNames, + PhysicalChannelNames: coll.PhysicalChannelNames, + ShardsNum: coll.ShardsNum, + ConsistencyLevel: coll.ConsistencyLevel, + CreateTime: coll.CreateTime, + StartPositions: coll.StartPositions, + State: coll.State, + Properties: coll.Properties, + EnableDynamicField: coll.Schema.EnableDynamicField, + UpdateTimestamp: coll.UpdateTimestamp, + SchemaVersion: coll.Schema.Version, + ShardInfos: shardInfos, } } @@ -265,20 +295,32 @@ func marshalCollectionModelWithConfig(coll *Collection, c *config) *pb.Collectio collSchema.StructArrayFields = structArrayFields } + shardInfos := make([]*pb.CollectionShardInfo, len(coll.ShardInfos)) + for idx, channelName := range coll.VirtualChannelNames { + if shard, ok := coll.ShardInfos[channelName]; ok { + shardInfos[idx] = &pb.CollectionShardInfo{ + LastTruncateTimeTick: shard.LastTruncateTimeTick, + } + } else { + shardInfos[idx] = &pb.CollectionShardInfo{ + LastTruncateTimeTick: 0, + } + } + } collectionPb := &pb.CollectionInfo{ - ID: coll.CollectionID, - DbId: coll.DBID, - Schema: collSchema, - CreateTime: coll.CreateTime, - VirtualChannelNames: coll.VirtualChannelNames, - PhysicalChannelNames: coll.PhysicalChannelNames, - ShardsNum: coll.ShardsNum, - ConsistencyLevel: coll.ConsistencyLevel, - StartPositions: coll.StartPositions, - State: coll.State, - Properties: coll.Properties, - UpdateTimestamp: coll.UpdateTimestamp, - LastTruncateTimestamps: coll.LastTruncateTimestamps, + ID: coll.CollectionID, + DbId: coll.DBID, + Schema: collSchema, + CreateTime: coll.CreateTime, + VirtualChannelNames: coll.VirtualChannelNames, + PhysicalChannelNames: coll.PhysicalChannelNames, + ShardsNum: coll.ShardsNum, + ConsistencyLevel: coll.ConsistencyLevel, + StartPositions: coll.StartPositions, + State: coll.State, + Properties: coll.Properties, + UpdateTimestamp: coll.UpdateTimestamp, + ShardInfos: shardInfos, } if c.withPartitions { diff --git a/internal/metastore/model/collection_test.go b/internal/metastore/model/collection_test.go index 5dfeef1c9a..27b3828c81 100644 --- a/internal/metastore/model/collection_test.go +++ b/internal/metastore/model/collection_test.go @@ -77,6 +77,13 @@ var ( Value: "v", }, }, + ShardInfos: map[string]*ShardInfo{ + "vch": { + PChannelName: "pch", + VChannelName: "vch", + LastTruncateTimeTick: 0, + }, + }, } deprecatedColPb = &pb.CollectionInfo{ @@ -588,6 +595,18 @@ func TestClone(t *testing.T) { Properties: []*commonpb.KeyValuePair{{Key: "32", Value: "33"}}, State: pb.CollectionState_CollectionCreated, EnableDynamicField: true, + ShardInfos: map[string]*ShardInfo{ + "c1": { + PChannelName: "c3", + VChannelName: "c1", + LastTruncateTimeTick: 0, + }, + "c2": { + PChannelName: "c4", + VChannelName: "c2", + LastTruncateTimeTick: 0, + }, + }, } clone1 := collection.Clone() diff --git a/internal/proxy/task_import.go b/internal/proxy/task_import.go index 087b27e539..80b4b444e6 100644 --- a/internal/proxy/task_import.go +++ b/internal/proxy/task_import.go @@ -214,7 +214,6 @@ func (it *importTask) Execute(ctx context.Context) error { log.Ctx(ctx).Warn("alloc job id failed", zap.Error(err)) return err } - resourceKey := message.NewImportJobIDResourceKey(jobID) msg, err := message.NewImportMessageBuilderV1(). WithHeader(&message.ImportMessageHeader{}).WithBody( &msgpb.ImportMsg{ @@ -231,7 +230,7 @@ func (it *importTask) Execute(ctx context.Context) error { Schema: it.schema.CollectionSchema, JobID: jobID, }). - WithBroadcast(it.vchannels, resourceKey). + WithBroadcast(it.vchannels). BuildBroadcast() if err != nil { log.Ctx(ctx).Warn("create import message failed", zap.Error(err)) diff --git a/internal/rootcoord/ddl_callbacks.go b/internal/rootcoord/ddl_callbacks.go index 872bbdbb7a..4d7aa39038 100644 --- a/internal/rootcoord/ddl_callbacks.go +++ b/internal/rootcoord/ddl_callbacks.go @@ -78,7 +78,9 @@ func (c *DDLCallback) registerCollectionCallbacks() { registry.RegisterCreateCollectionV1AckCallback(c.createCollectionV1AckCallback) registry.RegisterAlterCollectionV2AckCallback(c.alterCollectionV2AckCallback) registry.RegisterDropCollectionV1AckCallback(c.dropCollectionV1AckCallback) + registry.RegisterTruncateCollectionV2AckCallback(c.truncateCollectionV2AckCallback) + registry.RegisterTruncateCollectionV2AckOnceCallback(c.truncateCollectionV2AckOnceCallback) } // registerPartitionCallbacks registers the partition callbacks. diff --git a/internal/rootcoord/ddl_callbacks_collection_test.go b/internal/rootcoord/ddl_callbacks_collection_test.go index 636713df02..f7a8bd765c 100644 --- a/internal/rootcoord/ddl_callbacks_collection_test.go +++ b/internal/rootcoord/ddl_callbacks_collection_test.go @@ -151,6 +151,10 @@ func TestDDLCallbacksCollectionDDL(t *testing.T) { coll, err = core.meta.GetCollectionByName(ctx, dbName, collectionName, typeutil.MaxTimestamp) require.NoError(t, err) require.Equal(t, coll.Name, collectionName) + require.Equal(t, 1, len(coll.ShardInfos)) + for _, shardInfo := range coll.ShardInfos { + require.Greater(t, shardInfo.LastTruncateTimeTick, uint64(0)) + } // Test DropCollection // drop the collection should be ok. diff --git a/internal/rootcoord/ddl_callbacks_create_collection.go b/internal/rootcoord/ddl_callbacks_create_collection.go index 481d6737a3..e27e4fe427 100644 --- a/internal/rootcoord/ddl_callbacks_create_collection.go +++ b/internal/rootcoord/ddl_callbacks_create_collection.go @@ -86,10 +86,7 @@ func (c *Core) broadcastCreateCollectionV1(ctx context.Context, req *milvuspb.Cr msg := message.NewCreateCollectionMessageBuilderV1(). WithHeader(createCollectionTask.header). WithBody(createCollectionTask.body). - WithBroadcast(broadcastChannel, - message.NewSharedDBNameResourceKey(createCollectionTask.body.DbName), - message.NewExclusiveCollectionNameResourceKey(createCollectionTask.body.DbName, createCollectionTask.body.CollectionName), - ). + WithBroadcast(broadcastChannel). MustBuildBroadcast() if _, err := broadcaster.Broadcast(ctx, msg); err != nil { return err @@ -177,6 +174,14 @@ func newCollectionModel(header *message.CreateCollectionMessageHeader, body *mes }) } consistencyLevel, properties := mustConsumeConsistencyLevel(body.CollectionSchema.Properties) + shardInfos := make(map[string]*model.ShardInfo, len(body.VirtualChannelNames)) + for idx, vchannel := range body.VirtualChannelNames { + shardInfos[vchannel] = &model.ShardInfo{ + VChannelName: vchannel, + PChannelName: body.PhysicalChannelNames[idx], + LastTruncateTimeTick: 0, + } + } return &model.Collection{ CollectionID: header.CollectionId, DBID: header.DbId, @@ -197,6 +202,8 @@ func newCollectionModel(header *message.CreateCollectionMessageHeader, body *mes Properties: properties, EnableDynamicField: body.CollectionSchema.EnableDynamicField, UpdateTimestamp: ts, + SchemaVersion: 0, + ShardInfos: shardInfos, } } diff --git a/internal/rootcoord/ddl_callbacks_truncate_collection.go b/internal/rootcoord/ddl_callbacks_truncate_collection.go index 0bb9242ae7..dad7415d00 100644 --- a/internal/rootcoord/ddl_callbacks_truncate_collection.go +++ b/internal/rootcoord/ddl_callbacks_truncate_collection.go @@ -19,8 +19,11 @@ package rootcoord import ( "context" + "github.com/cockroachdb/errors" + "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/proto/messagespb" "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" "github.com/milvus-io/milvus/pkg/v2/util/funcutil" @@ -52,7 +55,7 @@ func (c *Core) broadcastTruncateCollection(ctx context.Context, req *milvuspb.Tr msg := message.NewTruncateCollectionMessageBuilderV2(). WithHeader(header). WithBody(body). - WithBroadcast(channels). + WithBroadcast(channels, message.OptBuildBroadcastAckSyncUp()). MustBuildBroadcast() if _, err := broadcaster.Broadcast(ctx, msg); err != nil { return err @@ -75,16 +78,51 @@ func (c *DDLCallback) truncateCollectionV2AckCallback(ctx context.Context, resul } // Drop segments that were updated before the flush timestamp - err := c.mixCoord.DropSegmentsByTime(ctx, header.CollectionId, flushTsList) - if err != nil { - return err + if err := c.mixCoord.DropSegmentsByTime(ctx, header.CollectionId, flushTsList); err != nil { + return errors.Wrap(err, "when dropping segments by time") } // manually update current target to sync QueryCoord's view - err = c.mixCoord.ManualUpdateCurrentTarget(ctx, header.CollectionId) - if err != nil { - return err + if err := c.mixCoord.ManualUpdateCurrentTarget(ctx, header.CollectionId); err != nil { + return errors.Wrap(err, "when manually updating current target") } + if err := c.meta.TruncateCollection(ctx, result); err != nil { + if errors.Is(err, errAlterCollectionNotFound) { + log.Ctx(ctx).Warn("truncate a non-existent collection, ignore it", log.FieldMessage(result.Message)) + return nil + } + return errors.Wrap(err, "when truncating collection") + } + + // notify datacoord to update their meta cache + if err := c.broker.BroadcastAlteredCollection(ctx, header.CollectionId); err != nil { + return errors.Wrap(err, "when broadcasting altered collection") + } + return nil +} + +// truncateCollectionV2AckOnceCallback is called when the truncate collection message is acknowledged once +func (c *DDLCallback) truncateCollectionV2AckOnceCallback(ctx context.Context, result message.AckResultTruncateCollectionMessageV2) error { + msg := result.Message + // When the ack callback of truncate collection operation is executing, + // the compaction and flush operation may be executed in parallel. + // So if some vchannel flush a new segment which order after the truncate collection operation, + // the segment should not be dropped, but the compaction may compact it with the segment which order before the truncate collection operation. + // the new compacted segment can not be dropped as whole, break the design of truncate collection operation. + // we need to forbid the compaction of current collection here. + collectionID := msg.Header().CollectionId + if err := c.meta.BeginTruncateCollection(ctx, collectionID); err != nil { + if errors.Is(err, errAlterCollectionNotFound) { + log.Ctx(ctx).Warn("begin to truncate a non-existent collection, ignore it", log.FieldMessage(result.Message)) + return nil + } + return errors.Wrap(err, "when beginning truncate collection") + } + + // notify datacoord to update their meta cache + if err := c.broker.BroadcastAlteredCollection(ctx, collectionID); err != nil { + return errors.Wrap(err, "when broadcasting altered collection") + } return nil } diff --git a/internal/rootcoord/meta_table.go b/internal/rootcoord/meta_table.go index 31f001e502..99baa9367d 100644 --- a/internal/rootcoord/meta_table.go +++ b/internal/rootcoord/meta_table.go @@ -113,6 +113,11 @@ type IMetaTable interface { ListAliases(ctx context.Context, dbName string, collectionName string, ts Timestamp) ([]string, error) AlterCollection(ctx context.Context, result message.BroadcastResultAlterCollectionMessageV2) error + // Deprecated: will be removed in the 3.0 after implementing ack sync up semantic. + // It will be used to forbid the compaction of current collection when truncate collection operation is in progress. + BeginTruncateCollection(ctx context.Context, collectionID UniqueID) error + // TruncateCollection is called when the truncate collection message is acknowledged. + TruncateCollection(ctx context.Context, result message.BroadcastResultTruncateCollectionMessageV2) error CheckIfCollectionRenamable(ctx context.Context, dbName string, oldName string, newDBName string, newName string) error GetGeneralCount(ctx context.Context) int @@ -974,6 +979,62 @@ func (mt *MetaTable) AlterCollection(ctx context.Context, result message.Broadca return nil } +func (mt *MetaTable) BeginTruncateCollection(ctx context.Context, collectionID UniqueID) error { + mt.ddLock.Lock() + defer mt.ddLock.Unlock() + + coll, ok := mt.collID2Meta[collectionID] + if !ok { + return errAlterCollectionNotFound + } + + // Apply the properties to override the existing properties. + newProperties := common.CloneKeyValuePairs(coll.Properties).ToMap() + key := common.CollectionOnTruncatingKey + if _, ok := newProperties[key]; ok && newProperties[key] == "1" { + return nil + } + newProperties[key] = "1" + oldColl := coll.Clone() + newColl := coll.Clone() + newColl.Properties = common.NewKeyValuePairs(newProperties) + + ctx1 := contextutil.WithTenantID(ctx, Params.CommonCfg.ClusterName.GetValue()) + if err := mt.catalog.AlterCollection(ctx1, oldColl, newColl, metastore.MODIFY, newColl.UpdateTimestamp, false); err != nil { + return err + } + mt.collID2Meta[coll.CollectionID] = newColl + return nil +} + +func (mt *MetaTable) TruncateCollection(ctx context.Context, result message.BroadcastResultTruncateCollectionMessageV2) error { + mt.ddLock.Lock() + defer mt.ddLock.Unlock() + + collectionID := result.Message.Header().CollectionId + coll, ok := mt.collID2Meta[collectionID] + if !ok { + return errAlterCollectionNotFound + } + + oldColl := coll.Clone() + + // remmove the truncating key from the properties and update the last truncate time tick of the shard infos + newColl := coll.Clone() + newProperties := common.CloneKeyValuePairs(coll.Properties).ToMap() + delete(newProperties, common.CollectionOnTruncatingKey) + newColl.Properties = common.NewKeyValuePairs(newProperties) + for vchannel := range newColl.ShardInfos { + newColl.ShardInfos[vchannel].LastTruncateTimeTick = result.Results[vchannel].TimeTick + } + ctx1 := contextutil.WithTenantID(ctx, Params.CommonCfg.ClusterName.GetValue()) + if err := mt.catalog.AlterCollection(ctx1, oldColl, newColl, metastore.MODIFY, newColl.UpdateTimestamp, false); err != nil { + return err + } + mt.collID2Meta[coll.CollectionID] = newColl + return nil +} + func (mt *MetaTable) CheckIfCollectionRenamable(ctx context.Context, dbName string, oldName string, newDBName string, newName string) error { mt.ddLock.RLock() defer mt.ddLock.RUnlock() diff --git a/internal/rootcoord/meta_table_test.go b/internal/rootcoord/meta_table_test.go index 489455e4dc..775a179d37 100644 --- a/internal/rootcoord/meta_table_test.go +++ b/internal/rootcoord/meta_table_test.go @@ -35,6 +35,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/channel" mocktso "github.com/milvus-io/milvus/internal/tso/mocks" kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" + "github.com/milvus-io/milvus/pkg/v2/common" pb "github.com/milvus-io/milvus/pkg/v2/proto/etcdpb" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" @@ -2288,3 +2289,94 @@ func TestMetaTable_PrivilegeGroup(t *testing.T) { _, err = mt.ListPrivilegeGroups(context.TODO()) assert.NoError(t, err) } + +func TestMetaTable_TruncateCollection(t *testing.T) { + channel.ResetStaticPChannelStatsManager() + + kv, _ := kvfactory.GetEtcdAndPath() + path := funcutil.RandomString(10) + "/meta" + catalogKV := etcdkv.NewEtcdKV(kv, path) + ss, err := rootcoord.NewSuffixSnapshot(catalogKV, rootcoord.SnapshotsSep, path, rootcoord.SnapshotPrefix) + require.NoError(t, err) + catalog := rootcoord.NewCatalog(catalogKV, ss) + + allocator := mocktso.NewAllocator(t) + allocator.EXPECT().GenerateTSO(mock.Anything).Return(1000, nil) + + meta, err := NewMetaTable(context.Background(), catalog, allocator) + require.NoError(t, err) + + err = meta.AddCollection(context.Background(), &model.Collection{ + CollectionID: 1, + PhysicalChannelNames: []string{"pchannel1"}, + VirtualChannelNames: []string{"vchannel1"}, + State: pb.CollectionState_CollectionCreated, + DBID: util.DefaultDBID, + Properties: common.NewKeyValuePairs(map[string]string{}), + ShardInfos: map[string]*model.ShardInfo{ + "vchannel1": { + VChannelName: "vchannel1", + PChannelName: "pchannel1", + LastTruncateTimeTick: 0, + }, + }, + }) + require.NoError(t, err) + + // begin truncate collection + err = meta.BeginTruncateCollection(context.Background(), 1) + require.NoError(t, err) + coll, err := meta.GetCollectionByID(context.Background(), util.DefaultDBName, 1, typeutil.MaxTimestamp, false) + require.NoError(t, err) + m := common.CloneKeyValuePairs(coll.Properties).ToMap() + require.Equal(t, "1", m[common.CollectionOnTruncatingKey]) + require.Equal(t, uint64(0), coll.ShardInfos["vchannel1"].LastTruncateTimeTick) + + // reload the meta + channel.ResetStaticPChannelStatsManager() + meta, err = NewMetaTable(context.Background(), catalog, allocator) + require.NoError(t, err) + coll, err = meta.GetCollectionByID(context.Background(), util.DefaultDBName, 1, typeutil.MaxTimestamp, false) + require.NoError(t, err) + m = common.CloneKeyValuePairs(coll.Properties).ToMap() + require.Equal(t, "1", m[common.CollectionOnTruncatingKey]) + require.Equal(t, uint64(0), coll.ShardInfos["vchannel1"].LastTruncateTimeTick) + + // remove the temp property + b := message.NewTruncateCollectionMessageBuilderV2(). + WithHeader(&message.TruncateCollectionMessageHeader{ + CollectionId: 1, + }). + WithBody(&message.TruncateCollectionMessageBody{}). + WithBroadcast(coll.VirtualChannelNames, message.OptBuildBroadcastAckSyncUp()). + MustBuildBroadcast() + + meta.TruncateCollection(context.Background(), message.BroadcastResultTruncateCollectionMessageV2{ + Message: message.MustAsBroadcastTruncateCollectionMessageV2(b), + Results: map[string]*message.AppendResult{ + "vchannel1": { + TimeTick: 1000, + }, + }, + }) + + require.NoError(t, err) + coll, err = meta.GetCollectionByID(context.Background(), util.DefaultDBName, 1, typeutil.MaxTimestamp, false) + require.NoError(t, err) + m = common.CloneKeyValuePairs(coll.Properties).ToMap() + _, ok := m[common.CollectionOnTruncatingKey] + require.False(t, ok) + require.Equal(t, uint64(1000), coll.ShardInfos["vchannel1"].LastTruncateTimeTick) + + // reload the meta again + channel.ResetStaticPChannelStatsManager() + meta, err = NewMetaTable(context.Background(), catalog, allocator) + require.NoError(t, err) + coll, err = meta.GetCollectionByID(context.Background(), util.DefaultDBName, 1, typeutil.MaxTimestamp, false) + require.NoError(t, err) + m = common.CloneKeyValuePairs(coll.Properties).ToMap() + _, ok = m[common.CollectionOnTruncatingKey] + require.False(t, ok) + require.Equal(t, 1, len(coll.ShardInfos)) + require.Equal(t, uint64(1000), coll.ShardInfos["vchannel1"].LastTruncateTimeTick) +} diff --git a/internal/rootcoord/mocks/meta_table.go b/internal/rootcoord/mocks/meta_table.go index aa9b4e9543..2fbe0fc897 100644 --- a/internal/rootcoord/mocks/meta_table.go +++ b/internal/rootcoord/mocks/meta_table.go @@ -374,6 +374,53 @@ func (_c *IMetaTable_BackupRBAC_Call) RunAndReturn(run func(context.Context, str return _c } +// BeginTruncateCollection provides a mock function with given fields: ctx, collectionID +func (_m *IMetaTable) BeginTruncateCollection(ctx context.Context, collectionID int64) error { + ret := _m.Called(ctx, collectionID) + + if len(ret) == 0 { + panic("no return value specified for BeginTruncateCollection") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, int64) error); ok { + r0 = rf(ctx, collectionID) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// IMetaTable_BeginTruncateCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BeginTruncateCollection' +type IMetaTable_BeginTruncateCollection_Call struct { + *mock.Call +} + +// BeginTruncateCollection is a helper method to define mock.On call +// - ctx context.Context +// - collectionID int64 +func (_e *IMetaTable_Expecter) BeginTruncateCollection(ctx interface{}, collectionID interface{}) *IMetaTable_BeginTruncateCollection_Call { + return &IMetaTable_BeginTruncateCollection_Call{Call: _e.mock.On("BeginTruncateCollection", ctx, collectionID)} +} + +func (_c *IMetaTable_BeginTruncateCollection_Call) Run(run func(ctx context.Context, collectionID int64)) *IMetaTable_BeginTruncateCollection_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(int64)) + }) + return _c +} + +func (_c *IMetaTable_BeginTruncateCollection_Call) Return(_a0 error) *IMetaTable_BeginTruncateCollection_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *IMetaTable_BeginTruncateCollection_Call) RunAndReturn(run func(context.Context, int64) error) *IMetaTable_BeginTruncateCollection_Call { + _c.Call.Return(run) + return _c +} + // CheckIfAddCredential provides a mock function with given fields: ctx, req func (_m *IMetaTable) CheckIfAddCredential(ctx context.Context, req *internalpb.CredentialInfo) error { ret := _m.Called(ctx, req) @@ -3564,6 +3611,53 @@ func (_c *IMetaTable_SelectUser_Call) RunAndReturn(run func(context.Context, str return _c } +// TruncateCollection provides a mock function with given fields: ctx, result +func (_m *IMetaTable) TruncateCollection(ctx context.Context, result message.BroadcastResult[*messagespb.TruncateCollectionMessageHeader, *messagespb.TruncateCollectionMessageBody]) error { + ret := _m.Called(ctx, result) + + if len(ret) == 0 { + panic("no return value specified for TruncateCollection") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastResult[*messagespb.TruncateCollectionMessageHeader, *messagespb.TruncateCollectionMessageBody]) error); ok { + r0 = rf(ctx, result) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// IMetaTable_TruncateCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'TruncateCollection' +type IMetaTable_TruncateCollection_Call struct { + *mock.Call +} + +// TruncateCollection is a helper method to define mock.On call +// - ctx context.Context +// - result message.BroadcastResult[*messagespb.TruncateCollectionMessageHeader,*messagespb.TruncateCollectionMessageBody] +func (_e *IMetaTable_Expecter) TruncateCollection(ctx interface{}, result interface{}) *IMetaTable_TruncateCollection_Call { + return &IMetaTable_TruncateCollection_Call{Call: _e.mock.On("TruncateCollection", ctx, result)} +} + +func (_c *IMetaTable_TruncateCollection_Call) Run(run func(ctx context.Context, result message.BroadcastResult[*messagespb.TruncateCollectionMessageHeader, *messagespb.TruncateCollectionMessageBody])) *IMetaTable_TruncateCollection_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(message.BroadcastResult[*messagespb.TruncateCollectionMessageHeader, *messagespb.TruncateCollectionMessageBody])) + }) + return _c +} + +func (_c *IMetaTable_TruncateCollection_Call) Return(_a0 error) *IMetaTable_TruncateCollection_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *IMetaTable_TruncateCollection_Call) RunAndReturn(run func(context.Context, message.BroadcastResult[*messagespb.TruncateCollectionMessageHeader, *messagespb.TruncateCollectionMessageBody]) error) *IMetaTable_TruncateCollection_Call { + _c.Call.Return(run) + return _c +} + // NewIMetaTable creates a new instance of IMetaTable. 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 NewIMetaTable(t interface { diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 92708a45bf..efefb46a0b 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -21,6 +21,7 @@ import ( "fmt" "math/rand" "os" + "sync" "testing" "time" @@ -77,7 +78,7 @@ func TestMain(m *testing.M) { func initStreamingSystemAndCore(t *testing.T) *Core { kv, _ := kvfactory.GetEtcdAndPath() - path := funcutil.RandomString(10) + path := funcutil.RandomString(10) + "/meta" catalogKV := etcdkv.NewEtcdKV(kv, path) ss, err := rootcoord.NewSuffixSnapshot(catalogKV, rootcoord.SnapshotsSep, path, rootcoord.SnapshotPrefix) @@ -117,6 +118,7 @@ func initStreamingSystemAndCore(t *testing.T) *Core { bapi := mock_broadcaster.NewMockBroadcastAPI(t) bapi.EXPECT().Broadcast(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + msg = msg.WithBroadcastID(1) results := make(map[string]*message.AppendResult) for _, vchannel := range msg.BroadcastHeader().VChannels { results[vchannel] = &message.AppendResult{ @@ -125,6 +127,20 @@ func initStreamingSystemAndCore(t *testing.T) *Core { LastConfirmedMessageID: rmq.NewRmqID(1), } } + wg := sync.WaitGroup{} + for _, mutableMsg := range msg.SplitIntoMutableMessage() { + result := results[mutableMsg.VChannel()] + immutableMsg := mutableMsg.WithTimeTick(result.TimeTick).WithLastConfirmed(result.LastConfirmedMessageID).IntoImmutableMessage(result.MessageID) + wg.Add(1) + go func() { + defer wg.Done() + retry.Do(context.Background(), func() error { + return registry.CallMessageAckOnceCallbacks(context.Background(), immutableMsg) + }, retry.AttemptAlways()) + }() + } + wg.Wait() + retry.Do(context.Background(), func() error { log.Info("broadcast message", log.FieldMessage(msg)) return registry.CallMessageAckCallback(context.Background(), msg, results) diff --git a/internal/streamingcoord/client/broadcast/broadcast_test.go b/internal/streamingcoord/client/broadcast/broadcast_test.go index 47a6aa7fdb..f08e0ead42 100644 --- a/internal/streamingcoord/client/broadcast/broadcast_test.go +++ b/internal/streamingcoord/client/broadcast/broadcast_test.go @@ -24,7 +24,7 @@ func TestBroadcast(t *testing.T) { msg := message.NewDropCollectionMessageBuilderV1(). WithHeader(&message.DropCollectionMessageHeader{}). WithBody(&msgpb.DropCollectionRequest{}). - WithBroadcast([]string{"v1"}, message.NewCollectionNameResourceKey("r1")). + WithBroadcast([]string{"v1"}). MustBuildBroadcast() _, err := bs.Broadcast(context.Background(), msg) assert.NoError(t, err) diff --git a/internal/streamingcoord/server/broadcaster/broadcast_task.go b/internal/streamingcoord/server/broadcaster/broadcast_task.go index 20cfbf5207..5048dbbb96 100644 --- a/internal/streamingcoord/server/broadcaster/broadcast_task.go +++ b/internal/streamingcoord/server/broadcaster/broadcast_task.go @@ -8,6 +8,7 @@ import ( "go.uber.org/zap" "google.golang.org/protobuf/proto" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb" @@ -245,11 +246,11 @@ func (b *broadcastTask) getImmutableMessageFromVChannel(vchannel string, result // Ack acknowledges the message at the specified vchannel. // return true if all the vchannels are acked at first time, false if not. -func (b *broadcastTask) Ack(ctx context.Context, msgs ...message.ImmutableMessage) (err error) { +func (b *broadcastTask) Ack(ctx context.Context, msgs message.ImmutableMessage) (err error) { b.mu.Lock() defer b.mu.Unlock() - return b.ack(ctx, msgs...) + return b.ack(ctx, msgs) } // ack acknowledges the message at the specified vchannel. @@ -258,10 +259,14 @@ func (b *broadcastTask) ack(ctx context.Context, msgs ...message.ImmutableMessag if !b.dirty { return nil } + // because the incoming ack operation is always with one vchannel at a time or with all the vchannels at once, + // so we don't need to filter the vchannel that has been acked. + if err := registry.CallMessageAckOnceCallbacks(ctx, msgs...); err != nil { + return err + } if err := b.saveTaskIfDirty(ctx, b.Logger()); err != nil { return err } - allDone := isAllDone(b.task) if (isControlChannelAcked || allDone) && !b.joinAckCallbackScheduled { // after 2.6.5, the control channel is always broadcasted, it's used to determine the order of the ack callback operations. @@ -375,6 +380,13 @@ func (b *broadcastTask) FastAck(ctx context.Context, broadcastResult map[string] defer b.mu.Unlock() b.ObserveBroadcastDone() + + if b.Header().AckSyncUp { + // Because the ack sync up is enabled, the ack operation want to be synced up at comsuming side of streaming node, + // so we can not make a fast ack operation here to speed up the ack operation. + return nil + } + // because we need to wait for the streamingnode to ack the message, // however, if the message is already write into wal, the message is determined, // so we can make a fast ack operation here to speed up the ack operation. diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_test.go b/internal/streamingcoord/server/broadcaster/broadcaster_test.go index f6d8e12558..b1e7515d29 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_test.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_test.go @@ -220,16 +220,16 @@ func createNewBroadcastMsg(vchannels []string, rks ...message.ResourceKey) messa msg, err := message.NewDropCollectionMessageBuilderV1(). WithHeader(&messagespb.DropCollectionMessageHeader{}). WithBody(&msgpb.DropCollectionRequest{}). - WithBroadcast(vchannels, rks...). + WithBroadcast(vchannels). BuildBroadcast() if err != nil { panic(err) } - return msg + return msg.OverwriteBroadcastHeader(0, rks...) } func createNewBroadcastTask(broadcastID uint64, vchannels []string, rks ...message.ResourceKey) *streamingpb.BroadcastTask { - msg := createNewBroadcastMsg(vchannels, rks...).WithBroadcastID(broadcastID) + msg := createNewBroadcastMsg(vchannels).OverwriteBroadcastHeader(broadcastID, rks...) pb := msg.IntoMessageProto() return &streamingpb.BroadcastTask{ Message: &messagespb.Message{ diff --git a/internal/streamingcoord/server/broadcaster/registry/ack_message_callback.go b/internal/streamingcoord/server/broadcaster/registry/ack_message_callback.go index a6ffe834c5..6b6894e8ab 100644 --- a/internal/streamingcoord/server/broadcaster/registry/ack_message_callback.go +++ b/internal/streamingcoord/server/broadcaster/registry/ack_message_callback.go @@ -12,6 +12,7 @@ import ( ) // MessageAckCallback is the callback function for the message type. +// It will be called when all the message are acked. type ( MessageAckCallback[H proto.Message, B proto.Message] = func(ctx context.Context, result message.BroadcastResult[H, B]) error messageInnerAckCallback = func(ctx context.Context, msg message.BroadcastMutableMessage, result map[string]*message.AppendResult) error diff --git a/internal/streamingcoord/server/broadcaster/registry/ack_once_message_callback.go b/internal/streamingcoord/server/broadcaster/registry/ack_once_message_callback.go new file mode 100644 index 0000000000..60c599cd7b --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/registry/ack_once_message_callback.go @@ -0,0 +1,83 @@ +package registry + +import ( + "context" + "fmt" + "sync" + + "github.com/cockroachdb/errors" + "google.golang.org/protobuf/proto" + + "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" + "github.com/milvus-io/milvus/pkg/v2/util/syncutil" +) + +// MessageAckOnceCallback is the callback function for the message type. +// It will be called when the message is acked by streamingnode at one channel at a time. +type ( + MessageAckOnceCallback[H proto.Message, B proto.Message] = func(ctx context.Context, result message.AckResult[H, B]) error + messageInnerAckOnceCallback = func(ctx context.Context, msg message.ImmutableMessage) error +) + +// messageAckOnceCallbacks is the map of message type to the ack once callback function. +var messageAckOnceCallbacks map[message.MessageTypeWithVersion]*syncutil.Future[messageInnerAckOnceCallback] + +// registerMessageAckOnceCallback registers the ack once callback function for the message type. +func registerMessageAckOnceCallback[H proto.Message, B proto.Message](callback MessageAckOnceCallback[H, B]) { + typ := message.MustGetMessageTypeWithVersion[H, B]() + future, ok := messageAckOnceCallbacks[typ] + if !ok { + panic(fmt.Sprintf("the future of ack once callback for type %s is not registered", typ)) + } + if future.Ready() { + // only for test, the register callback should be called once and only once + return + } + future.Set(func(ctx context.Context, msg message.ImmutableMessage) error { + return callback(ctx, message.AckResult[H, B]{ + Message: message.MustAsSpecializedImmutableMessage[H, B](msg), + }) + }) +} + +// CallMessageAckOnceCallbacks calls the ack callback function for the message type in batch. +func CallMessageAckOnceCallbacks(ctx context.Context, msgs ...message.ImmutableMessage) error { + if len(msgs) == 0 { + return nil + } + if len(msgs) == 1 { + return callMessageAckOnceCallback(ctx, msgs[0]) + } + wg := &sync.WaitGroup{} + wg.Add(len(msgs)) + errc := make(chan error, len(msgs)) + for _, msg := range msgs { + msg := msg + go func() { + defer wg.Done() + errc <- callMessageAckOnceCallback(ctx, msg) + }() + } + wg.Wait() + close(errc) + for err := range errc { + if err != nil { + return err + } + } + return nil +} + +// callMessageAckOnceCallback calls the ack once callback function for the message type. +func callMessageAckOnceCallback(ctx context.Context, msg message.ImmutableMessage) error { + callbackFuture, ok := messageAckOnceCallbacks[msg.MessageTypeWithVersion()] + if !ok { + // No callback need tobe called, return nil + return nil + } + callback, err := callbackFuture.GetWithContext(ctx) + if err != nil { + return errors.Wrap(err, "when waiting callback registered") + } + return callback(ctx, msg) +} diff --git a/internal/streamingcoord/server/broadcaster/registry/specialized_ack_once_callback.go b/internal/streamingcoord/server/broadcaster/registry/specialized_ack_once_callback.go new file mode 100644 index 0000000000..147bdd9865 --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/registry/specialized_ack_once_callback.go @@ -0,0 +1,31 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package registry + +import ( + "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" + "github.com/milvus-io/milvus/pkg/v2/util/syncutil" +) + +// Collection +var RegisterTruncateCollectionV2AckOnceCallback = registerMessageAckOnceCallback[*message.TruncateCollectionMessageHeader, *message.TruncateCollectionMessageBody] + +func resetMessageAckOnceCallbacks() { + messageAckOnceCallbacks = map[message.MessageTypeWithVersion]*syncutil.Future[messageInnerAckOnceCallback]{ + message.MessageTypeTruncateCollectionV2: syncutil.NewFuture[messageInnerAckOnceCallback](), + } +} diff --git a/internal/streamingcoord/server/broadcaster/registry/specialized_callback.go b/internal/streamingcoord/server/broadcaster/registry/specialized_callback.go index 84986b1c01..f1657f4c88 100644 --- a/internal/streamingcoord/server/broadcaster/registry/specialized_callback.go +++ b/internal/streamingcoord/server/broadcaster/registry/specialized_callback.go @@ -8,6 +8,7 @@ import ( // init the message ack callbacks func init() { + resetMessageAckOnceCallbacks() resetMessageAckCallbacks() resetMessageCheckCallbacks() } diff --git a/internal/streamingcoord/server/broadcaster/registry/test_utility.go b/internal/streamingcoord/server/broadcaster/registry/test_utility.go index 00602c1b6a..b7f332c867 100644 --- a/internal/streamingcoord/server/broadcaster/registry/test_utility.go +++ b/internal/streamingcoord/server/broadcaster/registry/test_utility.go @@ -5,5 +5,6 @@ package registry func ResetRegistration() { resetMessageAckCallbacks() + resetMessageAckOnceCallbacks() resetMessageCheckCallbacks() } diff --git a/internal/streamingcoord/server/service/broadcast_test.go b/internal/streamingcoord/server/service/broadcast_test.go index 0cd5dfe20d..0fdac6d140 100644 --- a/internal/streamingcoord/server/service/broadcast_test.go +++ b/internal/streamingcoord/server/service/broadcast_test.go @@ -52,7 +52,7 @@ func TestBroadcastService(t *testing.T) { msg := message.NewCreateCollectionMessageBuilderV1(). WithHeader(&message.CreateCollectionMessageHeader{}). WithBody(&msgpb.CreateCollectionRequest{}). - WithBroadcast([]string{"v1"}, message.NewCollectionNameResourceKey("r1")). + WithBroadcast([]string{"v1"}). MustBuildBroadcast() service := NewBroadcastService() diff --git a/internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go b/internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go index 9c267d5517..03908eed4d 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go +++ b/internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go @@ -206,13 +206,22 @@ func (impl *WALFlusherImpl) generateScanner(ctx context.Context, l wal.WAL, chec // dispatch dispatches the message to the related handler for flusher components. func (impl *WALFlusherImpl) dispatch(msg message.ImmutableMessage) (err error) { - // TODO: We will merge the flusher into recovery storage in future. - // Currently, flusher works as a separate component. - defer func() { - if err = impl.RecoveryStorage.ObserveMessage(impl.notifier.Context(), msg); err != nil { + // TODO: should be removed at 3.0, after merge the flusher logic into recovery storage. + // only for truncate api now. + if bh := msg.BroadcastHeader(); bh != nil && bh.AckSyncUp { + if err := impl.RecoveryStorage.ObserveMessage(impl.notifier.Context(), msg); err != nil { impl.logger.Warn("failed to observe message", zap.Error(err)) + return err } - }() + } else { + // TODO: We will merge the flusher into recovery storage in future. + // Currently, flusher works as a separate component. + defer func() { + if err = impl.RecoveryStorage.ObserveMessage(impl.notifier.Context(), msg); err != nil { + impl.logger.Warn("failed to observe message", zap.Error(err)) + } + }() + } // wal flusher will not handle the control channel message. if funcutil.IsControlChannel(msg.VChannel()) && !isBroadcastToAllMessage(msg.MessageType()) { diff --git a/pkg/common/common.go b/pkg/common/common.go index ca1d937809..10c6585cfb 100644 --- a/pkg/common/common.go +++ b/pkg/common/common.go @@ -188,6 +188,9 @@ const ( CollectionAutoCompactionKey = "collection.autocompaction.enabled" CollectionDescription = "collection.description" + // Deprecated: will be removed in the 3.0 after implementing ack sync up semantic. + CollectionOnTruncatingKey = "collection.on.truncating" // when collection is on truncating, forbid the compaction of current collection. + // Note: // Function output fields cannot be included in inserted data. // In particular, the `bm25` function output field is always disallowed diff --git a/pkg/proto/etcd_meta.proto b/pkg/proto/etcd_meta.proto index ddb03a2007..35d9aeefeb 100644 --- a/pkg/proto/etcd_meta.proto +++ b/pkg/proto/etcd_meta.proto @@ -70,7 +70,11 @@ message CollectionInfo { repeated common.KeyValuePair properties = 14; int64 db_id = 15; uint64 UpdateTimestamp = 16; - map LastTruncateTimestamps = 17; + repeated CollectionShardInfo shard_infos = 17; // same length with virtual_channel_names, order one by one. +} + +message CollectionShardInfo { + uint64 last_truncate_time_tick = 1; // the last truncate time tick of the shard, if the shard is not truncated, the value is 0. } message PartitionInfo { diff --git a/pkg/proto/etcdpb/etcd_meta.pb.go b/pkg/proto/etcdpb/etcd_meta.pb.go index 5564d5b4d3..ba8a65f6ce 100644 --- a/pkg/proto/etcdpb/etcd_meta.pb.go +++ b/pkg/proto/etcdpb/etcd_meta.pb.go @@ -396,7 +396,7 @@ type CollectionInfo struct { Properties []*commonpb.KeyValuePair `protobuf:"bytes,14,rep,name=properties,proto3" json:"properties,omitempty"` DbId int64 `protobuf:"varint,15,opt,name=db_id,json=dbId,proto3" json:"db_id,omitempty"` UpdateTimestamp uint64 `protobuf:"varint,16,opt,name=UpdateTimestamp,proto3" json:"UpdateTimestamp,omitempty"` - LastTruncateTimestamps map[string]uint64 `protobuf:"bytes,17,rep,name=LastTruncateTimestamps,proto3" json:"LastTruncateTimestamps,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + ShardInfos []*CollectionShardInfo `protobuf:"bytes,17,rep,name=shard_infos,json=shardInfos,proto3" json:"shard_infos,omitempty"` // same length with virtual_channel_names, order one by one. } func (x *CollectionInfo) Reset() { @@ -543,13 +543,60 @@ func (x *CollectionInfo) GetUpdateTimestamp() uint64 { return 0 } -func (x *CollectionInfo) GetLastTruncateTimestamps() map[string]uint64 { +func (x *CollectionInfo) GetShardInfos() []*CollectionShardInfo { if x != nil { - return x.LastTruncateTimestamps + return x.ShardInfos } return nil } +type CollectionShardInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + LastTruncateTimeTick uint64 `protobuf:"varint,1,opt,name=last_truncate_time_tick,json=lastTruncateTimeTick,proto3" json:"last_truncate_time_tick,omitempty"` // the last truncate time tick of the shard, if the shard is not truncated, the value is 0. +} + +func (x *CollectionShardInfo) Reset() { + *x = CollectionShardInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_etcd_meta_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CollectionShardInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CollectionShardInfo) ProtoMessage() {} + +func (x *CollectionShardInfo) ProtoReflect() protoreflect.Message { + mi := &file_etcd_meta_proto_msgTypes[3] + 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 CollectionShardInfo.ProtoReflect.Descriptor instead. +func (*CollectionShardInfo) Descriptor() ([]byte, []int) { + return file_etcd_meta_proto_rawDescGZIP(), []int{3} +} + +func (x *CollectionShardInfo) GetLastTruncateTimeTick() uint64 { + if x != nil { + return x.LastTruncateTimeTick + } + return 0 +} + type PartitionInfo struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -565,7 +612,7 @@ type PartitionInfo struct { func (x *PartitionInfo) Reset() { *x = PartitionInfo{} if protoimpl.UnsafeEnabled { - mi := &file_etcd_meta_proto_msgTypes[3] + mi := &file_etcd_meta_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -578,7 +625,7 @@ func (x *PartitionInfo) String() string { func (*PartitionInfo) ProtoMessage() {} func (x *PartitionInfo) ProtoReflect() protoreflect.Message { - mi := &file_etcd_meta_proto_msgTypes[3] + mi := &file_etcd_meta_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -591,7 +638,7 @@ func (x *PartitionInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use PartitionInfo.ProtoReflect.Descriptor instead. func (*PartitionInfo) Descriptor() ([]byte, []int) { - return file_etcd_meta_proto_rawDescGZIP(), []int{3} + return file_etcd_meta_proto_rawDescGZIP(), []int{4} } func (x *PartitionInfo) GetPartitionID() int64 { @@ -644,7 +691,7 @@ type AliasInfo struct { func (x *AliasInfo) Reset() { *x = AliasInfo{} if protoimpl.UnsafeEnabled { - mi := &file_etcd_meta_proto_msgTypes[4] + mi := &file_etcd_meta_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -657,7 +704,7 @@ func (x *AliasInfo) String() string { func (*AliasInfo) ProtoMessage() {} func (x *AliasInfo) ProtoReflect() protoreflect.Message { - mi := &file_etcd_meta_proto_msgTypes[4] + mi := &file_etcd_meta_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -670,7 +717,7 @@ func (x *AliasInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use AliasInfo.ProtoReflect.Descriptor instead. func (*AliasInfo) Descriptor() ([]byte, []int) { - return file_etcd_meta_proto_rawDescGZIP(), []int{4} + return file_etcd_meta_proto_rawDescGZIP(), []int{5} } func (x *AliasInfo) GetAliasName() string { @@ -724,7 +771,7 @@ type DatabaseInfo struct { func (x *DatabaseInfo) Reset() { *x = DatabaseInfo{} if protoimpl.UnsafeEnabled { - mi := &file_etcd_meta_proto_msgTypes[5] + mi := &file_etcd_meta_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -737,7 +784,7 @@ func (x *DatabaseInfo) String() string { func (*DatabaseInfo) ProtoMessage() {} func (x *DatabaseInfo) ProtoReflect() protoreflect.Message { - mi := &file_etcd_meta_proto_msgTypes[5] + mi := &file_etcd_meta_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -750,7 +797,7 @@ func (x *DatabaseInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use DatabaseInfo.ProtoReflect.Descriptor instead. func (*DatabaseInfo) Descriptor() ([]byte, []int) { - return file_etcd_meta_proto_rawDescGZIP(), []int{5} + return file_etcd_meta_proto_rawDescGZIP(), []int{6} } func (x *DatabaseInfo) GetTenantId() string { @@ -813,7 +860,7 @@ type SegmentIndexInfo struct { func (x *SegmentIndexInfo) Reset() { *x = SegmentIndexInfo{} if protoimpl.UnsafeEnabled { - mi := &file_etcd_meta_proto_msgTypes[6] + mi := &file_etcd_meta_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -826,7 +873,7 @@ func (x *SegmentIndexInfo) String() string { func (*SegmentIndexInfo) ProtoMessage() {} func (x *SegmentIndexInfo) ProtoReflect() protoreflect.Message { - mi := &file_etcd_meta_proto_msgTypes[6] + mi := &file_etcd_meta_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -839,7 +886,7 @@ func (x *SegmentIndexInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use SegmentIndexInfo.ProtoReflect.Descriptor instead. func (*SegmentIndexInfo) Descriptor() ([]byte, []int) { - return file_etcd_meta_proto_rawDescGZIP(), []int{6} + return file_etcd_meta_proto_rawDescGZIP(), []int{7} } func (x *SegmentIndexInfo) GetCollectionID() int64 { @@ -915,7 +962,7 @@ type CollectionMeta struct { func (x *CollectionMeta) Reset() { *x = CollectionMeta{} if protoimpl.UnsafeEnabled { - mi := &file_etcd_meta_proto_msgTypes[7] + mi := &file_etcd_meta_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -928,7 +975,7 @@ func (x *CollectionMeta) String() string { func (*CollectionMeta) ProtoMessage() {} func (x *CollectionMeta) ProtoReflect() protoreflect.Message { - mi := &file_etcd_meta_proto_msgTypes[7] + mi := &file_etcd_meta_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -941,7 +988,7 @@ func (x *CollectionMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionMeta.ProtoReflect.Descriptor instead. func (*CollectionMeta) Descriptor() ([]byte, []int) { - return file_etcd_meta_proto_rawDescGZIP(), []int{7} + return file_etcd_meta_proto_rawDescGZIP(), []int{8} } func (x *CollectionMeta) GetID() int64 { @@ -1003,7 +1050,7 @@ type CredentialInfo struct { func (x *CredentialInfo) Reset() { *x = CredentialInfo{} if protoimpl.UnsafeEnabled { - mi := &file_etcd_meta_proto_msgTypes[8] + mi := &file_etcd_meta_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1016,7 +1063,7 @@ func (x *CredentialInfo) String() string { func (*CredentialInfo) ProtoMessage() {} func (x *CredentialInfo) ProtoReflect() protoreflect.Message { - mi := &file_etcd_meta_proto_msgTypes[8] + mi := &file_etcd_meta_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1029,7 +1076,7 @@ func (x *CredentialInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use CredentialInfo.ProtoReflect.Descriptor instead. func (*CredentialInfo) Descriptor() ([]byte, []int) { - return file_etcd_meta_proto_rawDescGZIP(), []int{8} + return file_etcd_meta_proto_rawDescGZIP(), []int{9} } func (x *CredentialInfo) GetUsername() string { @@ -1090,7 +1137,7 @@ var file_etcd_meta_proto_rawDesc = []byte{ 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x65, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x22, 0xfc, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x22, 0x83, 0x07, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, @@ -1142,137 +1189,134 @@ var file_etcd_meta_proto_rawDesc = []byte{ 0x64, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x64, 0x12, 0x28, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x10, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x75, 0x0a, 0x16, 0x4c, - 0x61, 0x73, 0x74, 0x54, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x73, 0x18, 0x11, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, 0x63, 0x64, 0x2e, - 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4c, - 0x61, 0x73, 0x74, 0x54, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x4c, 0x61, 0x73, 0x74, - 0x54, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x73, 0x1a, 0x49, 0x0a, 0x1b, 0x4c, 0x61, 0x73, 0x74, 0x54, 0x72, 0x75, 0x6e, 0x63, 0x61, - 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf5, 0x01, - 0x0a, 0x0d, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, - 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, - 0x44, 0x12, 0x24, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3e, 0x0a, 0x1b, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, - 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x19, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, - 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x37, 0x0a, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, 0x63, 0x64, 0x2e, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xbc, 0x01, 0x0a, 0x09, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x49, - 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x05, 0x73, 0x74, - 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, 0x63, 0x64, 0x2e, 0x41, 0x6c, - 0x69, 0x61, 0x73, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, - 0x13, 0x0a, 0x05, 0x64, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, - 0x64, 0x62, 0x49, 0x64, 0x22, 0xed, 0x01, 0x0a, 0x0c, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, - 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, - 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x36, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, 0x63, 0x64, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, - 0x73, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x21, - 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, - 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, - 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, - 0x74, 0x69, 0x65, 0x73, 0x22, 0x88, 0x02, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, - 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, - 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x18, 0x0a, - 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, - 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, - 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x65, - 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x1f, - 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x22, - 0xeb, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, - 0x74, 0x61, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, - 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, - 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, - 0x44, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x74, 0x61, 0x67, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x67, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x03, 0x52, - 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x22, 0xb7, 0x01, - 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x12, - 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, - 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, - 0x74, 0x65, 0x64, 0x50, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, - 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x65, 0x6e, - 0x61, 0x6e, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x73, 0x75, 0x70, 0x65, 0x72, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x53, 0x75, 0x70, 0x65, 0x72, 0x12, 0x27, - 0x0a, 0x0f, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x5f, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, - 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x50, - 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x2a, 0x7e, 0x0a, 0x0d, 0x44, 0x61, 0x74, 0x61, 0x62, - 0x61, 0x73, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x13, 0x0a, 0x0f, 0x44, 0x61, 0x74, 0x61, - 0x62, 0x61, 0x73, 0x65, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x13, 0x0a, - 0x0f, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, - 0x10, 0x01, 0x12, 0x18, 0x0a, 0x10, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x1a, 0x02, 0x08, 0x01, 0x12, 0x14, 0x0a, 0x10, - 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x10, 0x03, 0x12, 0x13, 0x0a, 0x0f, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x44, 0x72, - 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x04, 0x2a, 0x73, 0x0a, 0x0f, 0x43, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x10, - 0x00, 0x12, 0x1a, 0x0a, 0x12, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x1a, 0x02, 0x08, 0x01, 0x12, 0x16, 0x0a, - 0x12, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x03, 0x2a, 0x6e, 0x0a, 0x0e, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x14, - 0x0a, 0x10, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x64, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x11, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x1a, 0x02, 0x08, 0x01, 0x12, - 0x15, 0x0a, 0x11, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, - 0x70, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x03, 0x2a, 0x5a, 0x0a, 0x0a, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x6c, - 0x69, 0x61, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x10, 0x00, 0x12, 0x15, 0x0a, 0x0d, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x1a, - 0x02, 0x08, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x44, 0x72, 0x6f, 0x70, - 0x70, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x44, - 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x03, 0x42, 0x31, 0x5a, 0x2f, 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, 0x76, 0x32, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, 0x63, 0x64, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x33, + 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x47, 0x0a, 0x0b, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x11, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x65, 0x74, 0x63, 0x64, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x49, + 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x4c, 0x0a, 0x13, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x35, 0x0a, 0x17, 0x6c, + 0x61, 0x73, 0x74, 0x5f, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x14, 0x6c, 0x61, + 0x73, 0x74, 0x54, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, + 0x63, 0x6b, 0x22, 0xf5, 0x01, 0x0a, 0x0d, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3e, 0x0a, 0x1b, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x23, 0x0a, 0x0d, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x64, 0x12, 0x37, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x65, 0x74, 0x63, 0x64, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xbc, 0x01, 0x0a, 0x09, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x6c, 0x69, 0x61, + 0x73, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, + 0x33, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, + 0x63, 0x64, 0x2e, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x64, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x64, 0x22, 0xed, 0x01, 0x0a, 0x0c, 0x44, 0x61, + 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, + 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x36, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x65, 0x74, 0x63, 0x64, 0x2e, 0x44, + 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, + 0x74, 0x69, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x0a, 0x70, + 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x22, 0x88, 0x02, 0x0a, 0x10, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, + 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, + 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44, + 0x12, 0x21, 0x0a, 0x0c, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x54, 0x69, 0x6d, 0x65, 0x22, 0xeb, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x67, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x67, 0x73, 0x12, 0x22, + 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x06, + 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x73, 0x22, 0xb7, 0x01, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, + 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x2d, 0x0a, 0x12, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x70, + 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x65, + 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x65, 0x64, 0x50, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, + 0x12, 0x16, 0x0a, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x73, + 0x75, 0x70, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x53, 0x75, + 0x70, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x5f, 0x70, 0x61, + 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x68, + 0x61, 0x32, 0x35, 0x36, 0x50, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x2a, 0x7e, 0x0a, 0x0d, + 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x13, 0x0a, + 0x0f, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, + 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x64, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x10, 0x44, 0x61, 0x74, 0x61, 0x62, + 0x61, 0x73, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x1a, 0x02, 0x08, + 0x01, 0x12, 0x14, 0x0a, 0x10, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x44, 0x72, 0x6f, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x13, 0x0a, 0x0f, 0x44, 0x61, 0x74, 0x61, 0x62, + 0x61, 0x73, 0x65, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x04, 0x2a, 0x73, 0x0a, 0x0f, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x15, 0x0a, 0x11, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x64, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x12, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x1a, 0x02, + 0x08, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x44, 0x72, 0x6f, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, + 0x03, 0x2a, 0x6e, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x11, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, + 0x1a, 0x02, 0x08, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, + 0x03, 0x2a, 0x5a, 0x0a, 0x0a, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x10, 0x0a, 0x0c, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x10, + 0x00, 0x12, 0x15, 0x0a, 0x0d, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, + 0x6e, 0x67, 0x10, 0x01, 0x1a, 0x02, 0x08, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x6c, 0x69, 0x61, + 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x03, 0x42, 0x31, 0x5a, + 0x2f, 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, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, 0x63, 0x64, 0x70, 0x62, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1297,13 +1341,13 @@ var file_etcd_meta_proto_goTypes = []interface{}{ (*IndexInfo)(nil), // 4: milvus.proto.etcd.IndexInfo (*FieldIndexInfo)(nil), // 5: milvus.proto.etcd.FieldIndexInfo (*CollectionInfo)(nil), // 6: milvus.proto.etcd.CollectionInfo - (*PartitionInfo)(nil), // 7: milvus.proto.etcd.PartitionInfo - (*AliasInfo)(nil), // 8: milvus.proto.etcd.AliasInfo - (*DatabaseInfo)(nil), // 9: milvus.proto.etcd.DatabaseInfo - (*SegmentIndexInfo)(nil), // 10: milvus.proto.etcd.SegmentIndexInfo - (*CollectionMeta)(nil), // 11: milvus.proto.etcd.CollectionMeta - (*CredentialInfo)(nil), // 12: milvus.proto.etcd.CredentialInfo - nil, // 13: milvus.proto.etcd.CollectionInfo.LastTruncateTimestampsEntry + (*CollectionShardInfo)(nil), // 7: milvus.proto.etcd.CollectionShardInfo + (*PartitionInfo)(nil), // 8: milvus.proto.etcd.PartitionInfo + (*AliasInfo)(nil), // 9: milvus.proto.etcd.AliasInfo + (*DatabaseInfo)(nil), // 10: milvus.proto.etcd.DatabaseInfo + (*SegmentIndexInfo)(nil), // 11: milvus.proto.etcd.SegmentIndexInfo + (*CollectionMeta)(nil), // 12: milvus.proto.etcd.CollectionMeta + (*CredentialInfo)(nil), // 13: milvus.proto.etcd.CredentialInfo (*commonpb.KeyValuePair)(nil), // 14: milvus.proto.common.KeyValuePair (*schemapb.CollectionSchema)(nil), // 15: milvus.proto.schema.CollectionSchema (*commonpb.KeyDataPair)(nil), // 16: milvus.proto.common.KeyDataPair @@ -1317,7 +1361,7 @@ var file_etcd_meta_proto_depIdxs = []int32{ 17, // 4: milvus.proto.etcd.CollectionInfo.consistency_level:type_name -> milvus.proto.common.ConsistencyLevel 1, // 5: milvus.proto.etcd.CollectionInfo.state:type_name -> milvus.proto.etcd.CollectionState 14, // 6: milvus.proto.etcd.CollectionInfo.properties:type_name -> milvus.proto.common.KeyValuePair - 13, // 7: milvus.proto.etcd.CollectionInfo.LastTruncateTimestamps:type_name -> milvus.proto.etcd.CollectionInfo.LastTruncateTimestampsEntry + 7, // 7: milvus.proto.etcd.CollectionInfo.shard_infos:type_name -> milvus.proto.etcd.CollectionShardInfo 2, // 8: milvus.proto.etcd.PartitionInfo.state:type_name -> milvus.proto.etcd.PartitionState 3, // 9: milvus.proto.etcd.AliasInfo.state:type_name -> milvus.proto.etcd.AliasState 0, // 10: milvus.proto.etcd.DatabaseInfo.state:type_name -> milvus.proto.etcd.DatabaseState @@ -1373,7 +1417,7 @@ func file_etcd_meta_proto_init() { } } file_etcd_meta_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PartitionInfo); i { + switch v := v.(*CollectionShardInfo); i { case 0: return &v.state case 1: @@ -1385,7 +1429,7 @@ func file_etcd_meta_proto_init() { } } file_etcd_meta_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AliasInfo); i { + switch v := v.(*PartitionInfo); i { case 0: return &v.state case 1: @@ -1397,7 +1441,7 @@ func file_etcd_meta_proto_init() { } } file_etcd_meta_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DatabaseInfo); i { + switch v := v.(*AliasInfo); i { case 0: return &v.state case 1: @@ -1409,7 +1453,7 @@ func file_etcd_meta_proto_init() { } } file_etcd_meta_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SegmentIndexInfo); i { + switch v := v.(*DatabaseInfo); i { case 0: return &v.state case 1: @@ -1421,7 +1465,7 @@ func file_etcd_meta_proto_init() { } } file_etcd_meta_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CollectionMeta); i { + switch v := v.(*SegmentIndexInfo); i { case 0: return &v.state case 1: @@ -1433,6 +1477,18 @@ func file_etcd_meta_proto_init() { } } file_etcd_meta_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CollectionMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_etcd_meta_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CredentialInfo); i { case 0: return &v.state diff --git a/pkg/proto/messages.proto b/pkg/proto/messages.proto index 4e9d221696..bacf90210c 100644 --- a/pkg/proto/messages.proto +++ b/pkg/proto/messages.proto @@ -630,6 +630,13 @@ message BroadcastHeader { // 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. + bool ack_sync_up = 4; // whether the broadcast operation is need to be synced up between the streaming node and the coordinator. + // If the ack_sync_up is false, the broadcast operation will be acked once the recovery storage see the message at current vchannel, + // the fast ack operation can be applied to speed up the broadcast operation. + // If the ack_sync_up is true, the broadcast operation will be acked after the checkpoint of current vchannel reach current message. + // the fast ack operation can not be applied to speed up the broadcast operation, because the ack operation need to be synced up with streaming node. + // e.g. if truncate collection operation want to call ack once callback after the all segment are flushed at current vchannel, + // it should set the ack_sync_up to be true. } // ReplicateHeader is the header of replicate message. diff --git a/pkg/proto/messagespb/messages.pb.go b/pkg/proto/messagespb/messages.pb.go index 4efe6d2da9..ae8bd214d4 100644 --- a/pkg/proto/messagespb/messages.pb.go +++ b/pkg/proto/messagespb/messages.pb.go @@ -5120,6 +5120,10 @@ type BroadcastHeader struct { 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. + // 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. + AckSyncUp bool `protobuf:"varint,4,opt,name=ack_sync_up,json=ackSyncUp,proto3" json:"ack_sync_up,omitempty"` // whether the broadcast operation is need to be synced up between the streaming node and the coordinator. } func (x *BroadcastHeader) Reset() { @@ -5175,6 +5179,13 @@ func (x *BroadcastHeader) GetResourceKeys() []*ResourceKey { return nil } +func (x *BroadcastHeader) GetAckSyncUp() bool { + if x != nil { + return x.AckSyncUp + } + return false +} + // ReplicateHeader is the header of replicate message. type ReplicateHeader struct { state protoimpl.MessageState @@ -6061,7 +6072,7 @@ var file_messages_proto_rawDesc = []byte{ 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xbb, 0x01, 0x0a, 0x0f, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, @@ -6071,7 +6082,9 @@ var file_messages_proto_rawDesc = []byte{ 0x65, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 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, 0x0c, 0x52, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x83, 0x02, 0x0a, 0x0f, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x61, + 0x63, 0x6b, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x75, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x09, 0x61, 0x63, 0x6b, 0x53, 0x79, 0x6e, 0x63, 0x55, 0x70, 0x22, 0x83, 0x02, 0x0a, 0x0f, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x64, 0x12, 0x3d, diff --git a/pkg/streaming/util/message/broadcast.go b/pkg/streaming/util/message/broadcast.go index 6dd74d1313..9bf2102da7 100644 --- a/pkg/streaming/util/message/broadcast.go +++ b/pkg/streaming/util/message/broadcast.go @@ -18,6 +18,7 @@ func newBroadcastHeaderFromProto(proto *messagespb.BroadcastHeader) *BroadcastHe BroadcastID: proto.GetBroadcastId(), VChannels: proto.GetVchannels(), ResourceKeys: rks, + AckSyncUp: proto.GetAckSyncUp(), } } @@ -25,6 +26,7 @@ type BroadcastHeader struct { BroadcastID uint64 VChannels []string ResourceKeys typeutil.Set[ResourceKey] + AckSyncUp bool } // BroadcastResult is the result of broadcast operation. @@ -33,6 +35,11 @@ type BroadcastResult[H proto.Message, B proto.Message] struct { Results map[string]*AppendResult } +// AckResult is the result of ack operation. +type AckResult[H proto.Message, B proto.Message] struct { + Message SpecializedImmutableMessage[H, B] +} + // GetMaxTimeTick returns the max time tick of the broadcast result. func (br *BroadcastResult[H, B]) GetMaxTimeTick() uint64 { maxTimeTick := uint64(0) diff --git a/pkg/streaming/util/message/builder.go b/pkg/streaming/util/message/builder.go index ff70d8f681..c116183221 100644 --- a/pkg/streaming/util/message/builder.go +++ b/pkg/streaming/util/message/builder.go @@ -155,8 +155,24 @@ func (b *mutableMesasgeBuilder[H, B]) WithVChannel(vchannel string) *mutableMesa return b } +// OptBuildBroadcast is the option for building broadcast message. +type OptBuildBroadcast func(*messagespb.BroadcastHeader) + +// OptBuildBroadcastAckSyncUp sets the ack sync up of the broadcast message. +// Whether the broadcast operation is need to be synced up between the streaming node and the coordinator. +// If set, the broadcast operation will be acked after the checkpoint of current vchannel reach current message. +// the fast ack operation can not be applied to speed up the broadcast operation, because the ack operation need to be synced up with streaming node. +// TODO: current implementation doesn't promise the ack sync up semantic, +// it only promise FastAck operation will not be applied, wait for 3.0 to implement the ack sync up semantic. +// only for truncate api now. +func OptBuildBroadcastAckSyncUp() OptBuildBroadcast { + return func(bh *messagespb.BroadcastHeader) { + bh.AckSyncUp = true + } +} + // WithBroadcast creates a new builder with broadcast property. -func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string, resourceKeys ...ResourceKey) *mutableMesasgeBuilder[H, B] { +func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string, opts ...OptBuildBroadcast) *mutableMesasgeBuilder[H, B] { if len(vchannels) < 1 { panic("broadcast message must have at least one vchannel") } @@ -167,11 +183,14 @@ func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string, resource panic("a broadcast message cannot set up vchannel property") } deduplicated := typeutil.NewSet(vchannels...) + bhpb := &messagespb.BroadcastHeader{ + Vchannels: deduplicated.Collect(), + } + for _, opt := range opts { + opt(bhpb) + } - bh, err := EncodeProto(&messagespb.BroadcastHeader{ - Vchannels: deduplicated.Collect(), - ResourceKeys: newProtoFromResourceKey(resourceKeys...), - }) + bh, err := EncodeProto(bhpb) if err != nil { panic("failed to encode vchannels") } diff --git a/pkg/streaming/util/message/builder_test.go b/pkg/streaming/util/message/builder_test.go index 52e8633b41..8088c59294 100644 --- a/pkg/streaming/util/message/builder_test.go +++ b/pkg/streaming/util/message/builder_test.go @@ -5,6 +5,7 @@ import ( "time" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" @@ -106,7 +107,14 @@ func TestReplicateBuilder(t *testing.T) { msgs := msg.WithBroadcastID(1).SplitIntoMutableMessage() msgID := walimplstest.NewTestMessageID(1) - immutableMsg := msgs[0].WithTimeTick(100).WithLastConfirmed(msgID).IntoImmutableMessage(msgID) + var immutableMsg message.ImmutableMessage + for _, msg := range msgs { + if msg.VChannel() == "v1" { + immutableMsg = msg.WithTimeTick(100).WithLastConfirmed(msgID).IntoImmutableMessage(msgID) + break + } + } + require.NotNil(t, immutableMsg) replicateMsg := message.MustNewReplicateMessage("by-dev", immutableMsg.IntoImmutableMessageProto()) assert.NotNil(t, replicateMsg) diff --git a/pkg/streaming/util/message/codegen/main.go b/pkg/streaming/util/message/codegen/main.go index 8701a35f39..e620c5dd74 100644 --- a/pkg/streaming/util/message/codegen/main.go +++ b/pkg/streaming/util/message/codegen/main.go @@ -158,6 +158,7 @@ func (g *Generator) generateHelperFunctions(info JSONMessageReflectInfo) { f.Type().Id("Immutable"+baseName).Op("=").Qual(messagePackage, "SpecializedImmutableMessage").Types(headerType, bodyType) f.Type().Id("Broadcast"+baseName).Op("=").Qual(messagePackage, "SpecializedBroadcastMessage").Types(headerType, bodyType) f.Type().Id("BroadcastResult"+baseName).Op("=").Qual(messagePackage, "BroadcastResult").Types(headerType, bodyType) + f.Type().Id("AckResult"+baseName).Op("=").Qual(messagePackage, "AckResult").Types(headerType, bodyType) f.Line() // MessageTypeWithVersion constant diff --git a/pkg/streaming/util/message/marshal_log_object.go b/pkg/streaming/util/message/marshal_log_object.go index 8d3a6fe260..8940ef2d47 100644 --- a/pkg/streaming/util/message/marshal_log_object.go +++ b/pkg/streaming/util/message/marshal_log_object.go @@ -12,6 +12,8 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/rgpb" ) +const maxPrintedIDs = 1024 + // MarshalLogObject encodes the message into zap log object. func (m *messageImpl) MarshalLogObject(enc zapcore.ObjectEncoder) error { if m == nil { @@ -30,6 +32,7 @@ func (m *messageImpl) MarshalLogObject(enc zapcore.ObjectEncoder) error { if broadcast := m.BroadcastHeader(); broadcast != nil { enc.AddInt64("broadcastID", int64(broadcast.BroadcastID)) enc.AddString("broadcastVChannels", strings.Join(broadcast.VChannels, ",")) + enc.AddBool("ackSyncUp", broadcast.AckSyncUp) } if replicate := m.ReplicateHeader(); replicate != nil { enc.AddString("rClusterID", replicate.ClusterID) @@ -216,6 +219,10 @@ func marshalSpecializedHeader(t MessageType, v Version, h string, enc zapcore.Ob func encodeIDs(name string, targetIDs []int64, enc zapcore.ObjectEncoder) { ids := make([]string, 0, len(targetIDs)) for _, id := range targetIDs { + if len(ids) > maxPrintedIDs { + ids = append(ids, fmt.Sprintf("(with more %d)", len(targetIDs))) + break + } ids = append(ids, strconv.FormatInt(id, 10)) } enc.AddString(name, strings.Join(ids, "|")) diff --git a/pkg/streaming/util/message/message_test.go b/pkg/streaming/util/message/message_test.go index fd82aed890..e8479e5ff0 100644 --- a/pkg/streaming/util/message/message_test.go +++ b/pkg/streaming/util/message/message_test.go @@ -79,11 +79,11 @@ func TestBroadcast(t *testing.T) { msg, err := NewCreateCollectionMessageBuilderV1(). WithHeader(&CreateCollectionMessageHeader{}). WithBody(&msgpb.CreateCollectionRequest{}). - WithBroadcast([]string{"v1", "v2"}, NewExclusiveCollectionNameResourceKey("1", "2"), NewImportJobIDResourceKey(1)). + WithBroadcast([]string{"v1", "v2"}, OptBuildBroadcastAckSyncUp()). BuildBroadcast() assert.NoError(t, err) assert.NotNil(t, msg) - msg.WithBroadcastID(1) + msg.OverwriteBroadcastHeader(1, NewSharedDBNameResourceKey("1"), NewExclusiveCollectionNameResourceKey("1", "2")) msgs := msg.SplitIntoMutableMessage() assert.NotNil(t, msgs) assert.Len(t, msgs, 2) @@ -91,6 +91,8 @@ func TestBroadcast(t *testing.T) { assert.Equal(t, uint64(1), msgs[1].BroadcastHeader().BroadcastID) assert.Len(t, msgs[0].BroadcastHeader().ResourceKeys, 2) assert.ElementsMatch(t, []string{"v1", "v2"}, []string{msgs[0].VChannel(), msgs[1].VChannel()}) + assert.True(t, msgs[0].BroadcastHeader().AckSyncUp) + assert.True(t, msgs[1].BroadcastHeader().AckSyncUp) MustAsBroadcastCreateCollectionMessageV1(msg) } diff --git a/pkg/streaming/util/message/reflect_info.go b/pkg/streaming/util/message/reflect_info.go index 2a96f0c5b4..7885cff937 100644 --- a/pkg/streaming/util/message/reflect_info.go +++ b/pkg/streaming/util/message/reflect_info.go @@ -178,6 +178,7 @@ type ( ImmutableTimeTickMessageV1 = SpecializedImmutableMessage[*TimeTickMessageHeader, *TimeTickMsg] BroadcastTimeTickMessageV1 = SpecializedBroadcastMessage[*TimeTickMessageHeader, *TimeTickMsg] BroadcastResultTimeTickMessageV1 = BroadcastResult[*TimeTickMessageHeader, *TimeTickMsg] + AckResultTimeTickMessageV1 = AckResult[*TimeTickMessageHeader, *TimeTickMsg] ) // MessageTypeWithVersion for TimeTickMessageV1 @@ -219,6 +220,7 @@ type ( ImmutableInsertMessageV1 = SpecializedImmutableMessage[*InsertMessageHeader, *InsertRequest] BroadcastInsertMessageV1 = SpecializedBroadcastMessage[*InsertMessageHeader, *InsertRequest] BroadcastResultInsertMessageV1 = BroadcastResult[*InsertMessageHeader, *InsertRequest] + AckResultInsertMessageV1 = AckResult[*InsertMessageHeader, *InsertRequest] ) // MessageTypeWithVersion for InsertMessageV1 @@ -260,6 +262,7 @@ type ( ImmutableDeleteMessageV1 = SpecializedImmutableMessage[*DeleteMessageHeader, *DeleteRequest] BroadcastDeleteMessageV1 = SpecializedBroadcastMessage[*DeleteMessageHeader, *DeleteRequest] BroadcastResultDeleteMessageV1 = BroadcastResult[*DeleteMessageHeader, *DeleteRequest] + AckResultDeleteMessageV1 = AckResult[*DeleteMessageHeader, *DeleteRequest] ) // MessageTypeWithVersion for DeleteMessageV1 @@ -301,6 +304,7 @@ type ( ImmutableCreateCollectionMessageV1 = SpecializedImmutableMessage[*CreateCollectionMessageHeader, *CreateCollectionRequest] BroadcastCreateCollectionMessageV1 = SpecializedBroadcastMessage[*CreateCollectionMessageHeader, *CreateCollectionRequest] BroadcastResultCreateCollectionMessageV1 = BroadcastResult[*CreateCollectionMessageHeader, *CreateCollectionRequest] + AckResultCreateCollectionMessageV1 = AckResult[*CreateCollectionMessageHeader, *CreateCollectionRequest] ) // MessageTypeWithVersion for CreateCollectionMessageV1 @@ -342,6 +346,7 @@ type ( ImmutableDropCollectionMessageV1 = SpecializedImmutableMessage[*DropCollectionMessageHeader, *DropCollectionRequest] BroadcastDropCollectionMessageV1 = SpecializedBroadcastMessage[*DropCollectionMessageHeader, *DropCollectionRequest] BroadcastResultDropCollectionMessageV1 = BroadcastResult[*DropCollectionMessageHeader, *DropCollectionRequest] + AckResultDropCollectionMessageV1 = AckResult[*DropCollectionMessageHeader, *DropCollectionRequest] ) // MessageTypeWithVersion for DropCollectionMessageV1 @@ -383,6 +388,7 @@ type ( ImmutableTruncateCollectionMessageV2 = SpecializedImmutableMessage[*TruncateCollectionMessageHeader, *TruncateCollectionMessageBody] BroadcastTruncateCollectionMessageV2 = SpecializedBroadcastMessage[*TruncateCollectionMessageHeader, *TruncateCollectionMessageBody] BroadcastResultTruncateCollectionMessageV2 = BroadcastResult[*TruncateCollectionMessageHeader, *TruncateCollectionMessageBody] + AckResultTruncateCollectionMessageV2 = AckResult[*TruncateCollectionMessageHeader, *TruncateCollectionMessageBody] ) // MessageTypeWithVersion for TruncateCollectionMessageV2 @@ -424,6 +430,7 @@ type ( ImmutableCreatePartitionMessageV1 = SpecializedImmutableMessage[*CreatePartitionMessageHeader, *CreatePartitionRequest] BroadcastCreatePartitionMessageV1 = SpecializedBroadcastMessage[*CreatePartitionMessageHeader, *CreatePartitionRequest] BroadcastResultCreatePartitionMessageV1 = BroadcastResult[*CreatePartitionMessageHeader, *CreatePartitionRequest] + AckResultCreatePartitionMessageV1 = AckResult[*CreatePartitionMessageHeader, *CreatePartitionRequest] ) // MessageTypeWithVersion for CreatePartitionMessageV1 @@ -465,6 +472,7 @@ type ( ImmutableDropPartitionMessageV1 = SpecializedImmutableMessage[*DropPartitionMessageHeader, *DropPartitionRequest] BroadcastDropPartitionMessageV1 = SpecializedBroadcastMessage[*DropPartitionMessageHeader, *DropPartitionRequest] BroadcastResultDropPartitionMessageV1 = BroadcastResult[*DropPartitionMessageHeader, *DropPartitionRequest] + AckResultDropPartitionMessageV1 = AckResult[*DropPartitionMessageHeader, *DropPartitionRequest] ) // MessageTypeWithVersion for DropPartitionMessageV1 @@ -506,6 +514,7 @@ type ( ImmutableImportMessageV1 = SpecializedImmutableMessage[*ImportMessageHeader, *ImportMsg] BroadcastImportMessageV1 = SpecializedBroadcastMessage[*ImportMessageHeader, *ImportMsg] BroadcastResultImportMessageV1 = BroadcastResult[*ImportMessageHeader, *ImportMsg] + AckResultImportMessageV1 = AckResult[*ImportMessageHeader, *ImportMsg] ) // MessageTypeWithVersion for ImportMessageV1 @@ -547,6 +556,7 @@ type ( ImmutableCreateSegmentMessageV2 = SpecializedImmutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody] BroadcastCreateSegmentMessageV2 = SpecializedBroadcastMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody] BroadcastResultCreateSegmentMessageV2 = BroadcastResult[*CreateSegmentMessageHeader, *CreateSegmentMessageBody] + AckResultCreateSegmentMessageV2 = AckResult[*CreateSegmentMessageHeader, *CreateSegmentMessageBody] ) // MessageTypeWithVersion for CreateSegmentMessageV2 @@ -588,6 +598,7 @@ type ( ImmutableFlushMessageV2 = SpecializedImmutableMessage[*FlushMessageHeader, *FlushMessageBody] BroadcastFlushMessageV2 = SpecializedBroadcastMessage[*FlushMessageHeader, *FlushMessageBody] BroadcastResultFlushMessageV2 = BroadcastResult[*FlushMessageHeader, *FlushMessageBody] + AckResultFlushMessageV2 = AckResult[*FlushMessageHeader, *FlushMessageBody] ) // MessageTypeWithVersion for FlushMessageV2 @@ -629,6 +640,7 @@ type ( ImmutableManualFlushMessageV2 = SpecializedImmutableMessage[*ManualFlushMessageHeader, *ManualFlushMessageBody] BroadcastManualFlushMessageV2 = SpecializedBroadcastMessage[*ManualFlushMessageHeader, *ManualFlushMessageBody] BroadcastResultManualFlushMessageV2 = BroadcastResult[*ManualFlushMessageHeader, *ManualFlushMessageBody] + AckResultManualFlushMessageV2 = AckResult[*ManualFlushMessageHeader, *ManualFlushMessageBody] ) // MessageTypeWithVersion for ManualFlushMessageV2 @@ -670,6 +682,7 @@ type ( ImmutableAlterReplicateConfigMessageV2 = SpecializedImmutableMessage[*AlterReplicateConfigMessageHeader, *AlterReplicateConfigMessageBody] BroadcastAlterReplicateConfigMessageV2 = SpecializedBroadcastMessage[*AlterReplicateConfigMessageHeader, *AlterReplicateConfigMessageBody] BroadcastResultAlterReplicateConfigMessageV2 = BroadcastResult[*AlterReplicateConfigMessageHeader, *AlterReplicateConfigMessageBody] + AckResultAlterReplicateConfigMessageV2 = AckResult[*AlterReplicateConfigMessageHeader, *AlterReplicateConfigMessageBody] ) // MessageTypeWithVersion for AlterReplicateConfigMessageV2 @@ -711,6 +724,7 @@ type ( ImmutableBeginTxnMessageV2 = SpecializedImmutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody] BroadcastBeginTxnMessageV2 = SpecializedBroadcastMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody] BroadcastResultBeginTxnMessageV2 = BroadcastResult[*BeginTxnMessageHeader, *BeginTxnMessageBody] + AckResultBeginTxnMessageV2 = AckResult[*BeginTxnMessageHeader, *BeginTxnMessageBody] ) // MessageTypeWithVersion for BeginTxnMessageV2 @@ -752,6 +766,7 @@ type ( ImmutableCommitTxnMessageV2 = SpecializedImmutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody] BroadcastCommitTxnMessageV2 = SpecializedBroadcastMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody] BroadcastResultCommitTxnMessageV2 = BroadcastResult[*CommitTxnMessageHeader, *CommitTxnMessageBody] + AckResultCommitTxnMessageV2 = AckResult[*CommitTxnMessageHeader, *CommitTxnMessageBody] ) // MessageTypeWithVersion for CommitTxnMessageV2 @@ -793,6 +808,7 @@ type ( ImmutableRollbackTxnMessageV2 = SpecializedImmutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody] BroadcastRollbackTxnMessageV2 = SpecializedBroadcastMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody] BroadcastResultRollbackTxnMessageV2 = BroadcastResult[*RollbackTxnMessageHeader, *RollbackTxnMessageBody] + AckResultRollbackTxnMessageV2 = AckResult[*RollbackTxnMessageHeader, *RollbackTxnMessageBody] ) // MessageTypeWithVersion for RollbackTxnMessageV2 @@ -834,6 +850,7 @@ type ( ImmutableTxnMessageV2 = SpecializedImmutableMessage[*TxnMessageHeader, *TxnMessageBody] BroadcastTxnMessageV2 = SpecializedBroadcastMessage[*TxnMessageHeader, *TxnMessageBody] BroadcastResultTxnMessageV2 = BroadcastResult[*TxnMessageHeader, *TxnMessageBody] + AckResultTxnMessageV2 = AckResult[*TxnMessageHeader, *TxnMessageBody] ) // MessageTypeWithVersion for TxnMessageV2 @@ -854,6 +871,7 @@ type ( ImmutableSchemaChangeMessageV2 = SpecializedImmutableMessage[*SchemaChangeMessageHeader, *SchemaChangeMessageBody] BroadcastSchemaChangeMessageV2 = SpecializedBroadcastMessage[*SchemaChangeMessageHeader, *SchemaChangeMessageBody] BroadcastResultSchemaChangeMessageV2 = BroadcastResult[*SchemaChangeMessageHeader, *SchemaChangeMessageBody] + AckResultSchemaChangeMessageV2 = AckResult[*SchemaChangeMessageHeader, *SchemaChangeMessageBody] ) // MessageTypeWithVersion for SchemaChangeMessageV2 @@ -895,6 +913,7 @@ type ( ImmutableAlterCollectionMessageV2 = SpecializedImmutableMessage[*AlterCollectionMessageHeader, *AlterCollectionMessageBody] BroadcastAlterCollectionMessageV2 = SpecializedBroadcastMessage[*AlterCollectionMessageHeader, *AlterCollectionMessageBody] BroadcastResultAlterCollectionMessageV2 = BroadcastResult[*AlterCollectionMessageHeader, *AlterCollectionMessageBody] + AckResultAlterCollectionMessageV2 = AckResult[*AlterCollectionMessageHeader, *AlterCollectionMessageBody] ) // MessageTypeWithVersion for AlterCollectionMessageV2 @@ -936,6 +955,7 @@ type ( ImmutableAlterLoadConfigMessageV2 = SpecializedImmutableMessage[*AlterLoadConfigMessageHeader, *AlterLoadConfigMessageBody] BroadcastAlterLoadConfigMessageV2 = SpecializedBroadcastMessage[*AlterLoadConfigMessageHeader, *AlterLoadConfigMessageBody] BroadcastResultAlterLoadConfigMessageV2 = BroadcastResult[*AlterLoadConfigMessageHeader, *AlterLoadConfigMessageBody] + AckResultAlterLoadConfigMessageV2 = AckResult[*AlterLoadConfigMessageHeader, *AlterLoadConfigMessageBody] ) // MessageTypeWithVersion for AlterLoadConfigMessageV2 @@ -977,6 +997,7 @@ type ( ImmutableDropLoadConfigMessageV2 = SpecializedImmutableMessage[*DropLoadConfigMessageHeader, *DropLoadConfigMessageBody] BroadcastDropLoadConfigMessageV2 = SpecializedBroadcastMessage[*DropLoadConfigMessageHeader, *DropLoadConfigMessageBody] BroadcastResultDropLoadConfigMessageV2 = BroadcastResult[*DropLoadConfigMessageHeader, *DropLoadConfigMessageBody] + AckResultDropLoadConfigMessageV2 = AckResult[*DropLoadConfigMessageHeader, *DropLoadConfigMessageBody] ) // MessageTypeWithVersion for DropLoadConfigMessageV2 @@ -1018,6 +1039,7 @@ type ( ImmutableCreateDatabaseMessageV2 = SpecializedImmutableMessage[*CreateDatabaseMessageHeader, *CreateDatabaseMessageBody] BroadcastCreateDatabaseMessageV2 = SpecializedBroadcastMessage[*CreateDatabaseMessageHeader, *CreateDatabaseMessageBody] BroadcastResultCreateDatabaseMessageV2 = BroadcastResult[*CreateDatabaseMessageHeader, *CreateDatabaseMessageBody] + AckResultCreateDatabaseMessageV2 = AckResult[*CreateDatabaseMessageHeader, *CreateDatabaseMessageBody] ) // MessageTypeWithVersion for CreateDatabaseMessageV2 @@ -1059,6 +1081,7 @@ type ( ImmutableAlterDatabaseMessageV2 = SpecializedImmutableMessage[*AlterDatabaseMessageHeader, *AlterDatabaseMessageBody] BroadcastAlterDatabaseMessageV2 = SpecializedBroadcastMessage[*AlterDatabaseMessageHeader, *AlterDatabaseMessageBody] BroadcastResultAlterDatabaseMessageV2 = BroadcastResult[*AlterDatabaseMessageHeader, *AlterDatabaseMessageBody] + AckResultAlterDatabaseMessageV2 = AckResult[*AlterDatabaseMessageHeader, *AlterDatabaseMessageBody] ) // MessageTypeWithVersion for AlterDatabaseMessageV2 @@ -1100,6 +1123,7 @@ type ( ImmutableDropDatabaseMessageV2 = SpecializedImmutableMessage[*DropDatabaseMessageHeader, *DropDatabaseMessageBody] BroadcastDropDatabaseMessageV2 = SpecializedBroadcastMessage[*DropDatabaseMessageHeader, *DropDatabaseMessageBody] BroadcastResultDropDatabaseMessageV2 = BroadcastResult[*DropDatabaseMessageHeader, *DropDatabaseMessageBody] + AckResultDropDatabaseMessageV2 = AckResult[*DropDatabaseMessageHeader, *DropDatabaseMessageBody] ) // MessageTypeWithVersion for DropDatabaseMessageV2 @@ -1141,6 +1165,7 @@ type ( ImmutableAlterAliasMessageV2 = SpecializedImmutableMessage[*AlterAliasMessageHeader, *AlterAliasMessageBody] BroadcastAlterAliasMessageV2 = SpecializedBroadcastMessage[*AlterAliasMessageHeader, *AlterAliasMessageBody] BroadcastResultAlterAliasMessageV2 = BroadcastResult[*AlterAliasMessageHeader, *AlterAliasMessageBody] + AckResultAlterAliasMessageV2 = AckResult[*AlterAliasMessageHeader, *AlterAliasMessageBody] ) // MessageTypeWithVersion for AlterAliasMessageV2 @@ -1182,6 +1207,7 @@ type ( ImmutableDropAliasMessageV2 = SpecializedImmutableMessage[*DropAliasMessageHeader, *DropAliasMessageBody] BroadcastDropAliasMessageV2 = SpecializedBroadcastMessage[*DropAliasMessageHeader, *DropAliasMessageBody] BroadcastResultDropAliasMessageV2 = BroadcastResult[*DropAliasMessageHeader, *DropAliasMessageBody] + AckResultDropAliasMessageV2 = AckResult[*DropAliasMessageHeader, *DropAliasMessageBody] ) // MessageTypeWithVersion for DropAliasMessageV2 @@ -1223,6 +1249,7 @@ type ( ImmutableAlterUserMessageV2 = SpecializedImmutableMessage[*AlterUserMessageHeader, *AlterUserMessageBody] BroadcastAlterUserMessageV2 = SpecializedBroadcastMessage[*AlterUserMessageHeader, *AlterUserMessageBody] BroadcastResultAlterUserMessageV2 = BroadcastResult[*AlterUserMessageHeader, *AlterUserMessageBody] + AckResultAlterUserMessageV2 = AckResult[*AlterUserMessageHeader, *AlterUserMessageBody] ) // MessageTypeWithVersion for AlterUserMessageV2 @@ -1264,6 +1291,7 @@ type ( ImmutableDropUserMessageV2 = SpecializedImmutableMessage[*DropUserMessageHeader, *DropUserMessageBody] BroadcastDropUserMessageV2 = SpecializedBroadcastMessage[*DropUserMessageHeader, *DropUserMessageBody] BroadcastResultDropUserMessageV2 = BroadcastResult[*DropUserMessageHeader, *DropUserMessageBody] + AckResultDropUserMessageV2 = AckResult[*DropUserMessageHeader, *DropUserMessageBody] ) // MessageTypeWithVersion for DropUserMessageV2 @@ -1305,6 +1333,7 @@ type ( ImmutableAlterRoleMessageV2 = SpecializedImmutableMessage[*AlterRoleMessageHeader, *AlterRoleMessageBody] BroadcastAlterRoleMessageV2 = SpecializedBroadcastMessage[*AlterRoleMessageHeader, *AlterRoleMessageBody] BroadcastResultAlterRoleMessageV2 = BroadcastResult[*AlterRoleMessageHeader, *AlterRoleMessageBody] + AckResultAlterRoleMessageV2 = AckResult[*AlterRoleMessageHeader, *AlterRoleMessageBody] ) // MessageTypeWithVersion for AlterRoleMessageV2 @@ -1346,6 +1375,7 @@ type ( ImmutableDropRoleMessageV2 = SpecializedImmutableMessage[*DropRoleMessageHeader, *DropRoleMessageBody] BroadcastDropRoleMessageV2 = SpecializedBroadcastMessage[*DropRoleMessageHeader, *DropRoleMessageBody] BroadcastResultDropRoleMessageV2 = BroadcastResult[*DropRoleMessageHeader, *DropRoleMessageBody] + AckResultDropRoleMessageV2 = AckResult[*DropRoleMessageHeader, *DropRoleMessageBody] ) // MessageTypeWithVersion for DropRoleMessageV2 @@ -1387,6 +1417,7 @@ type ( ImmutableAlterUserRoleMessageV2 = SpecializedImmutableMessage[*AlterUserRoleMessageHeader, *AlterUserRoleMessageBody] BroadcastAlterUserRoleMessageV2 = SpecializedBroadcastMessage[*AlterUserRoleMessageHeader, *AlterUserRoleMessageBody] BroadcastResultAlterUserRoleMessageV2 = BroadcastResult[*AlterUserRoleMessageHeader, *AlterUserRoleMessageBody] + AckResultAlterUserRoleMessageV2 = AckResult[*AlterUserRoleMessageHeader, *AlterUserRoleMessageBody] ) // MessageTypeWithVersion for AlterUserRoleMessageV2 @@ -1428,6 +1459,7 @@ type ( ImmutableDropUserRoleMessageV2 = SpecializedImmutableMessage[*DropUserRoleMessageHeader, *DropUserRoleMessageBody] BroadcastDropUserRoleMessageV2 = SpecializedBroadcastMessage[*DropUserRoleMessageHeader, *DropUserRoleMessageBody] BroadcastResultDropUserRoleMessageV2 = BroadcastResult[*DropUserRoleMessageHeader, *DropUserRoleMessageBody] + AckResultDropUserRoleMessageV2 = AckResult[*DropUserRoleMessageHeader, *DropUserRoleMessageBody] ) // MessageTypeWithVersion for DropUserRoleMessageV2 @@ -1469,6 +1501,7 @@ type ( ImmutableAlterPrivilegeMessageV2 = SpecializedImmutableMessage[*AlterPrivilegeMessageHeader, *AlterPrivilegeMessageBody] BroadcastAlterPrivilegeMessageV2 = SpecializedBroadcastMessage[*AlterPrivilegeMessageHeader, *AlterPrivilegeMessageBody] BroadcastResultAlterPrivilegeMessageV2 = BroadcastResult[*AlterPrivilegeMessageHeader, *AlterPrivilegeMessageBody] + AckResultAlterPrivilegeMessageV2 = AckResult[*AlterPrivilegeMessageHeader, *AlterPrivilegeMessageBody] ) // MessageTypeWithVersion for AlterPrivilegeMessageV2 @@ -1510,6 +1543,7 @@ type ( ImmutableDropPrivilegeMessageV2 = SpecializedImmutableMessage[*DropPrivilegeMessageHeader, *DropPrivilegeMessageBody] BroadcastDropPrivilegeMessageV2 = SpecializedBroadcastMessage[*DropPrivilegeMessageHeader, *DropPrivilegeMessageBody] BroadcastResultDropPrivilegeMessageV2 = BroadcastResult[*DropPrivilegeMessageHeader, *DropPrivilegeMessageBody] + AckResultDropPrivilegeMessageV2 = AckResult[*DropPrivilegeMessageHeader, *DropPrivilegeMessageBody] ) // MessageTypeWithVersion for DropPrivilegeMessageV2 @@ -1551,6 +1585,7 @@ type ( ImmutableAlterPrivilegeGroupMessageV2 = SpecializedImmutableMessage[*AlterPrivilegeGroupMessageHeader, *AlterPrivilegeGroupMessageBody] BroadcastAlterPrivilegeGroupMessageV2 = SpecializedBroadcastMessage[*AlterPrivilegeGroupMessageHeader, *AlterPrivilegeGroupMessageBody] BroadcastResultAlterPrivilegeGroupMessageV2 = BroadcastResult[*AlterPrivilegeGroupMessageHeader, *AlterPrivilegeGroupMessageBody] + AckResultAlterPrivilegeGroupMessageV2 = AckResult[*AlterPrivilegeGroupMessageHeader, *AlterPrivilegeGroupMessageBody] ) // MessageTypeWithVersion for AlterPrivilegeGroupMessageV2 @@ -1592,6 +1627,7 @@ type ( ImmutableDropPrivilegeGroupMessageV2 = SpecializedImmutableMessage[*DropPrivilegeGroupMessageHeader, *DropPrivilegeGroupMessageBody] BroadcastDropPrivilegeGroupMessageV2 = SpecializedBroadcastMessage[*DropPrivilegeGroupMessageHeader, *DropPrivilegeGroupMessageBody] BroadcastResultDropPrivilegeGroupMessageV2 = BroadcastResult[*DropPrivilegeGroupMessageHeader, *DropPrivilegeGroupMessageBody] + AckResultDropPrivilegeGroupMessageV2 = AckResult[*DropPrivilegeGroupMessageHeader, *DropPrivilegeGroupMessageBody] ) // MessageTypeWithVersion for DropPrivilegeGroupMessageV2 @@ -1633,6 +1669,7 @@ type ( ImmutableRestoreRBACMessageV2 = SpecializedImmutableMessage[*RestoreRBACMessageHeader, *RestoreRBACMessageBody] BroadcastRestoreRBACMessageV2 = SpecializedBroadcastMessage[*RestoreRBACMessageHeader, *RestoreRBACMessageBody] BroadcastResultRestoreRBACMessageV2 = BroadcastResult[*RestoreRBACMessageHeader, *RestoreRBACMessageBody] + AckResultRestoreRBACMessageV2 = AckResult[*RestoreRBACMessageHeader, *RestoreRBACMessageBody] ) // MessageTypeWithVersion for RestoreRBACMessageV2 @@ -1674,6 +1711,7 @@ type ( ImmutableAlterResourceGroupMessageV2 = SpecializedImmutableMessage[*AlterResourceGroupMessageHeader, *AlterResourceGroupMessageBody] BroadcastAlterResourceGroupMessageV2 = SpecializedBroadcastMessage[*AlterResourceGroupMessageHeader, *AlterResourceGroupMessageBody] BroadcastResultAlterResourceGroupMessageV2 = BroadcastResult[*AlterResourceGroupMessageHeader, *AlterResourceGroupMessageBody] + AckResultAlterResourceGroupMessageV2 = AckResult[*AlterResourceGroupMessageHeader, *AlterResourceGroupMessageBody] ) // MessageTypeWithVersion for AlterResourceGroupMessageV2 @@ -1715,6 +1753,7 @@ type ( ImmutableDropResourceGroupMessageV2 = SpecializedImmutableMessage[*DropResourceGroupMessageHeader, *DropResourceGroupMessageBody] BroadcastDropResourceGroupMessageV2 = SpecializedBroadcastMessage[*DropResourceGroupMessageHeader, *DropResourceGroupMessageBody] BroadcastResultDropResourceGroupMessageV2 = BroadcastResult[*DropResourceGroupMessageHeader, *DropResourceGroupMessageBody] + AckResultDropResourceGroupMessageV2 = AckResult[*DropResourceGroupMessageHeader, *DropResourceGroupMessageBody] ) // MessageTypeWithVersion for DropResourceGroupMessageV2 @@ -1756,6 +1795,7 @@ type ( ImmutableCreateIndexMessageV2 = SpecializedImmutableMessage[*CreateIndexMessageHeader, *CreateIndexMessageBody] BroadcastCreateIndexMessageV2 = SpecializedBroadcastMessage[*CreateIndexMessageHeader, *CreateIndexMessageBody] BroadcastResultCreateIndexMessageV2 = BroadcastResult[*CreateIndexMessageHeader, *CreateIndexMessageBody] + AckResultCreateIndexMessageV2 = AckResult[*CreateIndexMessageHeader, *CreateIndexMessageBody] ) // MessageTypeWithVersion for CreateIndexMessageV2 @@ -1797,6 +1837,7 @@ type ( ImmutableAlterIndexMessageV2 = SpecializedImmutableMessage[*AlterIndexMessageHeader, *AlterIndexMessageBody] BroadcastAlterIndexMessageV2 = SpecializedBroadcastMessage[*AlterIndexMessageHeader, *AlterIndexMessageBody] BroadcastResultAlterIndexMessageV2 = BroadcastResult[*AlterIndexMessageHeader, *AlterIndexMessageBody] + AckResultAlterIndexMessageV2 = AckResult[*AlterIndexMessageHeader, *AlterIndexMessageBody] ) // MessageTypeWithVersion for AlterIndexMessageV2 @@ -1838,6 +1879,7 @@ type ( ImmutableDropIndexMessageV2 = SpecializedImmutableMessage[*DropIndexMessageHeader, *DropIndexMessageBody] BroadcastDropIndexMessageV2 = SpecializedBroadcastMessage[*DropIndexMessageHeader, *DropIndexMessageBody] BroadcastResultDropIndexMessageV2 = BroadcastResult[*DropIndexMessageHeader, *DropIndexMessageBody] + AckResultDropIndexMessageV2 = AckResult[*DropIndexMessageHeader, *DropIndexMessageBody] ) // MessageTypeWithVersion for DropIndexMessageV2