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