mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
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>
666 lines
23 KiB
Protocol Buffer
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
|
|
}
|