milvus/pkg/proto/messages.proto
Zhen Ye 7c575a18b0
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>
2025-12-17 16:55:17 +08:00

690 lines
24 KiB
Protocol Buffer

syntax = "proto3";
package milvus.proto.messages;
option go_package = "github.com/milvus-io/milvus/pkg/v2/proto/messagespb";
import "common.proto";
import "schema.proto";
import "data_coord.proto"; // for SegmentLevel, but it's a basic type should not be in datacoord.proto.
import "index_coord.proto";
import "internal.proto";
import "milvus.proto";
import "rg.proto";
import "google/protobuf/field_mask.proto";
// Message is the basic unit of communication between publisher and consumer.
message Message {
bytes payload = 1; // message body
map<string, string> properties = 2; // message properties
}
// MessageType is the type of message.
enum MessageType {
Unknown = 0;
TimeTick = 1;
Insert = 2;
Delete = 3;
Flush = 4;
CreateCollection = 5;
DropCollection = 6;
CreatePartition = 7;
DropPartition = 8;
ManualFlush = 9;
CreateSegment = 10;
Import = 11;
SchemaChange = 12 [deprecated = true]; // merged into AlterCollection
AlterCollection = 13;
AlterLoadConfig = 14; // load config is simple, so CreateLoadConfig and AlterLoadConfig share one message.
DropLoadConfig = 15;
CreateDatabase = 16;
AlterDatabase = 17;
DropDatabase = 18;
AlterAlias = 19; // alias is simple, so CreateAlias and AlterAlias share one message.
DropAlias = 20;
RestoreRBAC = 21;
AlterUser = 22; // user is simple, so CreateUser and AlterUser share one message.
DropUser = 23;
AlterRole = 24; // role is simple, so CreateRole and AlterRole share one message.
DropRole = 25;
AlterUserRole = 26; // user role is simple, so CreateUserRole and AlterUserRole share one message.
DropUserRole = 27;
AlterPrivilege = 28; // privilege is simple, so CreatePrivilege and AlterPrivilege share one message.
DropPrivilege = 29;
AlterPrivilegeGroup = 30; // privilege group is simple, so CreatePrivilegeGroup and AlterPrivilegeGroup share one message.
DropPrivilegeGroup = 31;
AlterResourceGroup = 32; // resource group is simple, so CreateResourceGroup and AlterResourceGroup share one message.
DropResourceGroup = 33;
CreateIndex = 34;
AlterIndex = 35;
DropIndex = 36;
FlushAll = 37;
TruncateCollection = 38;
// AlterReplicateConfig is used to alter the replicate configuration to the current cluster.
// When the AlterReplicateConfig message is received, the replication topology is changed.
// Maybe some cluster give up the leader role, no any other message will be received from this cluster.
// So leader will stop writing message into wal and stop replicating any message to the other cluster,
// and the follower will stop receiving any message from the old leader.
// New leader will start to write message into wal and start replicating message to the other cluster.
AlterReplicateConfig = 800;
// begin transaction message is only used for transaction, once a begin
// transaction message is received, all messages combined with the
// transaction message cannot be consumed until a CommitTxn message
// is received.
BeginTxn = 900;
// commit transaction message is only used for transaction, once a commit
// transaction message is received, all messages combined with the
// transaction message can be consumed, the message combined with the
// transaction which is received after the commit transaction message will
// be drop.
CommitTxn = 901;
// rollback transaction message is only used for transaction, once a
// rollback transaction message is received, all messages combined with the
// transaction message can be discarded, the message combined with the
// transaction which is received after the rollback transaction message will
// be drop.
RollbackTxn = 902;
// txn message is a set of messages combined by multiple messages in a
// transaction. the txn properties is consist of the begin txn message and
// commit txn message.
Txn = 999;
}
///
/// Message Payload Definitions
/// Some message payload is defined at msg.proto at milvus-proto for
/// compatibility.
/// 1. InsertRequest
/// 2. DeleteRequest
/// 3. TimeTickRequest
/// 4. CreateCollectionRequest
/// 5. DropCollectionRequest
/// 6. CreatePartitionRequest
/// 7. DropPartitionRequest
///
// FlushMessageBody is the body of flush message.
message FlushMessageBody {}
// ManualFlushMessageBody is the body of manual flush message.
message ManualFlushMessageBody {}
// CreateSegmentMessageBody is the body of create segment message.
message CreateSegmentMessageBody {}
// BeginTxnMessageBody is the body of begin transaction message.
// Just do nothing now.
message BeginTxnMessageBody {}
// CommitTxnMessageBody is the body of commit transaction message.
// Just do nothing now.
message CommitTxnMessageBody {}
// RollbackTxnMessageBody is the body of rollback transaction message.
// Just do nothing now.
message RollbackTxnMessageBody {}
// TxnMessageBody is the body of transaction message.
// A transaction message is combined by multiple messages.
// It's only can be seen at consume side.
// All message in a transaction message only has same timetick which is equal to
// the CommitTransationMessage.
message TxnMessageBody {
repeated Message messages = 1;
}
///
/// Message Header Definitions
/// Used to fast handling at streaming node write ahead.
/// The header should be simple and light enough to be parsed.
/// Do not alter too much information in the header if unnecessary.
///
// TimeTickMessageHeader just nothing.
message TimeTickMessageHeader {}
// InsertMessageHeader is the header of insert message.
message InsertMessageHeader {
int64 collection_id = 1;
repeated PartitionSegmentAssignment partitions = 2;
}
// PartitionSegmentAssignment is the segment assignment of a partition.
message PartitionSegmentAssignment {
int64 partition_id = 1;
uint64 rows = 2;
uint64 binary_size = 3;
SegmentAssignment segment_assignment = 4;
}
// SegmentAssignment is the assignment of a segment.
message SegmentAssignment {
int64 segment_id = 1;
}
// DeleteMessageHeader
message DeleteMessageHeader {
int64 collection_id = 1;
uint64 rows = 2;
}
// FlushMessageHeader just nothing.
message FlushMessageHeader {
int64 collection_id = 1;
int64 partition_id = 2;
int64 segment_id = 3;
}
// CreateSegmentMessageHeader just nothing.
message CreateSegmentMessageHeader {
int64 collection_id = 1;
int64 partition_id = 2;
int64 segment_id = 3;
int64 storage_version = 4; // the storage version of the segment.
uint64 max_segment_size = 5; // the max size bytes of the segment.
uint64 max_rows = 6; // the max rows of the segment.
data.SegmentLevel level = 7; // the level of the segment.
}
message ManualFlushMessageHeader {
int64 collection_id = 1;
uint64 flush_ts = 2;
repeated int64 segment_ids = 3; // the segment ids to be flushed, will be filled by wal shard manager.
}
// CreateCollectionMessageHeader is the header of create collection message.
message CreateCollectionMessageHeader {
int64 collection_id = 1;
repeated int64 partition_ids = 2;
int64 db_id = 3;
}
// DropCollectionMessageHeader is the header of drop collection message.
message DropCollectionMessageHeader {
int64 collection_id = 1;
int64 db_id = 2;
}
// CreatePartitionMessageHeader is the header of create partition message.
message CreatePartitionMessageHeader {
int64 collection_id = 1;
int64 partition_id = 2;
}
// DropPartitionMessageHeader is the header of drop partition message.
message DropPartitionMessageHeader {
int64 collection_id = 1;
int64 partition_id = 2;
}
// AlterReplicateConfigMessageHeader is the header of alter replicate configuration message.
message AlterReplicateConfigMessageHeader {
common.ReplicateConfiguration replicate_configuration = 1;
}
// AlterReplicateConfigMessageBody is the body of alter replicate configuration message.
message AlterReplicateConfigMessageBody {
}
// BeginTxnMessageHeader is the header of begin transaction message.
// Just do nothing now.
// Add Channel info here to implement cross pchannel transaction.
message BeginTxnMessageHeader {
// the max milliseconds to keep alive of the transaction.
// the keepalive_milliseconds is never changed in a transaction by now,
int64 keepalive_milliseconds = 1;
}
// CommitTxnMessageHeader is the header of commit transaction message.
// Just do nothing now.
message CommitTxnMessageHeader {}
// RollbackTxnMessageHeader is the header of rollback transaction
// message.
// Just do nothing now.
message RollbackTxnMessageHeader {}
// TxnMessageHeader is the header of transaction message.
// Just do nothing now.
message TxnMessageHeader {}
message ImportMessageHeader {}
// SchemaChangeMessageHeader is the header of CollectionSchema update message.
message SchemaChangeMessageHeader{
int64 collection_id = 1;
repeated int64 flushed_segment_ids = 2; // will be filled by wal shard manager.
}
// SchemaChangeMessageBody is the body of CollectionSchema update message.
message SchemaChangeMessageBody{
schema.CollectionSchema schema = 1;
}
// AlterCollectionMessageHeader is the header of alter collection message.
message AlterCollectionMessageHeader {
int64 db_id = 1;
int64 collection_id = 2;
google.protobuf.FieldMask update_mask = 3;
CacheExpirations cache_expirations = 4;
repeated int64 flushed_segment_ids = 5; // will be filled by wal shard manager.
}
// AlterCollectionMessageBody is the body of alter collection message.
message AlterCollectionMessageBody {
AlterCollectionMessageUpdates updates = 1;
}
// AlterCollectionMessageUpdates is the updates of alter collection message.
message AlterCollectionMessageUpdates {
int64 db_id = 1; // collection db id should be updated.
string db_name = 2; // collection db name should be updated.
string collection_name = 3; // collection name should be updated.
string description = 4; // collection description should be updated.
schema.CollectionSchema schema = 5; // collection schema should be updated.
common.ConsistencyLevel consistency_level = 6; // consistency level should be updated.
repeated common.KeyValuePair properties = 7; // collection properties should be updated.
AlterLoadConfigOfAlterCollection alter_load_config = 8; // alter load config of alter collection.
}
// AlterLoadConfigOfAlterCollection is the body of alter load config of alter collection message.
message AlterLoadConfigOfAlterCollection {
int32 replica_number = 1;
repeated string resource_groups = 2;
}
// AlterLoadConfigMessageHeader is the header of alter load config message.
message AlterLoadConfigMessageHeader {
int64 db_id = 1;
int64 collection_id = 2; // the collection id that has to be loaded.
repeated int64 partition_ids = 3; // the partition ids that has to be loaded, empty means no partition has to be loaded.
repeated LoadFieldConfig load_fields = 4; // the field id that has to be loaded.
repeated LoadReplicaConfig replicas = 5; // the replicas that has to be loaded.
bool user_specified_replica_mode = 6; // whether the replica mode is user specified.
}
// AlterLoadConfigMessageBody is the body of alter load config message.
message AlterLoadConfigMessageBody {
}
// LoadFieldConfig is the config to load fields.
message LoadFieldConfig {
int64 field_id = 1;
int64 index_id = 2;
}
// LoadReplicaConfig is the config of a replica.
message LoadReplicaConfig {
int64 replica_id = 1;
string resource_group_name = 2;
common.LoadPriority priority = 3;
}
message DropLoadConfigMessageHeader {
int64 db_id = 1;
int64 collection_id = 2;
}
message DropLoadConfigMessageBody {}
// CreateDatabaseMessageHeader is the header of create database message.
message CreateDatabaseMessageHeader {
string db_name = 1;
int64 db_id = 2;
}
// CreateDatabaseMessageBody is the body of create database message.
message CreateDatabaseMessageBody {
repeated common.KeyValuePair properties = 1;
}
// AlterDatabaseMessageHeader is the header of alter database message.
message AlterDatabaseMessageHeader {
string db_name = 1;
int64 db_id = 2;
}
// AlterDatabaseMessageBody is the body of alter database message.
message AlterDatabaseMessageBody {
repeated common.KeyValuePair properties = 1;
AlterLoadConfigOfAlterDatabase alter_load_config = 2;
}
// AlterLoadConfigOfAlterDatabase is the body of alter load config of alter database message.
// When the database's resource group or replica number is changed, the load config of all collection in database will be updated.
message AlterLoadConfigOfAlterDatabase {
repeated int64 collection_ids = 1;
int32 replica_number = 2;
repeated string resource_groups = 3;
}
// DropDatabaseMessageHeader is the header of drop database message.
message DropDatabaseMessageHeader {
string db_name = 1;
int64 db_id = 2;
}
// DropDatabaseMessageBody is the body of drop database message.
message DropDatabaseMessageBody {
}
// AlterAliasMessageHeader is the header of alter alias message.
message AlterAliasMessageHeader {
int64 db_id = 1;
string db_name = 2;
int64 collection_id = 3;
string collection_name = 4;
string alias = 5;
}
// AlterAliasMessageBody is the body of alter alias message.
message AlterAliasMessageBody {
}
// DropAliasMessageHeader is the header of drop alias message.
message DropAliasMessageHeader {
int64 db_id = 1;
string db_name = 2;
string alias = 3;
}
// DropAliasMessageBody is the body of drop alias message.
message DropAliasMessageBody {
}
message CreateUserMessageHeader {
milvus.UserEntity user_entity = 1;
}
message CreateUserMessageBody {
internal.CredentialInfo credential_info = 1;
}
// AlterUserMessageHeader is the header of alter user message.
message AlterUserMessageHeader {
milvus.UserEntity user_entity = 1;
}
// AlterUserMessageBody is the body of alter user message.
message AlterUserMessageBody {
internal.CredentialInfo credential_info = 1;
}
// DropUserMessageHeader is the header of drop user message.
message DropUserMessageHeader {
string user_name = 1;
}
// DropUserMessageBody is the body of drop user message.
message DropUserMessageBody {}
// AlterRoleMessageHeader is the header of alter role message.
message AlterRoleMessageHeader {
milvus.RoleEntity role_entity = 1;
}
// AlterRoleMessageBody is the body of alter role message.
message AlterRoleMessageBody {
}
// DropRoleMessageHeader is the header of drop role message.
message DropRoleMessageHeader {
string role_name = 1;
bool force_drop = 2; // if true, the role will be dropped even if it has privileges.
}
// DropRoleMessageBody is the body of drop role message.
message DropRoleMessageBody {}
// RoleBinding is the binding of user and role.
message RoleBinding {
milvus.UserEntity user_entity = 1;
milvus.RoleEntity role_entity = 2;
}
// AlterUserRoleMessageHeader is the header of alter user role message.
message AlterUserRoleMessageHeader {
RoleBinding role_binding = 1; // TODO: support multiple role and user bindings in future.
}
// AlterUserRoleMessageBody is the body of alter user role message.
message AlterUserRoleMessageBody {}
// DropUserRoleMessageHeader is the header of drop user role message.
message DropUserRoleMessageHeader {
RoleBinding role_binding = 1; // TODO: support multiple role and user bindings in future.
}
// DropUserRoleMessageBody is the body of drop user role message.
message DropUserRoleMessageBody {}
// RestoreRBACMessageHeader is the header of restore rbac message.
message RestoreRBACMessageHeader {
}
// RestoreRBACMessageBody is the body of restore rbac message.
message RestoreRBACMessageBody {
milvus.RBACMeta rbac_meta = 1;
}
// AlterPrivilegeMessageHeader is the header of grant privilege message.
message AlterPrivilegeMessageHeader {
milvus.GrantEntity entity = 1;
}
// AlterPrivilegeMessageBody is the body of grant privilege message.
message AlterPrivilegeMessageBody {
}
// DropPrivilegeMessageHeader is the header of revoke privilege message.
message DropPrivilegeMessageHeader {
milvus.GrantEntity entity = 1;
}
// DropPrivilegeMessageBody is the body of revoke privilege message.
message DropPrivilegeMessageBody {}
// AlterPrivilegeGroupMessageHeader is the header of alter privilege group message.
message AlterPrivilegeGroupMessageHeader {
milvus.PrivilegeGroupInfo privilege_group_info = 1; // if privileges is empty, new privilege group will be created.
}
// AlterPrivilegeGroupMessageBody is the body of alter privilege group message.
message AlterPrivilegeGroupMessageBody {}
// DropPrivilegeGroupMessageHeader is the header of drop privilege group message.
message DropPrivilegeGroupMessageHeader {
milvus.PrivilegeGroupInfo privilege_group_info = 1; // if privileges is empty, privilege group will be dropped.
}
// DropPrivilegeGroupMessageBody is the body of drop privilege group message.
message DropPrivilegeGroupMessageBody {}
// AlterResourceGroupMessageHeader is the header of alter resource group message.
message AlterResourceGroupMessageHeader {
map<string, rg.ResourceGroupConfig> resource_group_configs = 3;
}
// AlterResourceGroupMessageBody is the body of alter resource group message.
message AlterResourceGroupMessageBody {}
// DropResourceGroupMessageHeader is the header of drop resource group message.
message DropResourceGroupMessageHeader {
string resource_group_name = 1;
}
// DropResourceGroupMessageBody is the body of drop resource group message.
message DropResourceGroupMessageBody {
}
// CreateIndexMessageHeader is the header of create index message.
message CreateIndexMessageHeader {
int64 db_id = 1;
int64 collection_id = 2;
int64 field_id = 3;
int64 index_id = 4;
string index_name = 5;
}
// CreateIndexMessageBody is the body of create index message.
message CreateIndexMessageBody {
index.FieldIndex field_index = 1;
}
// AlterIndexMessageHeader is the header of alter index message.
message AlterIndexMessageHeader {
int64 collection_id = 1;
repeated int64 index_ids = 2;
}
// AlterIndexMessageBody is the body of alter index message.
message AlterIndexMessageBody {
repeated index.FieldIndex field_indexes = 1;
}
// DropIndexMessageHeader is the header of drop index message.
message DropIndexMessageHeader {
int64 collection_id = 1;
repeated int64 index_ids = 2; // drop all indexes if empty.
}
// DropIndexMessageBody is the body of drop index message.
message DropIndexMessageBody {}
// CacheExpirations is the cache expirations of proxy collection meta cache.
message CacheExpirations {
repeated CacheExpiration cache_expirations = 1;
}
// CacheExpiration is the cache expiration of proxy collection meta cache.
message CacheExpiration {
oneof cache {
// LegacyProxyCollectionMetaCache is the cache expiration of legacy proxy collection meta cache.
LegacyProxyCollectionMetaCache legacy_proxy_collection_meta_cache = 1;
}
}
// LegacyProxyCollectionMetaCache is the cache expiration of legacy proxy collection meta cache.
message LegacyProxyCollectionMetaCache {
string db_name = 1;
string collection_name = 2;
int64 collection_id = 3;
string partition_name = 4;
common.MsgType msg_type = 5;
}
///
/// Message Extra Response
/// Used to add extra information when response to the client.
///
///
// ManualFlushExtraResponse is the extra response of manual flush message.
message ManualFlushExtraResponse {
repeated int64 segment_ids = 1;
}
message FlushAllMessageHeader {}
message FlushAllMessageBody {}
// TxnContext is the context of transaction.
// It will be carried by every message in a transaction.
message TxnContext {
// the unique id of the transaction.
// the txn_id is never changed in a transaction.
int64 txn_id = 1;
// the next keep alive timeout of the transaction.
// after the keep alive timeout, the transaction will be expired.
int64 keepalive_milliseconds = 2;
}
enum TxnState {
// should never be used.
TxnUnknown = 0;
// the transaction is in flight.
TxnInFlight = 1;
// the transaction is on commit.
TxnOnCommit = 2;
// the transaction is committed.
TxnCommitted = 3;
// the transaction is on rollback.
TxnOnRollback = 4;
// the transaction is rollbacked.
TxnRollbacked = 5;
}
// RMQMessageLayout is the layout of message for RMQ.
message RMQMessageLayout {
bytes payload = 1; // message body
map<string, string> properties = 2; // message properties
}
// BroadcastHeader is the common header of broadcast message.
message BroadcastHeader {
uint64 broadcast_id = 1;
repeated string vchannels = 2;
repeated ResourceKey Resource_keys = 3; // the resource key of the broadcast message.
// Once the broadcast is sent, the resource of resource key will be hold.
// New broadcast message with the same resource key will be rejected.
// And the user can watch the resource key to known when the resource is released.
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.
message ReplicateHeader {
string cluster_id = 1; // the cluster id of source cluster
common.MessageID message_id = 2; // the message id of replicate msg from source cluster
common.MessageID last_confirmed_message_id = 3; // the last confirmed message id of replicate msg from source cluster
uint64 time_tick = 4; // the time tick of replicate msg from source cluster
string vchannel = 5; // the vchannel of replicate msg from source cluster
}
// ResourceDomain is the domain of resource hold.
enum ResourceDomain {
ResourceDomainUnknown = 0; // should never be used.
ResourceDomainImportJobID = 1 [deprecated = true]; // the domain of import job id.
ResourceDomainCollectionName = 2; // the domain of collection name.
ResourceDomainDBName = 3; // the domain of db name.
ResourceDomainPrivilege = 4; // the domain of privilege.
ResourceDomainCluster = 127; // the domain of full cluster.
}
// ResourceKey is the key for resource hold.
// It's used to implement the resource acquirition mechanism for broadcast message.
// The key should be a unique identifier of the resource for different domain.
message ResourceKey {
ResourceDomain domain = 1;
string key = 2;
bool shared = 3; // whether the resource is shared,
// if true, the resource is shared by multiple broadcast message,
// otherwise, the resource is exclusive to the broadcast message.
}
// CipherHeader is the header of a message that is encrypted.
message CipherHeader {
int64 ez_id = 1; // related to the encryption zone id
int64 collection_id = 2; // related to the collection id
bytes safe_key = 3; // the safe key
int64 payload_bytes = 4; // the size of the payload before encryption
}
// TruncateCollectionMessageHeader is the header of truncate collection message.
message TruncateCollectionMessageHeader {
int64 db_id = 1;
int64 collection_id = 2;
repeated int64 segment_ids = 3;
}
// TruncateCollectionMessageBody is the body of truncate collection message.
message TruncateCollectionMessageBody {}