milvus/pkg/proto/messages.proto
Zhen Ye ba289891c0
enhance: add all ddl message into messages (#44407)
issue: #43897

- add ddl messages proto and add some message utilities.
- support shard/exclusive resource-key-lock.
- add all ddl callbacks future into broadcast registry.

---------

Signed-off-by: chyezh <chyezh@outlook.com>
2025-09-18 10:08:00 +08:00

666 lines
23 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;
// 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;
}
// 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.
}
// 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
}