enhance: support AckSyncUp for broadcaster, and enable it in truncate api (#46313)

issue: #43897
also for issue: #46166

add ack_sync_up flag into broadcast message header, which indicates that
whether the broadcast operation is need to be synced up between the
streaming node and the coordinator.
If the ack_sync_up is false, the broadcast operation will be acked once
the recovery storage see the message at current vchannel, the fast ack
operation can be applied to speed up the broadcast operation.
If the ack_sync_up is true, the broadcast operation will be acked after
the checkpoint of current vchannel reach current message.
The fast ack operation can not be applied to speed up the broadcast
operation, because the ack operation need to be synced up with streaming
node.
e.g. if truncate collection operation want to call ack once callback
after the all segment are flushed at current vchannel, it should set the
ack_sync_up to be true.

TODO: current implementation doesn't promise the ack sync up semantic,
it only promise FastAck operation will not be applied, wait for 3.0 to
implement the ack sync up semantic. only for truncate api now.

---------

Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
Zhen Ye 2025-12-17 16:55:17 +08:00 committed by GitHub
parent 46c14781be
commit 7c575a18b0
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
38 changed files with 1000 additions and 314 deletions

View File

@ -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
}

View File

@ -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": {

View File

@ -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)

View File

@ -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)

View File

@ -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))

View File

@ -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 {

View File

@ -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()

View File

@ -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))

View File

@ -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.

View File

@ -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.

View File

@ -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,
}
}

View File

@ -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
}

View File

@ -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()

View File

@ -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)
}

View File

@ -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 {

View File

@ -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)

View File

@ -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)

View File

@ -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.

View File

@ -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{

View File

@ -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

View File

@ -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)
}

View File

@ -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](),
}
}

View File

@ -8,6 +8,7 @@ import (
// init the message ack callbacks
func init() {
resetMessageAckOnceCallbacks()
resetMessageAckCallbacks()
resetMessageCheckCallbacks()
}

View File

@ -5,5 +5,6 @@ package registry
func ResetRegistration() {
resetMessageAckCallbacks()
resetMessageAckOnceCallbacks()
resetMessageCheckCallbacks()
}

View File

@ -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()

View File

@ -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()) {

View File

@ -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

View File

@ -70,7 +70,11 @@ message CollectionInfo {
repeated common.KeyValuePair properties = 14;
int64 db_id = 15;
uint64 UpdateTimestamp = 16;
map<string, uint64> 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 {

View File

@ -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

View File

@ -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.

View File

@ -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,

View File

@ -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)

View File

@ -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")
}

View File

@ -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)

View File

@ -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

View File

@ -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, "|"))

View File

@ -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)
}

View File

@ -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