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