syntax = "proto3"; package backup; import "kvrpcpb.proto"; import "errorpb.proto"; import "metapb.proto"; import "encryptionpb.proto"; import "gogoproto/gogo.proto"; import "rustproto.proto"; option (gogoproto.sizer_all) = true; option (gogoproto.marshaler_all) = true; option (gogoproto.unmarshaler_all) = true; option (rustproto.lite_runtime_all) = true; option java_package = "org.tikv.kvproto"; // The message save the metadata of a backup. message BackupMeta { // ID and version of backuped cluster. uint64 cluster_id = 1; string cluster_version = 2; // Save the version of BR running backup jobs. string br_version = 11; // The backupmeta scheme version. int32 version = 12; // path field is no longer used. reserved 3; reserved "path"; // A set of files that compose a backup. // Note: `files` is deprecated, as it bloats backupmeta. It is kept for // compatibility, so new BR can restore older backups. repeated File files = 4; // An index to files contains data files. MetaFile file_index = 13; // A pair of timestamp specifies a time range of a backup. // For full backup, the start_version equals to the end_version, // it means point in time. // For incremental backup, the time range is specified as // (start_version, end_version]. uint64 start_version = 5; uint64 end_version = 6; // Table metadata describes database and table info. // Note: `schemas` is deprecated, as it bloats backupmeta. It is kept for // compatibility, so new BR can restore older backups. repeated Schema schemas = 7; // An index to files contains Schemas. MetaFile schema_index = 14; // If in raw kv mode, `start_versions`, `end_versions` and `schemas` will be // ignored, and the backup data's range is represented by raw_ranges. bool is_raw_kv = 8; // Note: `raw_ranges` is deprecated, as it bloats backupmeta. It is kept for // compatibility, so new BR can restore older backups. repeated RawRange raw_ranges = 9; // An index to files contains RawRanges. MetaFile raw_range_index = 15; // In incremental backup, DDLs which are completed in // (lastBackupTS, backupTS] will be stored here. // Note: `raw_ranges` is deprecated, as it bloats backupmeta. It is kept for // compatibility, so new BR can restore older backups. bytes ddls = 10; // An index to files contains DDLs. MetaFile ddl_indexes = 16; // the backup result into `backupmeta` file string backup_result = 17; // API version implies the encode of the key and value. kvrpcpb.APIVersion api_version = 18; // the placement policy info in backup cluster. we assume the policy won't be too much for one cluster. repeated PlacementPolicy policies = 19; // new_collations_enabled specifies the config `new_collations_enabled_on_first_bootstrap` in tidb. string new_collations_enabled = 20; // If in txn kv mode, `schemas` will be ignored, the backup data's range is as same as normal backup. bool is_txn_kv = 21; // maintain the id mapping from upstream cluster to downstream cluster. repeated PitrDBMap db_maps = 22; BackupMode mode = 23; // record the backup range and the correspond SST files when using file-copy backup. repeated BackupRange ranges = 24; // record the size of the backup data files and meta files uint64 backup_size = 25; } message BackupRange { bytes start_key = 1; bytes end_key = 2; repeated File files = 3; } message File { string name = 1; bytes sha256 = 2; bytes start_key = 3; bytes end_key = 4; uint64 start_version = 5; uint64 end_version = 6; uint64 crc64xor = 7; uint64 total_kvs = 8; uint64 total_bytes = 9; string cf = 10; uint64 size = 11; // cipher_iv is used for AES cipher bytes cipher_iv = 12; } // MetaFile describes a multi-level index of data used in backup. message MetaFile { // A set of files that contains a MetaFile. // It is used as a multi-level index. repeated File meta_files = 1; // A set of files that contains user data. repeated File data_files = 2; // A set of files that contains Schemas. repeated Schema schemas = 3; // A set of files that contains RawRanges. repeated RawRange raw_ranges = 4; // A set of files that contains BackupRanges. repeated BackupRange backup_ranges = 6; // A set of files that contains DDLs. repeated bytes ddls = 5; } message PlacementPolicy { bytes info = 1; } message Schema { bytes db = 1; bytes table = 2; uint64 crc64xor = 3; uint64 total_kvs = 4; uint64 total_bytes = 5; uint32 tiflash_replicas = 6; // stats represents the dump stats for a analyzed table, which generate by DumpStatsToJSON // https://github.com/pingcap/tidb/blob/e136429d8dc5d70f43cd3f94179b0b9f47595097/statistics/handle/dump.go#L116 bytes stats = 7; } message IDMap { int64 upstream_id = 1; int64 downstream_id = 2; } message PitrTableMap{ string name = 1; IDMap id_map = 2; repeated IDMap partitions = 3; } message PitrDBMap{ string name = 1; IDMap id_map = 2; repeated PitrTableMap tables = 3; } message RawRange { bytes start_key = 1; bytes end_key = 2; string cf = 3; } message ClusterIDError { uint64 current = 1; uint64 request = 2; } message Error { string msg = 1; oneof detail { ClusterIDError cluster_id_error = 3; kvrpcpb.KeyError kv_error = 4; errorpb.Error region_error = 5; } } // sst files or log files compression algorithm // for log files, unknown means not use compression algorithm enum CompressionType { UNKNOWN = 0; LZ4 = 1; SNAPPY = 2; ZSTD = 3; } // BackupMpde represents the mode of this whole backup request to the cluster. // and we need to store it in `backupmeta`. enum BackupMode { // scan and generate precise SST file of a backup range. // then we don't need to do anything in future restore. // Note: this mode will cost the CPU resource of TiKV. SCAN = 0; // check and upload the coarse overlap SST files of a backup range. // then we need to use a merge iterator to filter unexpected kv in future restore. // Note: this mode will save the CPU resource of TiKV. FILE = 1; } message CipherInfo { encryptionpb.EncryptionMethod cipher_type = 1; bytes cipher_key = 2; } message BackupRequest { uint64 cluster_id = 1; bytes start_key = 2; bytes end_key = 3; uint64 start_version = 4; uint64 end_version = 5; // path field is deprecated, use storage_backend instead reserved 6; reserved "path"; // The I/O rate limit for backup request. uint64 rate_limit = 7; // The concurrency for executing the backup request in every tikv node. uint32 concurrency = 8; StorageBackend storage_backend = 9; // If raw kv mode is enabled, `start_version` and `end_version` will be ignored, and `cf` // specifies which cf to backup. bool is_raw_kv = 10; string cf = 11; // algorithm used for compress sst files CompressionType compression_type = 12; // sst compression level, some algorithms support negative compression levels int32 compression_level = 13; // The cipher_info is Used to encrypt sst CipherInfo cipher_info = 14; // dst_api_version indicates the key-value encoding version used by the // generated SST file. Accepted values: // 1. "v1": the generated SST files are encoded with api-v1, can be restored // to TiKV clusters whose api version is set to v1. // 2. "v2": the generated SST files are encoded with api-v2, can be restored // to TiKV clusters whose api version is set to v2. kvrpcpb.APIVersion dst_api_version = 15; // with checkpoint, some subintervals of the range have been backed up and recorded. // only the remaining sub ranges of the range need to be backed up this time. repeated kvrpcpb.KeyRange sub_ranges = 16; // replica_read indicates whether to use replica read for backup. // If it is false, the backup will only read data from leader. // If it is true, the backup will read data from both leader and follower. bool replica_read = 17; BackupMode mode = 18; // unique_id represents the handle of this backup. after we implement file-copy backup. // we need generate some internal states during the whole backup precedure. // this unique id is help to find the state effictively. string unique_id = 19; kvrpcpb.Context context = 20; } message StreamBackupTaskInfo { // The storage for backup, parsed by BR. StorageBackend storage = 1; // The time range for backing up. uint64 start_ts = 2; uint64 end_ts = 3; // Misc meta datas. // The name of the task, also the ID of the task. string name = 4; // The table filter of the task. // Only for displaying the task info. repeated string table_filter = 5; // compression type CompressionType compression_type = 6; // The last timestamp of the task has been updated. // This is a simple solution for unfrequent config changing: // When we watched a config change(via polling or etcd watching), // We perform a incremental scan between [last_update_ts, now), // for filling the diff data during conf changing. // The current implementation scan [0, now) for every task ranges newly added, // So this field is reserved for future usage. reserved "last_update_ts"; // We use '/tidb/br-stream/ranges// -> ' // for storing all the target ranges of the task, the "ranges" field will not be saved in the taskinfo. reserved "ranges"; // The status field allow us mark the task as 'paused'. // We use '/tidb/br-stream/pause/ -> ()' for pausing a task for now. // If we need more complexed status transforming, maybe add the status field back would be fine. reserved "status"; } message StorageBackend { oneof backend { Noop noop = 1; Local local = 2; S3 s3 = 3; GCS gcs = 4; CloudDynamic cloud_dynamic = 5; HDFS hdfs = 6; AzureBlobStorage azure_blob_storage = 7; } } // Noop storage backend saves files into void. message Noop {} // Local storage backend saves files into local disk message Local { string path = 1; } // S3 storage backend saves files into S3 compatible storages // For non-aws providers, endpoint must be provided message S3 { string endpoint = 1; string region = 2; string bucket = 3; string prefix = 4; string storage_class = 5; // server side encryption string sse = 6; string acl = 7; string access_key = 8; string secret_access_key = 9; bool force_path_style = 10; string sse_kms_key_id = 11; string role_arn = 12; string external_id = 13; bool object_lock_enabled = 14; string session_token = 15; string provider = 16; string profile = 17; } // GCS storage backend saves files into google cloud storage. message GCS { string endpoint = 1; string bucket = 2; string prefix = 3; string storage_class = 4; // If not empty, applies a predefined set of access controls. // See https://cloud.google.com/storage/docs/access-control/lists#predefined-acl // for valid values. string predefined_acl = 5; // Service Account Credentials JSON blob // You can get one from https://console.cloud.google.com/apis/credentials, and // copy the content, set it as string here. string credentials_blob = 6; } // The encryption algorithm must be AES256. message AzureCustomerKey { // A Base64-encoded AES-256 encryption key value. string encryption_key = 1; // The Base64-encoded SHA256 of the encryption key. string encryption_key_sha256 = 2; } // AzureBlobStorage storage backend saves files into azure blob storage. message AzureBlobStorage { string endpoint = 1; // Alias: container string bucket = 2; // Notice: prefix starts without `/`, otherwise the first directory's name is empty. string prefix = 3; // Alias: access_tier. // See https://docs.microsoft.com/en-us/azure/storage/blobs/access-tiers-overview string storage_class = 4; // if empty, try to read account_name from the node's environment variable $AZURE_STORAGE_ACCOUNT. string account_name = 5; // There are 3 kinds of credentials, and the priority order is // `SAS > Shared key > Azure AD (env) > Shared key (env)`. // // 1. Use shared key to access the azure blob // If the node's environment variables($AZURE_CLIENT_ID, $AZURE_TENANT_ID, $AZURE_CLIENT_SECRET) exist, // prefer to use token to access the azure blob. // // See https://learn.microsoft.com/en-us/rest/api/storageservices/authorize-with-shared-key // // Otherwise, if empty, try to read shared key from the node's environment variable $AZURE_STORAGE_KEY. string shared_key = 6; // 2. Use Azure AD (Azure Active Directory) to access the azure blob // // See https://learn.microsoft.com/en-us/rest/api/storageservices/authorize-with-azure-active-directory // // The Azure AD would generate the token, which tasks some time. // So it is not recommanded to generate the token in each request. // // AzureActiveDirectory azure_ad = #; // // 3. Use SAS (shared access signature) // // See https://learn.microsoft.com/en-us/rest/api/storageservices/delegate-access-with-shared-access-signature // string access_sig = 8; // Server Side Encryption, 2 types in total: // // 1. Specify an encryption scope for uploaded blobs. // // See https://learn.microsoft.com/en-us/azure/storage/blobs/encryption-scope-manage?tabs=powershell#upload-a-blob-with-an-encryption-scope // string encryption_scope = 9; // 2. Provide an encryption key on a request to blob storage. // // See https://learn.microsoft.com/en-us/azure/storage/blobs/encryption-customer-provided-keys // AzureCustomerKey encryption_key = 10; } message Bucket { string endpoint = 1; string region = 3; string bucket = 4; string prefix = 5; string storage_class = 6; } // CloudDynamic allows testing new cloud providers and new fields without changing protobuf definitions message CloudDynamic { Bucket bucket = 1; string provider_name = 2; // s3, gcs and azureBlobStorage are supported map attrs = 3; } // HDFS storage backend saves file into HDFS compatible storages message HDFS { // a URL: hdfs:///some/path or hdfs://host:port/some/path string remote = 1; } message BackupResponse { Error error = 1; bytes start_key = 2; bytes end_key = 3; repeated File files = 4; // API version implies the encode of the key and value. kvrpcpb.APIVersion api_version = 5; } message CleanupRequest { // unique_id represents the unique handle of the whole backup predecure. // it generated in prepare request and corrosponed to one specific backup. string unique_id = 1; } message CleanupResponse { Error error = 1; bool success = 2; } message PrepareRequest { // whether save state to the storage. bool save_to_storage = 1; } message PrepareResponse { Error error = 1; // unique_id represents the unique handle of the whole backup predecure. // if unique_id = 0 means prepare failed. // if unique_id > 0 means prepare success and all states saved with this unique info. string unique_id = 2; uint64 collect_file_count = 3; uint64 collect_file_size = 4; } message CheckAdminRequest { } message CheckAdminResponse { Error error = 1; metapb.Region region = 2; bool has_pending_admin = 3; } service Backup { rpc backup(BackupRequest) returns (stream BackupResponse) {} // CheckPendingAdminOp used for snapshot backup. before we start snapshot for a TiKV. // we need stop all schedule first and make sure all in-flight schedule has finished. // this rpc check all pending conf change for leader. rpc CheckPendingAdminOp(CheckAdminRequest) returns (stream CheckAdminResponse) {} // prepare is used for file-copy backup. before we start the backup for a TiKV. // we need invoke this function to generate the SST files map. or we get nothing to backup. rpc prepare(PrepareRequest) returns (PrepareResponse) {} // cleanup used for file-copy backup. after we finish the backup for a TiKV. // we need clean some internel state. e.g. checkpoint, SST File maps rpc cleanup(CleanupRequest) returns (CleanupResponse) {} } // ExternalStorage is a service for using a cloud backend from StorageBackend to store files. // This can be used to backup and restore SST files. service ExternalStorage { // Restore to a file rpc restore(ExternalStorageRestoreRequest) returns (ExternalStorageRestoreResponse) {} // Save a file to storage rpc save(ExternalStorageSaveRequest) returns (ExternalStorageSaveResponse) {} } message ExternalStorageRestoreRequest { StorageBackend storage_backend = 1; string object_name = 2; string restore_name = 3; uint64 content_length = 4; } message ExternalStorageRestoreResponse { } message ExternalStorageSaveRequest { StorageBackend storage_backend = 1; string object_name = 2; uint64 content_length = 3; } message ExternalStorageSaveResponse { } enum MetaVersion { V1 = 0; V2 = 1; } message Metadata { // deprecated after(in) v6.3.0 TiKV cluster repeated DataFileInfo files = 1; repeated DataFileGroup file_groups = 6; int64 store_id = 2; uint64 resolved_ts = 3; uint64 max_ts = 4; uint64 min_ts = 5; MetaVersion meta_version = 7; } enum FileType { Delete = 0; Put = 1; } // DataFileGroup is the merged file info in log-backup message DataFileGroup { // Path of the file. string path = 1; // Partitions of the file. repeated DataFileInfo data_files_info = 2; /// Below are extra information of the file, for better filtering files. // The min ts of the keys in the file. uint64 min_ts = 3; // The max ts of the keys in the file. uint64 max_ts = 4; // The resolved ts of the region when saving the file. uint64 min_resolved_ts = 5; // The file length after compressed. uint64 length = 6; } message DataFileInfo { // SHA256 of the file. bytes sha256 = 1; // Path of the file. string path = 2; int64 number_of_entries = 3; /// Below are extra information of the file, for better filtering files. // The min ts of the keys in the file. uint64 min_ts = 4; // The max ts of the keys in the file. uint64 max_ts = 5; // The resolved ts of the region when saving the file. uint64 resolved_ts = 6; // The region of the file. int64 region_id = 7; // The key range of the file. // Encoded and starts with 'z'(internal key). bytes start_key = 8; bytes end_key = 9; // The column family of the file. string cf = 10; // The operation type of the file. FileType type = 11; // Whether the data file contains meta keys(m prefixed keys) only. bool is_meta = 12; // The table ID of the file contains, when `is_meta` is true, would be ignored. int64 table_id = 13; // The file length. uint64 length = 14; // The minimal begin ts in default cf if this file is write cf. uint64 min_begin_ts_in_default_cf = 15; // Offset of the partition. compatible with V1 and V2. uint64 range_offset = 16; // The range length of the merged file, if it exists. uint64 range_length = 17; // The compression type for the file. CompressionType compression_type = 18; // It may support encrypting at future. reserved "iv"; } message StreamBackupError { // the unix epoch time (in millisecs) of the time the error reported. uint64 happen_at = 1; // the unified error code of the error. string error_code = 2; // the user-friendly error message. string error_message = 3; // the store id of who issues the error. uint64 store_id = 4; }