client-rust/proto/brpb.proto

646 lines
20 KiB
Protocol Buffer

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/<task-name>/<start-key> -> <end-key>'
// 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/<task-name> -> ()' 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<string, string> 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;
}