diff --git a/Cargo.toml b/Cargo.toml index 0aab2b0..a61b799 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -41,6 +41,8 @@ regex = "1" semver = "1.0" serde = "1.0" serde_derive = "1.0" +serde_json = "1" +take_mut = "0.2.2" thiserror = "1" tokio = { version = "1", features = ["sync", "rt-multi-thread", "macros"] } tonic = { version = "0.10", features = ["tls"] } @@ -51,9 +53,8 @@ env_logger = "0.10" fail = { version = "0.4", features = ["failpoints"] } proptest = "1" proptest-derive = "0.3" -reqwest = { version = "0.11", default-features = false, features = [ - "native-tls-vendored", -] } +reqwest = { version = "0.11", features = ["json", "native-tls-vendored"] } +rstest = "0.18.2" serde_json = "1" serial_test = "0.5.0" simple_logger = "1" diff --git a/config/tikv.toml b/config/tikv.toml index 5296525..c0f97d6 100644 --- a/config/tikv.toml +++ b/config/tikv.toml @@ -15,3 +15,7 @@ max-open-files = 10000 [raftdb] max-open-files = 10000 + +[storage] +api-version = 2 +enable-ttl = true diff --git a/examples/pessimistic.rs b/examples/pessimistic.rs index 95e5e6f..824710e 100644 --- a/examples/pessimistic.rs +++ b/examples/pessimistic.rs @@ -24,7 +24,9 @@ async fn main() { Config::default().with_security(ca, cert, key) } else { Config::default() - }; + } + // This example uses the default keyspace, so api-v2 must be enabled on the server. + .with_default_keyspace(); // init let client = Client::new_with_config(args.pd, config) diff --git a/examples/raw.rs b/examples/raw.rs index e5b24e1..cb3bdcd 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -31,7 +31,9 @@ async fn main() -> Result<()> { Config::default().with_security(ca, cert, key) } else { Config::default() - }; + } + // This example uses the default keyspace, so api-v2 must be enabled on the server. + .with_default_keyspace(); // When we first create a client we receive a `Connect` structure which must be resolved before // the client is actually connected and usable. @@ -136,6 +138,8 @@ async fn main() -> Result<()> { ); println!("Scanning batch scan from {batch_scan_keys:?} gives: {vals:?}"); - // Cleanly exit. + // Delete all keys in the whole range. + client.delete_range("".to_owned().."".to_owned()).await?; + Ok(()) } diff --git a/examples/transaction.rs b/examples/transaction.rs index 119fc88..cd3528e 100644 --- a/examples/transaction.rs +++ b/examples/transaction.rs @@ -87,7 +87,9 @@ async fn main() { Config::default().with_security(ca, cert, key) } else { Config::default() - }; + } + // This example uses the default keyspace, so api-v2 must be enabled on the server. + .with_default_keyspace(); let txn = Client::new_with_config(args.pd, config) .await diff --git a/proto-build/Cargo.toml b/proto-build/Cargo.toml index 45826e5..36e0302 100644 --- a/proto-build/Cargo.toml +++ b/proto-build/Cargo.toml @@ -12,4 +12,4 @@ edition = "2021" [dependencies] glob = "0.3" -tonic-build = { version = "0.10", features = ["cleanup-markdown"] } \ No newline at end of file +tonic-build = { version = "0.10", features = ["cleanup-markdown"] } diff --git a/proto/autoid.proto b/proto/autoid.proto new file mode 100644 index 0000000..252ae58 --- /dev/null +++ b/proto/autoid.proto @@ -0,0 +1,47 @@ +syntax = "proto3"; +package autoid; + +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"; + +message AutoIDRequest { + int64 dbID = 1; + int64 tblID = 2; + bool isUnsigned = 3; + uint64 n = 4; + int64 increment = 5; + int64 offset = 6; + uint32 keyspaceID = 7; +} + +message AutoIDResponse { + int64 min = 1; + int64 max = 2; + + bytes errmsg = 3; +} + +message RebaseRequest { + int64 dbID = 1; + int64 tblID = 2; + bool isUnsigned = 3; + int64 base = 4; + bool force = 5; +} + +message RebaseResponse { + bytes errmsg = 1; +} + +service AutoIDAlloc { + rpc AllocAutoID(AutoIDRequest) returns (AutoIDResponse) {} + rpc Rebase(RebaseRequest) returns (RebaseResponse) {} +} + diff --git a/proto/brpb.proto b/proto/brpb.proto index c25da40..13acb75 100644 --- a/proto/brpb.proto +++ b/proto/brpb.proto @@ -3,6 +3,7 @@ package backup; import "kvrpcpb.proto"; import "errorpb.proto"; +import "metapb.proto"; import "encryptionpb.proto"; import "gogoproto/gogo.proto"; import "rustproto.proto"; @@ -54,6 +55,7 @@ message BackupMeta { // 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; @@ -69,6 +71,33 @@ message BackupMeta { // 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 { @@ -102,10 +131,16 @@ message MetaFile { 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; @@ -121,6 +156,23 @@ message Schema { 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; @@ -141,7 +193,8 @@ message Error { } } -// sst files compression algorithm +// sst files or log files compression algorithm +// for log files, unknown means not use compression algorithm enum CompressionType { UNKNOWN = 0; LZ4 = 1; @@ -149,6 +202,19 @@ enum CompressionType { 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; @@ -182,6 +248,66 @@ message BackupRequest { 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 { @@ -219,6 +345,12 @@ message S3 { 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. @@ -237,6 +369,14 @@ message GCS { 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; @@ -250,14 +390,46 @@ message AzureBlobStorage { // if empty, try to read account_name from the node's environment variable $AZURE_STORAGE_ACCOUNT. string account_name = 5; - // Use shared key to access the azure blob + + // 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://docs.microsoft.com/en-us/azure/storage/common/identity-library-acquire-token?toc=/azure/storage/blobs/toc.json + // 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 { @@ -287,10 +459,60 @@ message BackupResponse { 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. @@ -321,3 +543,103 @@ message ExternalStorageSaveRequest { 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; +} diff --git a/proto/cdcpb.proto b/proto/cdcpb.proto index 5682eaf..6d27130 100644 --- a/proto/cdcpb.proto +++ b/proto/cdcpb.proto @@ -45,6 +45,7 @@ message Error { DuplicateRequest duplicate_request = 4; Compatibility compatibility = 5; ClusterIDMismatch cluster_id_mismatch = 6; + errorpb.ServerIsBusy server_is_busy = 7; } message TxnInfo { @@ -83,6 +84,12 @@ message Event { bytes key = 5; bytes value = 6; bytes old_value = 7; + // expire_ts_unix_secs is used for RawKV (see `ChangeDataRequest.KvApi`), + // and represents the expiration time of this row. + // Absolute time, seconds since Unix epoch. + uint64 expire_ts_unix_secs = 8; + // The source of this row. + uint64 txn_source = 9; } message Entries { @@ -112,6 +119,7 @@ message Event { } } +// NOTE: events and resolved_ts won't appear simultaneously in one ChangeDataEvent. message ChangeDataEvent { repeated Event events = 1; ResolvedTs resolved_ts = 2; @@ -121,11 +129,14 @@ message ChangeDataEvent { message ResolvedTs { repeated uint64 regions = 1; uint64 ts = 2; + uint64 request_id = 3; } message ChangeDataRequest { message Register {} + message Deregister {} + message NotifyTxnStatus { repeated TxnStatus txn_status = 1; } @@ -138,6 +149,8 @@ message ChangeDataRequest { bytes start_key = 5; bytes end_key = 6; // Used for CDC to identify events corresponding to different requests. + // Generally in one call stream, a region can be subscribed multiple + // times with different `request_id`s. uint64 request_id = 7; kvrpcpb.ExtraOp extra_op = 8; @@ -149,9 +162,30 @@ message ChangeDataRequest { // Notify the region that some of the running transactions on the region has a pushed // min_commit_ts so that the resolved_ts can be advanced. NotifyTxnStatus notify_txn_status = 10; + + Deregister deregister = 13; } + + // KvAPI specifies to capture data written by different KV API. + // See more details in https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md. + enum KvAPI { + TiDB = 0; + RawKV = 1; + TxnKV = 2; + } + + KvAPI kv_api = 11; + + // Whether to filter out the value write by cdc itself. + bool filter_loop = 12; } service ChangeData { rpc EventFeed(stream ChangeDataRequest) returns(stream ChangeDataEvent); + + // EventFeedV2 is like EventFeed, with some new changes: + // * clients send requested features in HTTP/2 headers; + // * if servers meets unsupported feature request, + // it can fail the stream with an UNIMPLEMENTED error. + rpc EventFeedV2(stream ChangeDataRequest) returns (stream ChangeDataEvent); } diff --git a/proto/coprocessor.proto b/proto/coprocessor.proto index 4078f23..d0f83a0 100644 --- a/proto/coprocessor.proto +++ b/proto/coprocessor.proto @@ -6,7 +6,6 @@ import "kvrpcpb.proto"; import "gogoproto/gogo.proto"; import "rustproto.proto"; import "metapb.proto"; -import "span.proto"; option (gogoproto.marshaler_all) = true; option (gogoproto.sizer_all) = true; @@ -38,6 +37,10 @@ message Request { bool is_trace_enabled = 9; // paging_size is 0 when it's disabled, otherwise, it should be a positive number. uint64 paging_size = 10; + // tasks stores the batched coprocessor tasks sent to the same tikv store. + repeated StoreBatchTask tasks = 11; + uint64 connection_id = 12; // This is the session id between a client and tidb + string connection_alias = 13; // This is the session alias between a client and tidb } message Response { @@ -57,7 +60,15 @@ message Response { bool is_cache_hit = 7; uint64 cache_last_version = 8; bool can_be_cached = 9; - repeated span.SpanSet spans = 10; + + reserved 10; + + // Contains the latest buckets version of the region. + // Clients should query PD to update buckets in cache if its is stale. + uint64 latest_buckets_version = 12; + + // StoreBatchTaskResponse is the collection of batch task responses. + repeated StoreBatchTaskResponse batch_responses = 13; } message RegionInfo { @@ -66,6 +77,11 @@ message RegionInfo { repeated KeyRange ranges = 3; } +message TableRegions { + int64 physical_table_id = 1; + repeated RegionInfo regions = 2; +} + message BatchRequest { kvrpcpb.Context context = 1; int64 tp = 2; @@ -74,6 +90,11 @@ message BatchRequest { uint64 start_ts = 5; // Any schema-ful storage to validate schema correctness if necessary. int64 schema_ver = 6; + // Used for partition table scan + repeated TableRegions table_regions = 7; + string log_id = 8; + uint64 connection_id = 9; // This is the session id between a client and tidb + string connection_alias = 10; // This is the session alias between a client and tidb } message BatchResponse { @@ -82,3 +103,20 @@ message BatchResponse { kvrpcpb.ExecDetails exec_details = 3; repeated metapb.Region retry_regions = 4; } + +message StoreBatchTask { + uint64 region_id = 1; + metapb.RegionEpoch region_epoch = 2; + metapb.Peer peer = 3; + repeated KeyRange ranges = 4; + uint64 task_id = 5; +} + +message StoreBatchTaskResponse { + bytes data = 1 [(gogoproto.customtype) = "github.com/pingcap/kvproto/pkg/sharedbytes.SharedBytes", (gogoproto.nullable) = false]; + errorpb.Error region_error = 2; + kvrpcpb.LockInfo locked = 3; + string other_error = 4; + uint64 task_id = 5; + kvrpcpb.ExecDetailsV2 exec_details_v2 = 6; +} diff --git a/proto/debugpb.proto b/proto/debugpb.proto index 400da51..dc612d1 100644 --- a/proto/debugpb.proto +++ b/proto/debugpb.proto @@ -73,6 +73,18 @@ service Debug { // Get all region IDs in the store rpc GetAllRegionsInStore(GetAllRegionsInStoreRequest) returns (GetAllRegionsInStoreResponse) {} + + // Make this TiKV node return to the status on this node to certain ts. + rpc ResetToVersion(ResetToVersionRequest) returns (ResetToVersionResponse) {} + + // Get range properties + rpc GetRangeProperties(GetRangePropertiesRequest) returns (GetRangePropertiesResponse) {} + + // Flashback given key range to a specified version. + rpc FlashbackToVersion(FlashbackToVersionRequest) returns (FlashbackToVersionResponse) {} + + // GetRegionReadProgress returns the some useful info in RegionReadProgress + rpc GetRegionReadProgress(GetRegionReadProgressRequest) returns (GetRegionReadProgressResponse) {} } enum DB { @@ -259,3 +271,71 @@ message GetAllRegionsInStoreRequest { message GetAllRegionsInStoreResponse { repeated uint64 regions = 1; } + +message ResetToVersionRequest { + uint64 ts = 1; +} + +message ResetToVersionResponse { +} + +message GetRangePropertiesRequest { + bytes start_key = 1; + bytes end_key = 2; +} + +message GetRangePropertiesResponse { + message RangeProperty { + string key = 1; + string value = 2; + } + + repeated RangeProperty properties = 1; +} + +message FlashbackToVersionRequest { + kvrpcpb.Context context = 1; + uint64 version = 2; + uint64 region_id = 3; + bytes start_key = 4; + bytes end_key = 5; + uint64 start_ts = 6; + uint64 commit_ts = 7; +} + +message FlashbackToVersionResponse { + string error = 1; +} + +message GetRegionReadProgressRequest { + uint64 region_id = 1; + bool log_locks = 2; // when set to true, print a log of the locks with min start_ts in the resolver. + uint64 min_start_ts = 3; // only print locks whose start_ts >= min_start_ts. Can be used to find certain transaction. +} + +message GetRegionReadProgressResponse { + // below are from region_read_progress module + uint64 safe_ts = 1; + uint64 applied_index = 2; + uint64 pending_front_applied_index = 3; + uint64 pending_front_ts = 4; + uint64 pending_back_applied_index = 5; + uint64 pending_back_ts = 6; + bool region_read_progress_paused = 7; + uint64 duration_to_last_update_safe_ts_ms = 8; + uint64 duration_to_last_consume_leader_ms = 9; + bool region_read_progress_exist = 10; + uint64 read_state_ts = 18; + uint64 read_state_apply_index = 19; + bool discard = 20; + + // below are from resolved-ts module + uint64 resolved_ts = 11; + uint64 resolver_tracked_index = 12; + bool resolver_exist = 13; + bool resolver_stopped = 14; + uint64 num_locks = 16; + uint64 num_transactions = 17; + + string error = 15; +} \ No newline at end of file diff --git a/proto/disaggregated.proto b/proto/disaggregated.proto new file mode 100644 index 0000000..adaa8f6 --- /dev/null +++ b/proto/disaggregated.proto @@ -0,0 +1,169 @@ +syntax = "proto3"; +package disaggregated; + +import "coprocessor.proto"; +import "kvrpcpb.proto"; + +option java_package = "org.tikv.kvproto"; + +/// For S3 lock service /// + +message S3LockResult { + oneof error { + Success success = 1; + NotOwner not_owner = 2; + Conflict conflict = 3; + } +} +message Success {} +// Error caused by S3GC owner changed +// client should retry +message NotOwner{ +} +// Error caused by concurrency conflict, +// request cancel +message Conflict { + string reason = 1; +} + +message TryAddLockRequest { + // The data file key to add lock + bytes data_file_key = 1; + // The lock store id + uint64 lock_store_id = 3; + // The upload sequence number of lock store + uint64 lock_seq = 4; +} + +message TryAddLockResponse { + S3LockResult result = 1; +} + +message TryMarkDeleteRequest { + // The data file key to be marked as deleted + bytes data_file_key = 1; +} + +message TryMarkDeleteResponse { + S3LockResult result = 1; +} + +/// For disagg compute node init its disagg configuration /// + +message GetDisaggConfigRequest { +} +message DisaggS3Config { + string bucket = 1; + string root = 2; + string endpoint = 3; +} +message GetDisaggConfigResponse { + DisaggS3Config s3_config = 1; +} + +/// For compute task dispatch and data exchange /// + +message DisaggTaskMeta { + uint64 start_ts = 1; // start ts of a query + // gather_id + query_ts + server_id + local_query_id to represent a global unique query. + int64 gather_id = 9; // used to distinguish different gathers in the mpp query + uint64 query_ts = 2; // timestamp when start to execute query, used for TiFlash miniTSO schedule. + uint64 server_id = 3; // TiDB server id + uint64 local_query_id = 4; // unique local query_id if tidb don't restart. + int64 task_id = 5; // if task id is -1 , it indicates a tidb task. + string executor_id = 6; // the exectuor id + uint32 keyspace_id = 7; // keyspace id of the request + kvrpcpb.APIVersion api_version = 8; // API version of the request + uint64 connection_id = 10; // This is the session id between a client and tidb + string connection_alias = 11; // This is the session alias between a client and tidb +} + +message DisaggReadError { + int32 code = 1; + string msg = 2; +} + +message EstablishDisaggTaskError { + oneof errors { + ErrorRegion error_region = 1; + ErrorLocked error_locked = 2; + + ErrorOther error_other = 99; + } +} + +message ErrorRegion { + string msg = 1; + // The read node needs to update its region cache about these regions. + repeated uint64 region_ids = 2; +} + +message ErrorLocked { + string msg = 1; + // The read node needs to resolve these locks. + repeated kvrpcpb.LockInfo locked = 2; +} + +message ErrorOther { + int32 code = 1; + string msg = 2; +} + +message EstablishDisaggTaskRequest { + DisaggTaskMeta meta = 1; + string address = 2; // target address of this task. + // The write node needs to ensure that subsequent + // FetchDisaggPagesRequest can be processed within timeout_s. + // unit: seconds + int64 timeout_s = 3; + // The key ranges, Region meta that read node need to execute TableScan + repeated coprocessor.RegionInfo regions = 4; + int64 schema_ver = 5; + // Used for PartitionTableScan + repeated coprocessor.TableRegions table_regions = 6; + // The encoded TableScan/PartitionTableScan + Selection. + bytes encoded_plan = 7; +} + +message EstablishDisaggTaskResponse { + EstablishDisaggTaskError error = 1; + + // Write node maintains a snapshot with a lease time. + // Read node should read the delta pages + // (ColumnFileInMemory and ColumnFileTiny) + // along with this store_id and snapshot_id. + uint64 store_id = 3; // metapb.Store.id + DisaggTaskMeta snapshot_id = 4; + // Serialized disaggregated tasks (per physical table) + repeated bytes tables = 5; +} + +message CancelDisaggTaskRequest { + DisaggTaskMeta meta = 1; +} + +message CancelDisaggTaskResponse {} + +message FetchDisaggPagesRequest { + // The snapshot id to fetch pages + DisaggTaskMeta snapshot_id = 1; + int64 table_id = 2; + uint64 segment_id = 3; + // It must be a subset of the delta pages ids returned + // in EstablishDisaggTaskResponse.segments + repeated uint64 page_ids = 4; +} + +message PagesPacket { + DisaggReadError error = 1; + + // Serialized column file data + // * ColumnFilePersisted alone with its schema, page data, field offsets + repeated bytes pages = 2; + // * ColumnFileInMemory alone with its serialized block + repeated bytes chunks = 3; + + // Return tipb.SelectResponse.execution_summaries in the + // last packet + repeated bytes summaries = 4; +} diff --git a/proto/encryptionpb.proto b/proto/encryptionpb.proto index ddae704..2328088 100644 --- a/proto/encryptionpb.proto +++ b/proto/encryptionpb.proto @@ -44,6 +44,7 @@ enum EncryptionMethod { AES128_CTR = 2; AES192_CTR = 3; AES256_CTR = 4; + SM4_CTR = 5; } // The key used to encrypt the user data. diff --git a/proto/errorpb.proto b/proto/errorpb.proto index 8fd9c23..11628cd 100644 --- a/proto/errorpb.proto +++ b/proto/errorpb.proto @@ -21,6 +21,20 @@ message NotLeader { metapb.Peer leader = 2; } +// IsWitness is the error variant that tells a request be handle by witness +// which should be forbidden and retry. +message IsWitness { + // The requested region ID + uint64 region_id = 1; +} + +// BucketVersionNotMatch is the error variant that tells the request buckets version is not match. +// client should update the buckets version and retry. +message BucketVersionNotMatch{ + uint64 version = 1; + repeated bytes keys = 2; +} + message DiskFull { // The requested store ID repeated uint64 store_id = 1; @@ -77,6 +91,9 @@ message ServerIsBusy { string reason = 1; // The suggested backoff time uint64 backoff_ms = 2; + uint32 estimated_wait_ms = 3; + // Current applied_index at the leader, may be used in replica read. + uint64 applied_index = 4; } // StaleCommand is the error variant that tells the command is stale, that is, @@ -128,6 +145,29 @@ message DataIsNotReady { uint64 safe_ts = 3; } +message RecoveryInProgress { + // The requested region ID + uint64 region_id = 1; +} + +message FlashbackInProgress { + // The requested region ID + uint64 region_id = 1; + uint64 flashback_start_ts = 2; +} + +message FlashbackNotPrepared { + // The requested region ID + uint64 region_id = 1; +} + +// MismatchPeerId is the error variant that tells the request is sent to wrong peer. +// Client receives this error should reload the region info and retry. +message MismatchPeerId { + uint64 request_peer_id = 1; + uint64 store_peer_id = 2; +} + // Error wraps all region errors, indicates an error encountered by a request. message Error { reserved "stale_epoch"; @@ -148,4 +188,21 @@ message Error { DataIsNotReady data_is_not_ready = 13; RegionNotInitialized region_not_initialized = 14; DiskFull disk_full = 15; + // Online recovery is still in performing, reject writes to avoid potential issues + RecoveryInProgress RecoveryInProgress = 16; + // Flashback is still in performing, reject any read or write to avoid potential issues. + // NOTICE: this error is non-retryable, the request should fail ASAP when it meets this error. + FlashbackInProgress FlashbackInProgress = 17; + // If the second phase flashback request is sent to a region that is not prepared for the flashback, + // this error will be returned. + // NOTICE: this error is non-retryable, the client should retry the first phase flashback request when it meets this error. + FlashbackNotPrepared FlashbackNotPrepared = 18; + // IsWitness is the error variant that tells a request be handle by witness + // which should be forbidden and retry. + IsWitness is_witness = 19; + + MismatchPeerId mismatch_peer_id = 20; + + // BucketVersionNotMatch is the error variant that tells the request buckets version is not match. + BucketVersionNotMatch bucket_version_not_match = 21; } diff --git a/proto/gcpb.proto b/proto/gcpb.proto new file mode 100644 index 0000000..053872b --- /dev/null +++ b/proto/gcpb.proto @@ -0,0 +1,117 @@ +syntax = "proto3"; +package gcpb; + +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"; + +service GC { + rpc ListKeySpaces(ListKeySpacesRequest) returns (ListKeySpacesResponse) {} + + rpc GetMinServiceSafePoint(GetMinServiceSafePointRequest) returns (GetMinServiceSafePointResponse) {} + + rpc UpdateGCSafePoint(UpdateGCSafePointRequest) returns (UpdateGCSafePointResponse) {} + + rpc UpdateServiceSafePoint(UpdateServiceSafePointRequest) returns (UpdateServiceSafePointResponse) {} +} + +message RequestHeader { + // cluster_id is the ID of the cluster which be sent to. + uint64 cluster_id = 1; + // sender_id is the ID of the sender server, also member ID or etcd ID. + uint64 sender_id = 2; +} + +message ResponseHeader { + // cluster_id is the ID of the cluster which sent the response. + uint64 cluster_id = 1; + Error error = 2; +} + +enum ErrorType { + OK = 0; + UNKNOWN = 1; + NOT_BOOTSTRAPPED = 2; + // revision supplied does not match the current etcd revision + REVISION_MISMATCH = 3; + // if the proposed safe point is earlier than old safe point or gc safe point + SAFEPOINT_ROLLBACK = 4; +} + +message Error { + ErrorType type = 1; + string message = 2; +} + +message KeySpace { + bytes space_id = 1; + uint64 gc_safe_point = 2; +} + +message ListKeySpacesRequest { + RequestHeader header = 1; + // set with_gc_safe_point to true to also receive gc safe point for each key space + bool with_gc_safe_point = 2; +} + +message ListKeySpacesResponse { + ResponseHeader header = 1; + repeated KeySpace key_spaces = 2; +} + +message GetMinServiceSafePointRequest { + RequestHeader header = 1; + bytes space_id = 2; +} + +message GetMinServiceSafePointResponse { + ResponseHeader header = 1; + uint64 safe_point = 2; + // revision here is to safeguard the validity of the obtained min, + // preventing cases where new services register their safe points after min is obtained by gc worker + int64 revision = 3; +} + +message UpdateGCSafePointRequest { + RequestHeader header = 1; + bytes space_id = 2; + uint64 safe_point = 3; + // here client need to provide the revision obtained from GetMinServiceSafePoint, + // so server can check if it's still valid + int64 revision = 4; +} + +message UpdateGCSafePointResponse { + ResponseHeader header = 1; + // update will be successful if revision is valid and new safepoint > old safe point + // if failed, previously obtained min might be incorrect, should retry from GetMinServiceSafePoint + bool succeeded = 2; + uint64 new_safe_point = 3; +} + +message UpdateServiceSafePointRequest { + RequestHeader header = 1; + bytes space_id = 2; + bytes service_id = 3; + // safe point will be set to expire on (PD Server time + TTL) + // pass in a ttl < 0 to remove target safe point + // pass in MAX_INT64 to set a safe point that never expire + int64 TTL = 4; + uint64 safe_point = 5; +} + +message UpdateServiceSafePointResponse { + ResponseHeader header = 1; + // update will be successful if ttl < 0 (a removal request) + // or if new safe point >= old safe point and new safe point >= gc safe point + bool succeeded = 2; + uint64 gc_safe_point = 3; + uint64 old_safe_point = 4; + uint64 new_safe_point = 5; +} diff --git a/proto/import_sstpb.proto b/proto/import_sstpb.proto index ec1f046..089dd35 100644 --- a/proto/import_sstpb.proto +++ b/proto/import_sstpb.proto @@ -5,7 +5,6 @@ package import_sstpb; import "metapb.proto"; import "errorpb.proto"; import "kvrpcpb.proto"; -import "raft_serverpb.proto"; import "gogoproto/gogo.proto"; import "rustproto.proto"; import "brpb.proto"; @@ -35,6 +34,8 @@ option java_package = "org.tikv.kvproto"; service ImportSST { // Switch to normal/import mode. rpc SwitchMode(SwitchModeRequest) returns (SwitchModeResponse) {} + // Get import mode(normal/import). + rpc GetMode(GetModeRequest) returns (GetModeResponse) {} // Upload an SST file to a server. rpc Upload(stream UploadRequest) returns (UploadResponse) {} // Ingest an uploaded SST file to a region. @@ -56,6 +57,33 @@ service ImportSST { // Collect duplicate data from TiKV. rpc DuplicateDetect(DuplicateDetectRequest) returns (stream DuplicateDetectResponse) {} + + // Apply download & apply increment kv files to TiKV. + rpc Apply(ApplyRequest) returns (ApplyResponse) {} + + // ClearFiles clear applied file after restore succeed. + rpc ClearFiles(ClearRequest) returns (ClearResponse) {} + + // Suspend ingest for data listeners don't support catching import data. + rpc SuspendImportRPC(SuspendImportRPCRequest) returns (SuspendImportRPCResponse) {} + +} + +message SuspendImportRPCRequest { + // whether to suspend new imports. + bool should_suspend_imports = 1; + + // the duration of import service suspension + // when should_deny_imports is false, + // this won't take effect. + uint64 duration_in_secs = 2; + // The identifier for the caller. + string caller = 3; +} + +message SuspendImportRPCResponse { + // The last state before this RPC. + bool already_suspended = 1; } enum SwitchMode { @@ -65,11 +93,19 @@ enum SwitchMode { message SwitchModeRequest { SwitchMode mode = 1; + repeated Range ranges = 2; } message SwitchModeResponse { } +message GetModeRequest { +} + +message GetModeResponse { + SwitchMode mode = 1; +} + message Range { bytes start = 1; bytes end = 2; @@ -132,12 +168,19 @@ message CompactRequest { // Compact all files to the bottommost level if the output level is -1. Range range = 1; int32 output_level = 2; + kvrpcpb.Context context = 3; } message CompactResponse { } message DownloadRequest { + // Map represents the map of . + // We'll generate all SSTMeta into one SST File. + map ssts = 1; + // resolved_ts is used to merge related SST Files. + uint64 resolved_ts = 3; + // The SST meta used to identify the downloaded file. // Must be the same among all nodes in the same Raft group. // Note: the "crc32" and "cf_name" fields are ignored in this request, @@ -166,16 +209,34 @@ message DownloadRequest { RewriteRule rewrite_rule = 13 [(gogoproto.nullable) = false]; backup.StorageBackend storage_backend = 14; + // The identity for the stroage backend. + // When this field presents, the storage would be cached. + // If there is a cached storage, TiKV would use it driectly. + string storage_cache_id = 17; bool is_raw_kv = 15; // cipher_info is used to decrypt sst when download sst backup.CipherInfo cipher_info = 16; + + // The type of the download request. + DownloadRequestType request_type = 18; + kvrpcpb.Context context = 19; +} + +enum DownloadRequestType { + // For the compatibility with old version of TiDBs + Legacy = 0; + // For the TiDBs with newer versions that support keyspace feature. + Keyspace = 1; } // For now it is just used for distinguishing the error of the request with the error // of gRPC, add more concrete types if it is necessary later. message Error { string message = 1; + + // We meet some internal errors of the store. + errorpb.Error store_error = 2; } message DownloadResponse { @@ -195,6 +256,10 @@ message DownloadResponse { uint32 crc32 = 4; // The actual length of the rewritten SST file. uint64 length = 5; + + // This field only return when file-copy backup enabled. + // Because it will merge many SST files in a download request. + repeated SSTMeta ssts = 6; } message SetDownloadSpeedLimitRequest { @@ -225,6 +290,7 @@ message WriteRequest { SSTMeta meta = 1; WriteBatch batch = 2; } + kvrpcpb.Context context = 3; } message WriteResponse { @@ -235,6 +301,13 @@ message WriteResponse { message RawWriteBatch { uint64 ttl = 1; repeated Pair pairs = 2; + + // To be compatible with the key encoding of API V2. + // This field should be generated from the client instead of the server, + // since the message will be send to all the replicas of a region. + // Otherwise, the underlying data generated by the server would be inconsistent which is hard to scale + // for other features like MVCC over RawKV. + uint64 ts = 3; } message RawWriteRequest { @@ -242,6 +315,7 @@ message RawWriteRequest { SSTMeta meta = 1; RawWriteBatch batch = 2; } + kvrpcpb.Context context = 3; } message RawWriteResponse { @@ -280,3 +354,95 @@ message DuplicateDetectResponse { // ] repeated KvPair pairs = 3; } + +message KVMeta { + // The file name of the KV file. + string name = 1; + + // file offset, sometimes only need to get a part of data from the merged file + uint64 range_offset = 11; + + // file length for check. + uint64 length = 2; + + // range length of the merged file, if it exists. + uint64 range_length = 12; + + // tell us which cf should apply. WRITE_CF or DEFAULT_CF e.g. + string cf = 3; + + // is_delete represents whether we should delete the kv in tikv. + // it may not be too much delete file. only rollBack operation will generate delete kv file. + bool is_delete = 4; + + // the key ts space being smaller than start_ts can be filter. + uint64 start_ts = 10; + + // the key ts space large than restore_ts can be filter. + uint64 restore_ts = 5; + + bytes start_key = 6; + + bytes end_key = 7; + + // used for checksum when download kv file. + bytes sha256 = 8; + + // the key ts space less than start_snapshot_ts can be filter. + // Deprecated: this field 'start_snapshot_ts' is replaced by the field 'start_ts'. + uint64 start_snapshot_ts = 9; + + // the compression type for the file. + backup.CompressionType compression_type = 13; +} + + +message ApplyRequest { + // The meta of the KV file. + KVMeta meta = 1; + repeated KVMeta metas = 12; + + // Performs a key prefix rewrite after downloading the file. + // All keys in the files will be rewritten as: + // + // new_key = new_key_prefix + old_key[len(old_key_prefix)..] + // + // When used for TiDB, rewriting the prefix changes the table ID. Please + // note that key-rewrite is applied on the origin keys in encoded + // representation. + // + // You need to ensure that the keys before and after rewriting are in the + // same order, otherwise the RPC request will fail. + RewriteRule rewrite_rule = 2 [(gogoproto.nullable) = false]; + repeated RewriteRule rewrite_rules = 13; + + // The identity for the stroage backend. + // When this field presents, the storage would be cached. + // If there is a cached storage, TiKV would use it driectly. + string storage_cache_id = 5; + backup.StorageBackend storage_backend = 3; + + // context represents region info and it used to build raft commands. + kvrpcpb.Context context = 4; + + // cipher_info is used to decrypt kv file when download file. + backup.CipherInfo cipher_info = 11; +} + +message ApplyResponse { + // The actual key range (after rewrite) of the downloaded file. The range is + // inclusive in both ends. + Range range = 1 [(gogoproto.nullable) = false]; + + Error error = 2; +} + +message ClearRequest { + // clear files in import directory with given prefix. + string prefix = 1; +} + +message ClearResponse { + Error error = 1; +} + diff --git a/proto/include/eraftpb.proto b/proto/include/eraftpb.proto index daf1c62..d9ec4b1 100644 --- a/proto/include/eraftpb.proto +++ b/proto/include/eraftpb.proto @@ -79,7 +79,13 @@ message Message { bool reject = 10; uint64 reject_hint = 11; bytes context = 12; - uint64 priority = 14; + uint64 deprecated_priority = 14; + // If this new field is not set, then use the above old field; otherwise + // use the new field. When broadcasting request vote, both fields are + // set if the priority is larger than 0. This change is not a fully + // compatible change, but it makes minimal impact that only new priority + // is not recognized by the old nodes during rolling update. + int64 priority = 15; } message HardState { diff --git a/proto/keyspacepb.proto b/proto/keyspacepb.proto new file mode 100644 index 0000000..e817e98 --- /dev/null +++ b/proto/keyspacepb.proto @@ -0,0 +1,81 @@ +syntax = "proto3"; +package keyspacepb; + +import "pdpb.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"; + +// Keyspace provides services to manage keyspaces. +service Keyspace { + rpc LoadKeyspace (LoadKeyspaceRequest) returns (LoadKeyspaceResponse) {} + // WatchKeyspaces first return all current keyspaces' metadata as its first response. + // Then, it returns responses containing keyspaces that had their metadata changed. + rpc WatchKeyspaces (WatchKeyspacesRequest) returns (stream WatchKeyspacesResponse) {} + rpc UpdateKeyspaceState(UpdateKeyspaceStateRequest) returns (UpdateKeyspaceStateResponse) {} + rpc GetAllKeyspaces(GetAllKeyspacesRequest) returns (GetAllKeyspacesResponse) {} +} + +message KeyspaceMeta { + uint32 id = 1; + string name = 2; + KeyspaceState state = 3; + int64 created_at = 4; + int64 state_changed_at = 5; + map config = 7; +} + +enum KeyspaceState { + ENABLED = 0; + DISABLED = 1; + ARCHIVED = 2; + TOMBSTONE = 3; +} + +message LoadKeyspaceRequest { + pdpb.RequestHeader header = 1; + string name = 2; +} + +message LoadKeyspaceResponse { + pdpb.ResponseHeader header = 1; + KeyspaceMeta keyspace = 2; +} + +message WatchKeyspacesRequest { + pdpb.RequestHeader header = 1; +} + +message WatchKeyspacesResponse { + pdpb.ResponseHeader header = 1; + repeated KeyspaceMeta keyspaces = 2; +} + +message UpdateKeyspaceStateRequest{ + pdpb.RequestHeader header = 1; + uint32 id = 2; + KeyspaceState state = 3; +} + +message UpdateKeyspaceStateResponse{ + pdpb.ResponseHeader header = 1; + KeyspaceMeta keyspace = 2; +} + +message GetAllKeyspacesRequest{ + pdpb.RequestHeader header = 1; + uint32 start_id = 2; + uint32 limit = 3; +} + +message GetAllKeyspacesResponse{ + pdpb.ResponseHeader header = 1; + repeated KeyspaceMeta keyspaces = 2; +} diff --git a/proto/kvrpcpb.proto b/proto/kvrpcpb.proto index 34d08be..69f2d4a 100644 --- a/proto/kvrpcpb.proto +++ b/proto/kvrpcpb.proto @@ -6,6 +6,8 @@ import "errorpb.proto"; import "gogoproto/gogo.proto"; import "rustproto.proto"; import "deadlock.proto"; +import "tracepb.proto"; +import "resource_manager.proto"; option (gogoproto.marshaler_all) = true; option (gogoproto.sizer_all) = true; @@ -75,6 +77,24 @@ message ScanResponse { // TiKV will write the data in a preliminary state. Data cannot be read until it has been committed. // The client should only commit a transaction once all prewrites succeed. message PrewriteRequest { + // What kind of checks need to be performed for keys in a pessimistic transaction. + enum PessimisticAction { + // The key needn't be locked and no extra write conflict checks are needed. + SKIP_PESSIMISTIC_CHECK = 0; + // The key should have been locked at the time of prewrite. + DO_PESSIMISTIC_CHECK = 1; + // The key doesn't need a pessimistic lock. But we need to do data constraint checks. + DO_CONSTRAINT_CHECK = 2; + } + + // for_update_ts constriants that should be checked when prewriting a pessimistic transaction. + message ForUpdateTSConstraint { + // The index of key in the prewrite request that should be checked. + uint32 index = 1; + // The expected for_update_ts of the pessimistic lock of the key. + uint64 expected_for_update_ts = 2; + } + Context context = 1; // The data to be written to the database. repeated Mutation mutations = 2; @@ -88,7 +108,8 @@ message PrewriteRequest { // TiKV can skip some checks, used for speeding up data migration. bool skip_constraint_check = 6; // For pessimistic transaction, some mutations don't need to be locked, for example, non-unique index key. - repeated bool is_pessimistic_lock = 7; + // Keys with deferred constraint checks are not locked. + repeated PessimisticAction pessimistic_actions = 7; // How many keys this transaction involves in this region. uint64 txn_size = 8; // For pessimistic transactions only; used to check if a conflict lock is already committed. @@ -108,6 +129,9 @@ message PrewriteRequest { uint64 max_commit_ts = 14; // The level of assertion to use on this prewrte request. AssertionLevel assertion_level = 15; + // for_update_ts constriants that should be checked when prewriting a pessimistic transaction. + // See https://github.com/tikv/tikv/issues/14311 + repeated ForUpdateTSConstraint for_update_ts_constraints = 16; } message PrewriteResponse { @@ -121,6 +145,22 @@ message PrewriteResponse { // the commit ts of the transaction. Otherwise, if TiKV failed to commit it with 1PC or the // transaction is not 1PC, the value will be 0. uint64 one_pc_commit_ts = 4; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 5; +} + +// Used to specify the behavior when a pessimistic lock request is woken up after waiting for another +// lock. +enum PessimisticLockWakeUpMode { + // When woken up, returns WriteConflict error to the client and the client should retry if necessary. + // In this mode, results of `return_values` or `check_existence` will be set to `values` and `not_founds` + // fields of the PessimisticLockResponse, which is compatible with old versions. + WakeUpModeNormal = 0; + // When woken up, continue trying to lock the key. This implicitly enables the `allow_lock_with_conflict` + // behavior, which means, allow acquiring the lock even if there is WriteConflict on the key. + // In this mode, `return_values` or `check_existence` fields of PessimisticLockResponse won't be used, and + // all results are carried in the `results` field. + WakeUpModeForceLock = 1; } // Lock a set of keys to prepare to write to them. @@ -157,6 +197,33 @@ message PessimisticLockRequest { // `return_values` is set, it simply makes no difference; otherwise, the `value` field of the // repsonse will be empty while the `not_founds` field still indicates the keys' existence. bool check_existence = 12; + // TiKV lock the record only when it exists + bool lock_only_if_exists = 13; + // Specifies the behavior when the request is woken up after wating for lock of another transaction. + PessimisticLockWakeUpMode wake_up_mode = 14; +} + +enum PessimisticLockKeyResultType { + LockResultNormal = 0; + LockResultLockedWithConflict = 1; + LockResultFailed = 2; +} + +message PessimisticLockKeyResult { + PessimisticLockKeyResultType type = 1; + bytes value = 2; + bool existence = 3; + // We allow a key be locked when there is write conflict (latest commit_ts > for_update_ts). + // In this case, the key is semantically locked by a newer for_update_ts. + // For each requested key, the field is non-zero if the key is locked with write conflict, and it + // equals to the commit_ts of the latest version of the specified key. The for_update_ts field + // of the lock that's actually written to TiKV will also be this value. At the same time, + // `value` and `existence` will be returned regardless to how `return_values` and + // `check_existence` are set. + uint64 locked_with_conflict_ts = 4; + // Hint the client that resolving lock is not needed for this lock. For `PessimisticLock` + // requests only. + bool skip_resolving_lock = 11; } message PessimisticLockResponse { @@ -167,11 +234,17 @@ message PessimisticLockResponse { bytes value = 4 [deprecated = true]; // The values is set if 'return_values' is true in the request and no error. // If 'force' is true, this field is not used. + // Only used when `wake_up_mode` is `WakeUpModeNormal`. repeated bytes values = 5; // Indicates whether the values at the same index is correspond to an existing key. // In legacy TiKV, this field is not used even 'force' is false. In that case, an empty value indicates // two possible situations: (1) the key does not exist. (2) the key exists but the value is empty. + // Only used when `wake_up_mode` is `WakeUpModeNormal`. repeated bool not_founds = 6; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 7; + // Results of the request. Only used when `wake_up_mode` is `WakeUpModeForceLock`. + repeated PessimisticLockKeyResult results = 8; } // Unlock keys locked using `PessimisticLockRequest`. @@ -185,6 +258,8 @@ message PessimisticRollbackRequest { message PessimisticRollbackResponse { errorpb.Error region_error = 1; repeated KeyError errors = 2; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 3; } // Used to update the lock_ttl of a psessimistic and/or large transaction to prevent it from been killed. @@ -203,6 +278,8 @@ message TxnHeartBeatResponse { KeyError error = 2; // The TTL actually set on the requested lock. uint64 lock_ttl = 3; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 4; } // CheckTxnStatusRequest checks the status of a transaction. @@ -232,6 +309,12 @@ message CheckTxnStatusRequest { // lock, the transaction status could not be decided if the primary lock is pessimistic too and // it's still uncertain. bool resolving_pessimistic_lock = 8; + // Whether it's needed to check if the lock on the key (if any) is the primary lock. + // This is for handling some corner cases when a pessimistic transaction changes its primary + // (see https://github.com/pingcap/tidb/issues/42937 for details). This field is necessary + // because the old versions of clients cannot handle some results returned from TiKV correctly. + // For new versions, this field should always be set to true. + bool verify_is_primary = 9; } message CheckTxnStatusResponse { @@ -246,6 +329,8 @@ message CheckTxnStatusResponse { // The action performed by TiKV (and why if the action is to rollback). Action action = 5; LockInfo lock_info = 6; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 7; } // Part of the async commit protocol, checks for locks on all supplied keys. If a lock is missing, @@ -268,6 +353,8 @@ message CheckSecondaryLocksResponse { // If any of the locks have been committed, this is the commit ts used. If no // locks have been committed, it will be zero. uint64 commit_ts = 4; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 5; } // The second phase of writing to TiKV. If there are no errors or conflicts, then this request @@ -289,6 +376,8 @@ message CommitResponse { KeyError error = 2; // If the commit ts is derived from min_commit_ts, this field should be set. uint64 commit_version = 3; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 4; } // Not yet implemented. @@ -353,6 +442,8 @@ message BatchRollbackRequest { message BatchRollbackResponse { errorpb.Error region_error = 1; KeyError error = 2; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 3; } // Scan the database for locks. Used at the start of the GC process to find all @@ -374,6 +465,8 @@ message ScanLockResponse { KeyError error = 2; // Info on all locks found by the scan. repeated LockInfo locks = 3; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 4; } // For all keys locked by the transaction identified by `start_version`, either @@ -392,6 +485,8 @@ message ResolveLockRequest { message ResolveLockResponse { errorpb.Error region_error = 1; KeyError error = 2; + // Execution details about the request processing. + ExecDetailsV2 exec_details_v2 = 3; } // Request TiKV to garbage collect all non-current data older than `safe_point`. @@ -406,7 +501,6 @@ message GCResponse { } // Delete a range of data from TiKV. -// This message should not be used. message DeleteRangeRequest { Context context = 1; bytes start_key = 2; @@ -422,6 +516,45 @@ message DeleteRangeResponse { string error = 2; } +// Preparing the flashback for a region/key range will "lock" the region +// so that there is no any read, write or schedule operation could be proposed before +// the actual flashback operation. +message PrepareFlashbackToVersionRequest { + Context context = 1; + bytes start_key = 2; + bytes end_key = 3; + // The `start_ts` which we will use to write a lock to prevent + // the `resolved_ts` from advancing during the whole process. + uint64 start_ts = 4; + // The TS version which the data will flashback to later. + uint64 version = 5; +} + +message PrepareFlashbackToVersionResponse { + errorpb.Error region_error = 1; + string error = 2; +} + +// Flashback the region to a specific point with the given `version`, please +// make sure the region is "locked" by `PrepareFlashbackToVersionRequest` first, +// otherwise this request will fail. +message FlashbackToVersionRequest { + Context context = 1; + // The TS version which the data should flashback to. + uint64 version = 2; + bytes start_key = 3; + bytes end_key = 4; + // The `start_ts` and `commit_ts` which the newly written MVCC version will use. + // Please make sure the `start_ts` is the same one in `PrepareFlashbackToVersionRequest`. + uint64 start_ts = 5; + uint64 commit_ts = 6; +} + +message FlashbackToVersionResponse { + errorpb.Error region_error = 1; + string error = 2; +} + // Raw commands. message RawGetRequest { @@ -466,11 +599,13 @@ message RawBatchPutRequest { Context context = 1; repeated KvPair pairs = 2; string cf = 3; + uint64 ttl = 4 [deprecated=true]; + bool for_cas = 5; + // The time-to-live for each keys in seconds, and if the length of `ttls` // is exactly one, the ttl will be applied to all keys. Otherwise, the length // mismatch between `ttls` and `pairs` will return an error. - repeated uint64 ttls = 4; - bool for_cas = 5; + repeated uint64 ttls = 6; } message RawBatchPutResponse { @@ -710,57 +845,115 @@ message Context { // Read request should read through locks belonging to these transactions because these // transactions are committed and theirs commit_ts <= read request's start_ts. repeated uint64 committed_locks = 22; + + // The informantion to trace a request sent to TiKV. + tracepb.TraceContext trace_context = 23; + + // The source of the request, will be used as the tag of the metrics reporting. + // This field can be set for any requests that require to report metrics with any extra labels. + string request_source = 24; + + // The source of the current transaction. + uint64 txn_source = 25; + + reserved 26; // Used for resource_group_name, now it's moved into ResourceControlContext. + + // If `busy_threshold_ms` is given, TiKV can reject the request and return a `ServerIsBusy` + // error before processing if the estimated waiting duration exceeds the threshold. + uint32 busy_threshold_ms = 27; + + // Some information used for resource control. + ResourceControlContext resource_control_context = 28; + + // The keyspace that the request is sent to. + // NOTE: This field is only meaningful while the api_version is V2. + uint32 keyspace_id = 32; + + // The buckets version that the request is sent to. + // NOTE: This field is only meaningful while enable buckets. + uint64 buckets_version = 33; + + // It tells us where the request comes from in TiDB. If it isn't from TiDB, leave it blank. + // This is for tests only and thus can be safely changed/removed without affecting compatibility. + SourceStmt source_stmt = 34; +} + +message ResourceControlContext { + // It's used to identify which resource group the request belongs to. + string resource_group_name = 1; + + // The resource consumption of the resource group that have completed at all TiKVs between the previous request to this TiKV and current request. + // It's used as penalty to make the local resource scheduling on one TiKV takes the gloabl resource consumption into consideration. + resource_manager.Consumption penalty = 2; + + // This priority would override the original priority of the resource group for the request. + // Used to deprioritize the runaway queries. + uint64 override_priority = 3; } // The API version the server and the client is using. // See more details in https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md. enum APIVersion { - // Mainly for TxnKV and not safe to use RawKV along with TxnKV. - // - // V1 server only accepts V1 requests. Except that the V1 raw requests with TTL - // will be rejected. + // `V1` is mainly for TiDB & TxnKV, and is not safe to use RawKV along with the others. + // V1 server only accepts V1 requests. V1 raw requests with TTL will be rejected. V1 = 0; - // Only RawKV is available, and then 8 bytes representing the unix timestamp in - // seconds for expiring time will be append to the value of all RawKV kv pairs. // + // `V1TTL` is only available to RawKV, and 8 bytes representing the unix timestamp in + // seconds for expiring time will be append to the value of all RawKV entries. For example: // ------------------------------------------------------------ // | User value | Expire Ts | // ------------------------------------------------------------ // | 0x12 0x34 0x56 | 0x00 0x00 0x00 0x00 0x00 0x00 0xff 0xff | // ------------------------------------------------------------ - // // V1TTL server only accepts V1 raw requests. // V1 client should not use `V1TTL` in request. V1 client should always send `V1`. V1TTL = 1; - // TxnKV keys start with `x{keyspace id}`, `m`, or `t`. // - // RawKV keys must be in `default` CF and all start with `r{keyspace id}` prefix, - // where the keyspace id is in varint format (little endian), whose bytes expect - // the last one always sets the most significant bit to 1. + // `V2` use new encoding for RawKV & TxnKV to support more features. // - // The last byte in the raw value must be a meta flag. For example: + // Key Encoding: + // TiDB: start with `m` or `t`, the same as `V1`. + // TxnKV: prefix with `x`, encoded as `MCE( x{keyspace id} + {user key} ) + timestamp`. + // RawKV: prefix with `r`, encoded as `MCE( r{keyspace id} + {user key} ) + timestamp`. + // Where the `{keyspace id}` is fixed-length of 3 bytes in network byte order. + // Besides, RawKV entires must be in `default` CF. // - // -------------------------------------- - // | User value | Meta flags | - // -------------------------------------- - // | 0x12 0x34 0x56 | 0x00 (0b00000000) | - // -------------------------------------- + // Value Encoding: + // TiDB & TxnKV: the same as `V1`. + // RawKV: `{user value} + {optional fields} + {meta flag}`. The last byte in the + // raw value must be meta flags. For example: + // -------------------------------------- + // | User value | Meta flags | + // -------------------------------------- + // | 0x12 0x34 0x56 | 0x00 (0b00000000) | + // -------------------------------------- + // Bit 0 of meta flags is for TTL. If set, the value contains 8 bytes expiring time as + // unix timestamp in seconds at the very left to the meta flags. + // -------------------------------------------------------------------------------- + // | User value | Expiring time | Meta flags | + // -------------------------------------------------------------------------------- + // | 0x12 0x34 0x56 | 0x00 0x00 0x00 0x00 0x00 0x00 0xff 0xff | 0x01 (0b00000001) | + // -------------------------------------------------------------------------------- + // Bit 1 is for deletion. If set, the entry is logical deleted. + // --------------------- + // | Meta flags | + // --------------------- + // | 0x02 (0b00000010) | + // --------------------- // - // As shown in the example below, the least significant bit of the meta flag - // indicates whether the value contains 8 bytes expire ts at the very left to the - // meta flags. - // - // -------------------------------------------------------------------------------- - // | User value | Expire Ts | Meta flags | - // -------------------------------------------------------------------------------- - // | 0x12 0x34 0x56 | 0x00 0x00 0x00 0x00 0x00 0x00 0xff 0xff | 0x01 (0b00000001) | - // -------------------------------------------------------------------------------- - // - // V2 server accpets V2 requests and V1 txn requests that statrts with TiDB key + // V2 server accpets V2 requests and V1 transactional requests that statrts with TiDB key // prefix (`m` and `t`). V2 = 2; } +message SourceStmt { + uint64 start_ts = 1; + uint64 connection_id = 2; + uint64 stmt_id = 3; + // session alias set by user + string session_alias = 4; +} + message LockInfo { bytes primary_lock = 1; uint64 lock_version = 2; @@ -774,6 +967,10 @@ message LockInfo { bool use_async_commit = 8; uint64 min_commit_ts = 9; repeated bytes secondaries = 10; + // The time elapsed since last update of lock wait info when waiting. + // It's used in timeout errors. 0 means unknown or not applicable. + // It can be used to help the client decide whether to try resolving the lock. + uint64 duration_to_last_update_ms = 11; } message KeyError { @@ -787,14 +984,25 @@ message KeyError { TxnNotFound txn_not_found = 8; // Txn not found when checking txn status. CommitTsTooLarge commit_ts_too_large = 9; // Calculated commit TS exceeds the limit given by the user. AssertionFailed assertion_failed = 10; // Assertion of a `Mutation` is evaluated as a failure. + PrimaryMismatch primary_mismatch = 11; // CheckTxnStatus is sent to a lock that's not the primary. } message WriteConflict { + enum Reason { + Unknown = 0; + Optimistic = 1; // in optimistic transactions. + PessimisticRetry = 2; // a lock acquisition request waits for a lock and awakes, or meets a newer version of data, let TiDB retry. + SelfRolledBack = 3; // the transaction itself has been rolled back when it tries to prewrite. + RcCheckTs = 4; // RcCheckTs failure by meeting a newer version, let TiDB retry. + LazyUniquenessCheck = 5; // write conflict found in lazy uniqueness check in pessimistic transactions. + } + uint64 start_ts = 1; uint64 conflict_ts = 2; bytes key = 3; bytes primary = 4; uint64 conflict_commit_ts = 5; + Reason reason = 6; } message AlreadyExist { @@ -832,6 +1040,10 @@ message AssertionFailed { uint64 existing_commit_ts = 5; } +message PrimaryMismatch { + LockInfo lock_info = 1; +} + enum CommandPri { Normal = 0; // Normal is the default value. Low = 1; @@ -841,6 +1053,7 @@ enum CommandPri { enum IsolationLevel { SI = 0; // SI = snapshot isolation RC = 1; // RC = read committed + RCCheckTS = 2; // RC read and it's needed to check if there exists more recent versions. } // Operation allowed info during each TiKV storage threshold. @@ -852,16 +1065,36 @@ enum DiskFullOpt { message TimeDetail { // Off-cpu wall time elapsed in TiKV side. Usually this includes queue waiting time and - // other kind of waitings in series. - int64 wait_wall_time_ms = 1; + // other kind of waitings in series. (Wait time in the raftstore is not included.) + uint64 wait_wall_time_ms = 1; // Off-cpu and on-cpu wall time elapsed to actually process the request payload. It does not // include `wait_wall_time`. // This field is very close to the CPU time in most cases. Some wait time spend in RocksDB // cannot be excluded for now, like Mutex wait time, which is included in this field, so that // this field is called wall time instead of CPU time. - int64 process_wall_time_ms = 2; + uint64 process_wall_time_ms = 2; // KV read wall Time means the time used in key/value scan and get. - int64 kv_read_wall_time_ms = 3; + uint64 kv_read_wall_time_ms = 3; + // Total wall clock time spent on this RPC in TiKV . + uint64 total_rpc_wall_time_ns = 4; +} + +message TimeDetailV2 { + // Off-cpu wall time elapsed in TiKV side. Usually this includes queue waiting time and + // other kind of waitings in series. (Wait time in the raftstore is not included.) + uint64 wait_wall_time_ns = 1; + // Off-cpu and on-cpu wall time elapsed to actually process the request payload. It does not + // include `wait_wall_time` and `suspend_wall_time`. + // This field is very close to the CPU time in most cases. Some wait time spend in RocksDB + // cannot be excluded for now, like Mutex wait time, which is included in this field, so that + // this field is called wall time instead of CPU time. + uint64 process_wall_time_ns = 2; + // Cpu wall time elapsed that task is waiting in queue. + uint64 process_suspend_wall_time_ns = 3; + // KV read wall Time means the time used in key/value scan and get. + uint64 kv_read_wall_time_ns = 4; + // Total wall clock time spent on this RPC in TiKV . + uint64 total_rpc_wall_time_ns = 5; } message ScanInfo { @@ -911,6 +1144,21 @@ message ScanDetailV2 { // Total number of bytes from block reads. uint64 rocksdb_block_read_byte = 7; + + // Total time used for block reads. + uint64 rocksdb_block_read_nanos = 9; + + // Time used for getting a raftstore snapshot (including proposing read index, leader confirmation and getting the RocksDB snapshot). + uint64 get_snapshot_nanos = 10; + + // Time used for proposing read index from read pool to store pool, equals 0 when performing lease read. + uint64 read_index_propose_wait_nanos = 11; + + // Time used for leader confirmation, equals 0 when performing lease read. + uint64 read_index_confirm_wait_nanos = 12; + + // Time used for read pool scheduling. + uint64 read_pool_schedule_wait_nanos = 13; } message ExecDetails { @@ -927,10 +1175,69 @@ message ExecDetails { message ExecDetailsV2 { // Available when ctx.record_time_stat = true or meet slow query. + // deprecated. Should use `time_detail_v2` instead. TimeDetail time_detail = 1; // Available when ctx.record_scan_stat = true or meet slow query. ScanDetailV2 scan_detail_v2 = 2; + + // Raftstore writing durations of the request. Only available for some write requests. + WriteDetail write_detail = 3; + // Available when ctx.record_time_stat = true or meet slow query. + TimeDetailV2 time_detail_v2 = 4; +} + +message WriteDetail { + // Wait duration in the store loop. + uint64 store_batch_wait_nanos = 1; + + // Wait duration before sending proposal to peers. + uint64 propose_send_wait_nanos = 2; + + // Total time spent on persisting the log. + uint64 persist_log_nanos = 3; + + // Wait time until the Raft log write leader begins to write. + uint64 raft_db_write_leader_wait_nanos = 4; + + // Time spent on synchronizing the Raft log to the disk. + uint64 raft_db_sync_log_nanos = 5; + + // Time spent on writing the Raft log to the Raft memtable. + uint64 raft_db_write_memtable_nanos = 6; + + // Time waiting for peers to confirm the proposal (counting from the instant when the leader sends the proposal message). + uint64 commit_log_nanos = 7; + + // Wait duration in the apply loop. + uint64 apply_batch_wait_nanos = 8; + + // Total time spend to applying the log. + uint64 apply_log_nanos = 9; + + // Wait time until the KV RocksDB lock is acquired. + uint64 apply_mutex_lock_nanos = 10; + + // Wait time until becoming the KV RocksDB write leader. + uint64 apply_write_leader_wait_nanos = 11; + + // Time spent on writing the KV DB WAL to the disk. + uint64 apply_write_wal_nanos = 12; + + // Time spent on writing to the memtable of the KV RocksDB. + uint64 apply_write_memtable_nanos = 13; + + // Time spent on waiting in the latch. + uint64 latch_wait_nanos = 14; + + // Processing time in the transaction layer. + uint64 process_nanos = 15; + + // Wait time because of the scheduler flow control or quota limiter throttling. + uint64 throttle_nanos = 16; + + // Wait time in the waiter manager for pessimistic locking. + uint64 pessimistic_lock_wait_nanos = 17; } message KvPair { @@ -980,6 +1287,8 @@ message MvccWrite { bool has_overlapped_rollback = 5; bool has_gc_fence = 6; uint64 gc_fence = 7; + uint64 last_change_ts = 8; + uint64 versions_to_last_change = 9; } message MvccValue { @@ -998,6 +1307,8 @@ message MvccLock { bool use_async_commit = 8; repeated bytes secondaries = 9; repeated uint64 rollback_ts = 10; + uint64 last_change_ts = 11; + uint64 versions_to_last_change = 12; } message MvccInfo { @@ -1108,6 +1419,18 @@ message GetLockWaitInfoResponse { repeated deadlock.WaitForEntry entries = 3; } +message GetLockWaitHistoryRequest { + Context context = 1; + // TODO: There may need some filter options to be used on conditional querying, e.g., finding + // the lock waiting status for some specified transaction. +} + +message GetLockWaitHistoryResponse { + errorpb.Error region_error = 1; + string error = 2; + repeated deadlock.WaitForEntry entries = 3; +} + message RawCoprocessorRequest { kvrpcpb.Context context = 1; @@ -1144,3 +1467,66 @@ message RawChecksumResponse { uint64 total_kvs = 4; uint64 total_bytes = 5; } + +message CompactError { + oneof error { + CompactErrorInvalidStartKey err_invalid_start_key = 1; + CompactErrorPhysicalTableNotExist err_physical_table_not_exist = 2; + CompactErrorCompactInProgress err_compact_in_progress = 3; + CompactErrorTooManyPendingTasks err_too_many_pending_tasks = 4; + } +} + +message CompactErrorInvalidStartKey {} + +message CompactErrorPhysicalTableNotExist {} + +message CompactErrorCompactInProgress {} + +message CompactErrorTooManyPendingTasks {} + +message CompactRequest { + // If specified, the compaction will start from this start key. + // If unspecified, the compaction will start from beginning. + // NOTE 1: The start key should be never manually constructed. You should always use a key + // returned in CompactResponse. + // NOTE 2: the compaction range will be always restricted by physical_table_id. + bytes start_key = 1; + + // The physical table that will be compacted. + // + // TODO: this is information that TiKV doesn't need to know. + // See https://github.com/pingcap/kvproto/issues/912 + int64 physical_table_id = 2; + + // The logical table id of the compaction. When receiving parallel requests with the same + // logical table id, err_compact_in_progress will be returned. + // + // TODO: this is information that TiKV doesn't need to know. + // See https://github.com/pingcap/kvproto/issues/912 + int64 logical_table_id = 3; + + // API version of the request + APIVersion api_version = 7; + // Keyspace of the table located in. + uint32 keyspace_id = 8; +} + +message CompactResponse { + CompactError error = 1; + + // The compaction is done incrementally. If there are more data to compact, this field + // will be set. The client can request to compact more data according to the `compacted_end_key`. + bool has_remaining = 2; + + bytes compacted_start_key = 3; + bytes compacted_end_key = 4; +} + +message TiFlashSystemTableRequest { + string sql = 1; +} + +message TiFlashSystemTableResponse { + bytes data = 1 [(gogoproto.customtype) = "github.com/pingcap/kvproto/pkg/sharedbytes.SharedBytes", (gogoproto.nullable) = false]; +} diff --git a/proto/logbackuppb.proto b/proto/logbackuppb.proto new file mode 100644 index 0000000..17e9820 --- /dev/null +++ b/proto/logbackuppb.proto @@ -0,0 +1,58 @@ +syntax = "proto3"; +package logbackup; + +import "gogoproto/gogo.proto"; +import "rustproto.proto"; +import "errorpb.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 minimal information for identify a region. +message RegionIdentity { + uint64 id = 1; + uint64 epoch_version = 2; + // We omitted epoch_conf_version because config change won't make range change. +} + +// The last flush ts with region information. +message RegionCheckpoint { + errorpb.Error err = 1; + RegionIdentity region = 2; + uint64 checkpoint = 3; +} + +message GetLastFlushTSOfRegionRequest { + repeated RegionIdentity regions = 1; +} + +message GetLastFlushTSOfRegionResponse { + repeated RegionCheckpoint checkpoints = 1; +} + +message SubscribeFlushEventRequest { + string client_id = 1; +} + +message SubscribeFlushEventResponse { + repeated FlushEvent events = 1; +} + +message FlushEvent { + bytes start_key = 1; + bytes end_key = 2; + uint64 checkpoint = 3; +} + +// The log backup service. +// Generally, most essential interfaces of log backup (say, checkpoint management, task management) are +// provided by adding some key in the embed etcd of PD. +// This interface is mainly provided for the checkpoint advancer and debug usage. +service LogBackup { + rpc GetLastFlushTSOfRegion(GetLastFlushTSOfRegionRequest) returns (GetLastFlushTSOfRegionResponse) {} + rpc SubscribeFlushEvent(SubscribeFlushEventRequest) returns (stream SubscribeFlushEventResponse) {} +} diff --git a/proto/meta_storagepb.proto b/proto/meta_storagepb.proto new file mode 100644 index 0000000..bae7ae5 --- /dev/null +++ b/proto/meta_storagepb.proto @@ -0,0 +1,138 @@ +syntax = "proto3"; +package meta_storagepb; + +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"; + +// MetaStorage is the meta storage service. +service MetaStorage { + rpc Watch(WatchRequest) returns (stream WatchResponse) {} + + // Get is the same as etcd Range which might be implemented in a more common way + // so that we can use other storages to replace etcd in the future. + rpc Get(GetRequest) returns (GetResponse) {} + + rpc Put(PutRequest) returns (PutResponse) {} +} + +enum ErrorType { + OK = 0; + UNKNOWN = 1; + // required watch revision is smaller than current compact/min revision. + DATA_COMPACTED = 2; +} + +message Error { + ErrorType type = 1; + string message = 2; +} + +message RequestHeader { + // cluster_id is the ID of the cluster which be sent to. + uint64 cluster_id = 1; + // source is the source of the request. + string source = 2; +} + +message ResponseHeader { + // cluster_id is the ID of the cluster which sent the response. + uint64 cluster_id = 1; + Error error = 2; + int64 revision = 3; +} + +// copied part of https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto +message WatchRequest { + RequestHeader header = 1; + bytes key = 2; + bytes range_end = 3; + int64 start_revision = 4; + bool prev_kv = 5; +} + +// copied part of https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto +message WatchResponse { + ResponseHeader header = 1; + int64 compact_revision = 2; + repeated Event events = 3; +} + +// copied part of https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto +message GetRequest { + RequestHeader header = 1; + bytes key = 2; + bytes range_end = 3; + int64 limit = 4; + int64 revision = 5; +} + +// copied part of https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto +message GetResponse { + ResponseHeader header = 1; + repeated KeyValue kvs = 2; + bool more = 3; + int64 count = 4; +} + +// copied part of https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto +message PutRequest { + RequestHeader header = 1; + bytes key = 2; + bytes value = 3; + int64 lease = 4; + bool prev_kv = 5; +} + +// copied part of https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto +message PutResponse { + ResponseHeader header = 1; + KeyValue prev_kv = 2; +} + +// copied from etcd https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/mvccpb/kv.proto +message KeyValue { + // key is the key in bytes. An empty key is not allowed. + bytes key = 1; + // create_revision is the revision of last creation on this key. + int64 create_revision = 2; + // mod_revision is the revision of last modification on this key. + int64 mod_revision = 3; + // version is the version of the key. A deletion resets + // the version to zero and any modification of the key + // increases its version. + int64 version = 4; + // value is the value held by the key, in bytes. + bytes value = 5; + // lease is the ID of the lease that attached to key. + // When the attached lease expires, the key will be deleted. + // If lease is 0, then no lease is attached to the key. + int64 lease = 6; +} + +// copied from etcd https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/mvccpb/kv.proto +message Event { + enum EventType { + PUT = 0; + DELETE = 1; + } + // type is the kind of event. If type is a PUT, it indicates + // new data has been stored to the key. If type is a DELETE, + // it indicates the key was deleted. + EventType type = 1; + // kv holds the KeyValue for the event. + // A PUT event contains current kv pair. + // A PUT event with kv.Version=1 indicates the creation of a key. + // A DELETE/EXPIRE event contains the deleted key with + // its modification revision set to the revision of deletion. + KeyValue kv = 2; + + // prev_kv holds the key-value pair before the event happens. + KeyValue prev_kv = 3; +} diff --git a/proto/metapb.proto b/proto/metapb.proto index f59de84..ef3c8d4 100644 --- a/proto/metapb.proto +++ b/proto/metapb.proto @@ -26,6 +26,17 @@ enum StoreState { Tombstone = 2; } +// NodeState is going to replace StoreState to make the state concept more clear. +// "Up" is devided into "Preparing" and "Serving" stages so that we can better describe the online process. +// "Removing" is just like previous `Offline` which is more accurate. +// "Removed" has the same meaning with `Tombstone`. +enum NodeState { + Preparing = 0; + Serving = 1; + Removing = 2; + Removed = 3; +} + // Case insensitive key/value for replica constraints. message StoreLabel { string key = 1; @@ -52,6 +63,8 @@ message Store { int64 last_heartbeat = 11; // If the store is physically destroyed, which means it can never up again. bool physically_destroyed = 12; + // NodeState is used to replace StoreState which will be deprecated in the future. + NodeState node_state = 13; } message RegionEpoch { @@ -61,6 +74,42 @@ message RegionEpoch { uint64 version = 2; } +message BucketStats { + // total read in bytes of each bucket + repeated uint64 read_bytes = 1; + + // total write in bytes of each bucket + repeated uint64 write_bytes = 2; + + // total read qps of each bucket + repeated uint64 read_qps = 3; + + // total write qps of each bucket + repeated uint64 write_qps = 4; + + // total read keys of each bucket + repeated uint64 read_keys = 5; + + // total write keys of each bucket + repeated uint64 write_keys = 6; +} + +message Buckets { + uint64 region_id = 1; + + // A hint indicate if keys have changed. + uint64 version = 2; + + // keys of buckets, include start/end key of region + repeated bytes keys = 3; + + // bucket stats + BucketStats stats = 4; + + // The period in milliseconds that stats are collected with in + uint64 period_in_ms = 5; +} + message Region { uint64 id = 1; // Region key range [start_key, end_key). @@ -73,6 +122,11 @@ message Region { // The field is only used by PD and should be ignored otherwise. // If encryption_meta is empty (i.e. nil), it means start_key and end_key are unencrypted. encryptionpb.EncryptionMeta encryption_meta = 6; + // The flashback state indicates whether this region is in the flashback state. + // TODO: only check by `flashback_start_ts` in the future. Keep for compatibility now. + bool is_in_flashback = 7; + // The start_ts that the current flashback progress is using. + uint64 flashback_start_ts = 8; } enum PeerRole { @@ -94,4 +148,5 @@ message Peer { uint64 id = 1; uint64 store_id = 2; PeerRole role = 3; + bool is_witness = 4; } diff --git a/proto/mpp.proto b/proto/mpp.proto index 4167b49..268e218 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -4,6 +4,7 @@ package mpp; import "gogoproto/gogo.proto"; import "coprocessor.proto"; import "metapb.proto"; +import "kvrpcpb.proto"; option (gogoproto.marshaler_all) = true; option (gogoproto.sizer_all) = true; @@ -17,13 +18,26 @@ message TaskMeta { int64 task_id = 2; // if task id is -1 , it indicates a tidb task. int64 partition_id = 3; // Only used for hash partition string address = 4; // target address of this task. + uint64 gather_id = 5; // used to distinguish different gathers in the mpp query. + uint64 query_ts = 6; // timestamp when start to execute query, used for TiFlash miniTSO schedule. + uint64 local_query_id = 7; // unique local query_id if tidb don't restart. So we can use gather_id + query_ts + local_query_id + server_id to represent a global unique query. + uint64 server_id = 8; // TiDB server id + int64 mpp_version = 9; // mpp version + uint32 keyspace_id = 10; // keyspace id of the request + string coordinator_address = 11; // coordinator_address of this query + bool report_execution_summary = 12; // Only when coordinator_address is not empty, this flag can be true. When set to true, TiFlash only report execution summary through ReportMPPTaskStatus service, don't include summaries in MppDataPacket + kvrpcpb.APIVersion api_version = 16; // API version of the request + string resource_group_name = 17; + uint64 connection_id = 18; // This is the session id between a client and tidb + string connection_alias = 19; // This is the session alias between a client and tidb } message IsAliveRequest { } message IsAliveResponse { - bool available = 1; + bool available = 1; + int64 mpp_version = 2; } // Dipsatch the task request to different tiflash servers. @@ -31,9 +45,11 @@ message DispatchTaskRequest { TaskMeta meta = 1; bytes encoded_plan = 2; int64 timeout = 3; - repeated coprocessor.RegionInfo regions = 4; + repeated coprocessor.RegionInfo regions = 4; // If this task contains table scan, we still need their region info. int64 schema_ver = 5; + // Used for partition table scan + repeated coprocessor.TableRegions table_regions = 6; } // Get response of DispatchTaskRequest. @@ -52,21 +68,36 @@ message CancelTaskResponse { Error error = 1; } +// ReportTaskStatus reports the execution status of a task. +// when TiFlash reports status to TiDB, ReportTaskStatusRequest serialize tipb.TiFlashExecutionInfo into data; +message ReportTaskStatusRequest { + TaskMeta meta = 1; + bytes data = 2; + Error error = 3; +} + +message ReportTaskStatusResponse { + Error error = 1; +} + // build connection between different tasks. Data is sent by the tasks that are closer to the data sources. message EstablishMPPConnectionRequest { - TaskMeta sender_meta = 1; // node closer to the source + TaskMeta sender_meta = 1; // node closer to the source TaskMeta receiver_meta = 2; // node closer to the tidb mpp gather. } // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. message MPPDataPacket { - bytes data = 1; + bytes data = 1; Error error = 2; repeated bytes chunks = 3; + repeated uint64 stream_ids = 4; + int64 version = 5; // version of data packet format } message Error { int32 code = 1; string msg = 2; + int64 mpp_version = 3; } diff --git a/proto/pdpb.proto b/proto/pdpb.proto index 4f3d08f..c8fec84 100644 --- a/proto/pdpb.proto +++ b/proto/pdpb.proto @@ -17,6 +17,10 @@ option (rustproto.lite_runtime_all) = true; option java_package = "org.tikv.kvproto"; service PD { + // GetClusterInfo get the information of this cluster. It does not require + // the cluster_id in request matchs the id of this cluster. + rpc GetClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse) {} + // GetMembers get the member list of this cluster. It does not require // the cluster_id in request matchs the id of this cluster. rpc GetMembers(GetMembersRequest) returns (GetMembersResponse) {} @@ -29,6 +33,8 @@ service PD { rpc AllocID(AllocIDRequest) returns (AllocIDResponse) {} + rpc IsSnapshotRecovering(IsSnapshotRecoveringRequest) returns (IsSnapshotRecoveringResponse) {} + rpc GetStore(GetStoreRequest) returns (GetStoreResponse) {} rpc PutStore(PutStoreRequest) returns (PutStoreResponse) {} @@ -73,6 +79,16 @@ service PD { rpc UpdateServiceGCSafePoint(UpdateServiceGCSafePointRequest) returns (UpdateServiceGCSafePointResponse) {} + rpc GetGCSafePointV2(GetGCSafePointV2Request) returns (GetGCSafePointV2Response) {} + + rpc WatchGCSafePointV2(WatchGCSafePointV2Request) returns (stream WatchGCSafePointV2Response) {} + + rpc UpdateGCSafePointV2(UpdateGCSafePointV2Request) returns (UpdateGCSafePointV2Response) {} + + rpc UpdateServiceSafePointV2(UpdateServiceSafePointV2Request) returns (UpdateServiceSafePointV2Response) {} + + rpc GetAllGCSafePointV2(GetAllGCSafePointV2Request) returns (GetAllGCSafePointV2Response) {} + rpc SyncRegions(stream SyncRegionRequest) returns (stream SyncRegionResponse) {} rpc GetOperator(GetOperatorRequest) returns (GetOperatorResponse) {} @@ -84,6 +100,71 @@ service PD { rpc SplitAndScatterRegions(SplitAndScatterRegionsRequest) returns (SplitAndScatterRegionsResponse) {} rpc GetDCLocationInfo(GetDCLocationInfoRequest) returns (GetDCLocationInfoResponse) {} + + rpc StoreGlobalConfig(StoreGlobalConfigRequest) returns (StoreGlobalConfigResponse) {} + + rpc LoadGlobalConfig(LoadGlobalConfigRequest) returns (LoadGlobalConfigResponse) {} + + rpc WatchGlobalConfig(WatchGlobalConfigRequest) returns (stream WatchGlobalConfigResponse) {} + + rpc ReportBuckets(stream ReportBucketsRequest) returns (ReportBucketsResponse) {} + + rpc ReportMinResolvedTS(ReportMinResolvedTsRequest) returns (ReportMinResolvedTsResponse) {} + + rpc SetExternalTimestamp(SetExternalTimestampRequest) returns (SetExternalTimestampResponse) {} + + rpc GetExternalTimestamp(GetExternalTimestampRequest) returns (GetExternalTimestampResponse) {} + + // Get the minimum timestamp across all keyspace groups from API server + // TODO: Currently, we need to ask API server to get the minimum timestamp. + // Once we support service discovery, we can remove it. + rpc GetMinTS (GetMinTSRequest) returns (GetMinTSResponse) {} +} + +message WatchGlobalConfigRequest { + string config_path = 1; + int64 revision = 2; +} + +message WatchGlobalConfigResponse { + repeated GlobalConfigItem changes = 1; + int64 revision = 2; + ResponseHeader header = 3; +} + +message StoreGlobalConfigRequest { + repeated GlobalConfigItem changes = 1; + string config_path = 2; +} + +message StoreGlobalConfigResponse { + Error error = 1; +} + +message LoadGlobalConfigRequest { + repeated string names = 1; + string config_path = 2; +} + +message LoadGlobalConfigResponse { + repeated GlobalConfigItem items = 1; + int64 revision = 2; +} + +enum EventType { + PUT = 0; + DELETE = 1; +} + +message GlobalConfigItem { + string name = 1; + // this field 'value' is replaced by the field 'payload'. + string value = 2; + Error error = 3; + EventType kind = 4; + // Since item value needs to support marshal of different struct types, + // it should be set to bytes instead of string. + bytes payload = 5; } message RequestHeader { @@ -107,6 +188,12 @@ enum ErrorType { ALREADY_BOOTSTRAPPED = 4; INCOMPATIBLE_VERSION = 5; REGION_NOT_FOUND = 6; + GLOBAL_CONFIG_NOT_FOUND = 7; + DUPLICATED_ENTRY = 8; + ENTRY_NOT_FOUND = 9; + INVALID_VALUE = 10; + // required watch revision is smaller than current compact/min revision. + DATA_COMPACTED = 11; } message Error { @@ -126,7 +213,8 @@ message Timestamp { int64 logical = 2; // Number of suffix bits used for global distinction, // PD client will use this to compute a TSO's logical part. - uint32 suffix_bits = 3;} + uint32 suffix_bits = 3; +} message TsoResponse { ResponseHeader header = 1; @@ -167,6 +255,15 @@ message AllocIDResponse { uint64 id = 2; } +message IsSnapshotRecoveringRequest { + RequestHeader header = 1; +} + +message IsSnapshotRecoveringResponse { + ResponseHeader header = 1; + bool marked = 2; +} + message GetStoreRequest { RequestHeader header = 1; @@ -207,6 +304,7 @@ message GetRegionRequest { RequestHeader header = 1; bytes region_key = 2; + bool need_buckets = 3; } message GetRegionResponse { @@ -221,12 +319,15 @@ message GetRegionResponse { // Pending peers are the peers that the leader can't consider as // working followers. repeated metapb.Peer pending_peers = 6; + // buckets isn't nil if GetRegion.* requests set need_buckets. + metapb.Buckets buckets = 7; } message GetRegionByIDRequest { RequestHeader header = 1; uint64 region_id = 2; + bool need_buckets = 3; } // Use GetRegionResponse as the response of GetRegionByIDRequest. @@ -307,6 +408,25 @@ message GetMembersResponse { map tso_allocator_leaders = 5; } +message GetClusterInfoRequest { + ResponseHeader header = 1; +} + +enum ServiceMode { + UNKNOWN_SVC_MODE = 0; + PD_SVC_MODE = 1; + API_SVC_MODE = 2; +} + +message GetClusterInfoResponse { + ResponseHeader header = 1; + + repeated ServiceMode serviceModes = 2; + // If service mode is API_SVC_MODE, this field will be set to the + // registered tso service addresses. + repeated string tso_urls = 3; +} + message PeerStats { metapb.Peer peer = 1; uint64 down_seconds = 2; @@ -344,6 +464,9 @@ message RegionHeartbeatRequest { // cpu_usage is the CPU time usage of the leader region since the last heartbeat, // which is calculated by cpu_time_delta/heartbeat_reported_interval. uint64 cpu_usage = 17; + // (Serverless) Approximate size of key-value pairs for billing. + // It's counted on size of user key & value (excluding metadata fields), before compression, and latest versions only. + uint64 approximate_kv_size = 18; } message ChangePeer { @@ -370,6 +493,15 @@ message SplitRegion { repeated bytes keys = 2; } +message SwitchWitness { + uint64 peer_id = 1; + bool is_witness = 2; +} + +message BatchSwitchWitness { + repeated SwitchWitness switch_witnesses = 1; +} + enum CheckPolicy { SCAN = 0; APPROXIMATE = 1; @@ -407,8 +539,11 @@ message RegionHeartbeatResponse { // Multiple change peer operations atomically. // Note: PD can use both ChangePeer and ChangePeerV2 at the same time // (not in the same RegionHeartbeatResponse). - // Now, PD use ChangePeerV2 only for replacing peers. + // Now, PD use ChangePeerV2 in following scenarios: + // 1. replacing peers + // 2. demoting voter directly ChangePeerV2 change_peer_v2 = 9; + BatchSwitchWitness switch_witnesses = 10; } message AskSplitRequest { @@ -484,6 +619,8 @@ message PeerStat { uint64 read_keys = 2; uint64 read_bytes = 3; QueryStats query_stats = 4; + uint64 written_keys = 5; + uint64 written_bytes = 6; } message StoreStats { @@ -532,37 +669,122 @@ message StoreStats { uint64 slow_score = 22; // Damaged regions on the store that need to be removed by PD. repeated uint64 damaged_regions_id = 23; + // If the apply worker is busy, namely high apply wait duration + bool is_apply_busy = 24; + // Snapshot stats in the store + repeated SnapshotStat snapshot_stats = 25; + SlowTrend slow_trend = 26; + // If the grpc server is paused. + bool is_grpc_paused = 27; + // Total memory of the store in bytes. + uint64 total_memory = 28; + // Used memory of the store in bytes. + uint64 used_memory = 29; +} + +message SlowTrend{ + double cause_value = 1; + double cause_rate = 2; + double result_value = 3; + double result_rate = 4; +} + +message SnapshotStat{ + uint64 region_id = 1; + // Generate snapshot duration + uint64 generate_duration_sec = 2; + // Send snapshot duration + uint64 send_duration_sec = 3; + // |-- waiting --|-- generate --|-- send --| + // |-----------total duration---------------| + // Total duration include waiting and executing duration + uint64 total_duration_sec = 4; + // Size is the transport data size + uint64 transport_size = 5; } message PeerReport { raft_serverpb.RaftLocalState raft_state = 1; raft_serverpb.RegionLocalState region_state = 2; + bool is_force_leader = 3; + // The peer has proposed but uncommitted commit merge. + bool has_commit_merge = 4; } message StoreReport { repeated PeerReport peer_reports = 1; + uint64 step = 2; } message StoreHeartbeatRequest { RequestHeader header = 1; StoreStats stats = 2; - // Detailed store report that is only filled up on PD's demand for online unsafe recover. + // Detailed store report that is only filled up on PD's demand for online unsafe recovery. StoreReport store_report = 3; + replication_modepb.StoreDRAutoSyncStatus dr_autosync_status = 4; +} + +message DemoteFailedVoters { + uint64 region_id = 1; + repeated metapb.Peer failed_voters = 2; +} + +message ForceLeader { + // The store ids of the failed stores, TiKV uses it to decide if a peer is alive. + repeated uint64 failed_stores = 1; + // The region ids of the peer which is to be force leader. + repeated uint64 enter_force_leaders = 2; } message RecoveryPlan { + // Create empty regions to fill the key range hole. repeated metapb.Region creates = 1; - repeated metapb.Region updates = 2; - repeated uint64 deletes = 3; + // Update the meta of the regions, including peer lists, epoch and key range. + repeated metapb.Region updates = 2 [deprecated=true]; + // Tombstone the peers on the store locally. + repeated uint64 tombstones = 3; + // Issue conf change that demote voters on failed stores to learners on the regions. + repeated DemoteFailedVoters demotes = 4; + // Make the peers to be force leaders. + ForceLeader force_leader = 5; + // Step is an increasing number to note the round of recovery, + // It should be filled in the corresponding store report. + uint64 step = 6; +} + +message AwakenRegions { + // Awake all regions if abnormal_stores is empty. + repeated uint64 abnormal_stores = 1; +} + +enum ControlGrpcEvent { + // Pause TiKV grpc server. + PAUSE = 0; + // Resume TiKV grpc server. + RESUME = 1; +} + +message ControlGrpc { + ControlGrpcEvent ctrl_event = 1; } message StoreHeartbeatResponse { ResponseHeader header = 1; replication_modepb.ReplicationStatus replication_status = 2; string cluster_version = 3; - bool require_detailed_report = 4; - RecoveryPlan plan = 5; + + // Used by online unsafe recovery to request store report. + // Now it's substituted by reusing recovery_plan field. PD will send a empty + // recovery plan instead to request store report. + bool require_detailed_report = 4 [deprecated=true]; + // Operations of recovery. After the plan is executed, TiKV should attach the + // store report in store heartbeat. + RecoveryPlan recovery_plan = 5; + // Pd can return awaken_regions to let TiKV awaken hibernated regions itself. + AwakenRegions awaken_regions = 6; + // Pd can return operations to let TiKV forcely PAUSE | RESUME grpc server. + ControlGrpc control_grpc = 7; } message ScatterRegionRequest { @@ -582,6 +804,7 @@ message ScatterRegionRequest { // If regions_id is defined, the region_id would be ignored. repeated uint64 regions_id = 6; uint64 retry_limit = 7; + bool skip_store_limit = 8; } message ScatterRegionResponse { @@ -627,6 +850,87 @@ message UpdateServiceGCSafePointResponse { uint64 min_safe_point = 4; } +message GetGCSafePointV2Request { + RequestHeader header = 1; + + uint32 keyspace_id = 2; +} + +message GetGCSafePointV2Response { + ResponseHeader header = 1; + + uint64 safe_point = 2; +} + +message WatchGCSafePointV2Request { + RequestHeader header = 1; + int64 revision = 2; + +} + +// SafePointEvent is for the rpc WatchGCSafePointV2. +message SafePointEvent { + uint32 keyspace_id = 1; + uint64 safe_point = 2; + EventType type = 3; +} + +message WatchGCSafePointV2Response { + ResponseHeader header = 1; + repeated SafePointEvent events = 2; + int64 revision = 3; +} + +message UpdateGCSafePointV2Request { + RequestHeader header = 1; + + uint32 keyspace_id = 2; + uint64 safe_point = 3; +} + +message UpdateGCSafePointV2Response { + ResponseHeader header = 1; + + uint64 new_safe_point = 2; +} + +message UpdateServiceSafePointV2Request { + RequestHeader header = 1; + + uint32 keyspace_id = 2; + bytes service_id = 3; + uint64 safe_point = 4; + // Safe point will be set to expire on (PD Server time + TTL), + // pass in a ttl < 0 to remove target safe point; + // pass in MAX_INT64 to set a safe point that never expire. + // This should be set by component that may crash unexpectedly so that it doesn't block + // cluster garbage collection. + int64 ttl = 5; +} + +message UpdateServiceSafePointV2Response { + ResponseHeader header = 1; + + bytes service_id = 2; + int64 ttl = 3; + uint64 min_safe_point = 4; +} + +message GetAllGCSafePointV2Request { + RequestHeader header = 1; +} + +message GCSafePointV2 { + uint32 keyspace_id = 1; + uint64 gc_safe_point = 2; +} + +message GetAllGCSafePointV2Response { + ResponseHeader header = 1; + repeated GCSafePointV2 gc_safe_points = 2; + int64 revision = 3; +} + message RegionStat { // Bytes read/written during this period. uint64 bytes_written = 1; @@ -644,6 +948,14 @@ message SyncRegionRequest{ uint64 start_index = 3; } +message PeersStats { + repeated PeerStats peers = 1; +} + +message Peers { + repeated metapb.Peer peers = 1; +} + message SyncRegionResponse{ ResponseHeader header = 1; // the leader PD will send the repsonds include @@ -652,6 +964,10 @@ message SyncRegionResponse{ uint64 start_index = 3; repeated RegionStat region_stats = 4; repeated metapb.Peer region_leaders = 5; + // the buckets informations without stats. + repeated metapb.Buckets buckets = 6; + repeated PeersStats down_peers = 16; + repeated Peers pending_peers = 17; } message GetOperatorRequest { @@ -756,3 +1072,56 @@ enum QueryKind { Commit = 10; Rollback = 11; } + +message ReportBucketsRequest { + RequestHeader header = 1; + + metapb.RegionEpoch region_epoch = 2; + metapb.Buckets buckets = 3; +} + +message ReportBucketsResponse { + ResponseHeader header = 1; +} + +message ReportMinResolvedTsRequest { + RequestHeader header = 1; + + uint64 store_id = 2; + + uint64 min_resolved_ts = 3; +} + +message ReportMinResolvedTsResponse { + ResponseHeader header = 1; +} + +message SetExternalTimestampRequest { + RequestHeader header = 1; + + uint64 timestamp = 2; +} + +message SetExternalTimestampResponse { + ResponseHeader header = 1; +} + +message GetExternalTimestampRequest { + RequestHeader header = 1; +} + +message GetExternalTimestampResponse { + ResponseHeader header = 1; + + uint64 timestamp = 2; +} + +message GetMinTSRequest { + RequestHeader header = 1; +} + +message GetMinTSResponse { + ResponseHeader header = 1; + + Timestamp timestamp = 2; +} diff --git a/proto/raft_cmdpb.proto b/proto/raft_cmdpb.proto index fcb488b..155210b 100644 --- a/proto/raft_cmdpb.proto +++ b/proto/raft_cmdpb.proto @@ -6,6 +6,7 @@ import "errorpb.proto"; import "eraftpb.proto"; import "kvrpcpb.proto"; import "import_sstpb.proto"; +import "raft_serverpb.proto"; import "rustproto.proto"; option (rustproto.lite_runtime_all) = true; @@ -145,6 +146,11 @@ message SplitRequest { // left region use new_region_id. // Will be ignored in batch split, use `BatchSplitRequest::right_derive` instead. bool right_derive = 4 [deprecated=true]; + // It should be false iff the region split by user key such as split table or create partion table etc, + // the new region's will not share the source region size, so it's region size is zero. + // It should be true iff the region's load reaches the threshold such as size, keys, load check etc, + // the new region's size will share the origin region, so it's region size is half of the source region. + bool share_source_region_size = 5; } message SplitResponse { @@ -157,6 +163,11 @@ message BatchSplitRequest { // If true, the last region derive the origin region_id, // other regions use new ids. bool right_derive = 2; + // It should be false iff the region split by user key such as split table or create partion table etc, + // the new region's will not share the source region size, so it's region size is zero. + // It should be true iff the region's load reaches the threshold such as size, keys, load check etc, + // the new region's size will share the origin region, so it's region size is half of the source region. + bool share_source_region_size = 3; } message BatchSplitResponse { @@ -166,6 +177,7 @@ message BatchSplitResponse { message CompactLogRequest { uint64 compact_index = 1; uint64 compact_term = 2; + uint64 voter_replicated_index = 3; } message CompactLogResponse {} @@ -197,10 +209,25 @@ message PrepareMergeRequest { message PrepareMergeResponse {} +message PrepareFlashbackRequest { + // The start_ts that the current flashback progress is using. + uint64 start_ts = 1; +} + +message PrepareFlashbackResponse {} + +message FinishFlashbackRequest {} + +message FinishFlashbackResponse {} + message CommitMergeRequest { metapb.Region source = 1; uint64 commit = 2; repeated eraftpb.Entry entries = 3; + // Used in v2. When it's present, `source` and `commit` will not be set. + raft_serverpb.RegionLocalState source_state = 4; + + reserved 100 to 200; } message CommitMergeResponse {} @@ -211,6 +238,21 @@ message RollbackMergeRequest { message RollbackMergeResponse {} +message SwitchWitnessRequest { + uint64 peer_id = 1; + bool is_witness = 2; +} + +message BatchSwitchWitnessRequest { + repeated SwitchWitnessRequest switch_witnesses = 1; +} + +message BatchSwitchWitnessResponse {} + +message UpdateGcPeerRequest { + repeated uint64 peer_id = 1; +} + enum AdminCmdType { InvalidAdmin = 0; ChangePeer = 1; @@ -225,6 +267,11 @@ enum AdminCmdType { RollbackMerge = 9; BatchSplit = 10; ChangePeerV2 = 11; + PrepareFlashback = 12; + FinishFlashback = 13; + BatchSwitchWitness = 14; + // Command that updates RegionLocalState.gc_peers + UpdateGcPeer = 15; } message AdminRequest { @@ -240,6 +287,10 @@ message AdminRequest { BatchSplitRequest splits = 10; ChangePeerV2Request change_peer_v2 = 11; ComputeHashRequest compute_hash = 12; + PrepareFlashbackRequest prepare_flashback = 13; + FinishFlashbackRequest finish_flashback = 14; + BatchSwitchWitnessRequest switch_witnesses = 15; + UpdateGcPeerRequest update_gc_peers = 16; } message AdminResponse { @@ -254,6 +305,10 @@ message AdminResponse { RollbackMergeResponse rollback_merge = 9; BatchSplitResponse splits = 10; ChangePeerV2Response change_peer_v2 = 11; + PrepareFlashbackResponse prepare_flashback = 12; + FinishFlashbackResponse finish_flashback = 13; + BatchSwitchWitnessResponse switch_witnesses = 14; + // UpdateGcPeer doesn't need to be responded. Avoid consuming a tag number. } // For get the leader of the region. @@ -318,6 +373,9 @@ message RaftRequestHeader { // Custom flags for this raft request. uint64 flags = 10; bytes flag_data = 11; + + kvrpcpb.CommandPri priority = 12; + string resource_group_name = 13; } message RaftResponseHeader { @@ -333,6 +391,9 @@ message RaftCmdRequest { repeated Request requests = 2; AdminRequest admin_request = 3; StatusRequest status_request = 4; + + reserved 5; + reserved 100 to 200; } message RaftCmdResponse { diff --git a/proto/raft_serverpb.proto b/proto/raft_serverpb.proto index 8b07133..49d513c 100644 --- a/proto/raft_serverpb.proto +++ b/proto/raft_serverpb.proto @@ -5,6 +5,7 @@ import "eraftpb.proto"; import "metapb.proto"; import "kvrpcpb.proto"; import "disk_usage.proto"; +import "encryptionpb.proto"; import "rustproto.proto"; option (rustproto.lite_runtime_all) = true; @@ -45,6 +46,19 @@ message SnapshotMeta { repeated SnapshotCFFile cf_files = 1; // true means this snapshot is triggered for load balance bool for_balance = 2; + // true means this is an empty snapshot for witness + bool for_witness = 3; + // the timestamp second to generate snapshot + uint64 start = 4; + // the duration of generating snapshot + uint64 generate_duration_sec = 5; + // the path of the tablet snapshot, it should only be used for v1 to receive + // snapshot from v2 + string tablet_snap_path = 6; + // A hint of the latest commit index on leader when sending snapshot. + // It should only be used for v2 to send snapshot to v1. + // See https://github.com/pingcap/tiflash/issues/7568 + uint64 commit_index_hint = 7; } message SnapshotChunk { @@ -54,6 +68,64 @@ message SnapshotChunk { message Done {} +message TabletSnapshotFileMeta { + uint64 file_size = 1; + string file_name = 2; + // Some block data. Unencrypted. + bytes head_chunk = 3; + // trailing data including checksum. Unencrypted. + bytes trailing_chunk = 4; +} + +// Snapshot preview for server to decide whether skip some files. +// Server should send back an `AcceptedSnapshotFile` to let client +// keep sending specified files. Only SST files can be skipped, all +// other files should always be sent. +message TabletSnapshotPreview { + repeated TabletSnapshotFileMeta metas = 1; + // There may be too many metas, use a flag to indicate all metas + // are sent. + bool end = 2; +} + +message TabletSnapshotFileChunk { + uint64 file_size = 1; + string file_name = 2; + // Encrypted. + bytes data = 3; + // Initial vector if encryption is enabled. + bytes iv = 4; + encryptionpb.DataKey key = 5; +} + +message TabletSnapshotHead { + RaftMessage message = 1; + bool use_cache = 2; +} + +message TabletSnapshotEnd { + // Checksum of all data sent in `TabletSnapshotFileChunk.data` and + // `TabletSnapshotFileChunk.file_name`. + uint64 checksum = 1; +} + +message TabletSnapshotRequest { + oneof payload { + TabletSnapshotHead head = 1; + TabletSnapshotPreview preview = 2; + TabletSnapshotFileChunk chunk = 3; + TabletSnapshotEnd end = 4; + } +} + +message AcceptedSnapshotFiles { + repeated string file_name = 1; +} + +message TabletSnapshotResponse { + AcceptedSnapshotFiles files = 1; +} + message KeyValue { bytes key = 1; bytes value = 2; @@ -65,6 +137,8 @@ message RaftSnapshotData { repeated KeyValue data = 3; uint64 version = 4; SnapshotMeta meta = 5; + repeated metapb.Peer removed_records = 6; + repeated MergedRecord merged_records = 7; } message StoreIdent { @@ -73,6 +147,15 @@ message StoreIdent { kvrpcpb.APIVersion api_version = 3; } +message StoreRecoverState { + // Used for TiKV start recovery when WAL of KVDB was disabled. + // TiKV may read all relations between seqno and raft log index, and replay + // all raft logs which corresponding seqno smaller than the seqno here. + // After TiKV replays all raft logs and flushed KV data, the seqno here must + // be updated. + uint64 seqno = 1; +} + message RaftLocalState { eraftpb.HardState hard_state = 1; uint64 last_index = 2; @@ -91,6 +174,11 @@ enum PeerState { Applying = 1; Tombstone = 2; Merging = 3; + // Currently used for witness to non-witness conversion: When a witness + // has just become a non-witness, we need to set and persist this state, + // so that when the service restarts before applying snapshot, we can + // actively request snapshot when initializing this peer. + Unavailable = 4; } message MergeState { @@ -99,10 +187,50 @@ message MergeState { uint64 commit = 3; } +message MergedRecord { + uint64 source_region_id = 1; + metapb.RegionEpoch source_epoch = 2; + // Peers of source region when merge is committed. + repeated metapb.Peer source_peers = 3; + // Removed peers (by confchange) of source region when merge is committed. + repeated metapb.Peer source_removed_records = 9; + uint64 target_region_id = 4; + metapb.RegionEpoch target_epoch = 5; + repeated metapb.Peer target_peers = 6; + // Commit merge index. + uint64 index = 7; + // Prepare merge index. + uint64 source_index = 8; +} + message RegionLocalState { PeerState state = 1; metapb.Region region = 2; MergeState merge_state = 3; + // The apply index corresponding to the storage when it's initialized. + uint64 tablet_index = 4; + // Raft doesn't guarantee peer will be removed in the end. In v1, peer finds + // out its destiny by logs or broadcast; in v2, leader is responsible to + // ensure removed peers are destroyed. + // Note: only peers who has been part of this region can be in this list. + repeated metapb.Peer removed_records = 5; + // Merged peer can't be deleted like gc peers. Instead, leader needs to + // query target peer to decide whether source peer can be destroyed. + repeated MergedRecord merged_records = 6; +} + +message RegionSequenceNumberRelation { + uint64 region_id = 1; + uint64 sequence_number = 2; + RaftApplyState apply_state = 3; + RegionLocalState region_state = 4; +} + +message AvailabilityContext { + uint64 from_region_id = 1; + metapb.RegionEpoch from_region_epoch = 2; + bool unavailable = 3; + bool trimmed = 4; } enum ExtraMessageType { @@ -114,10 +242,55 @@ enum ExtraMessageType { // to make sure they all agree to sleep. MsgHibernateRequest = 4; MsgHibernateResponse = 5; + MsgRejectRaftLogCausedByMemoryUsage = 6; + MsgAvailabilityRequest = 7; + MsgAvailabilityResponse = 8; + MsgVoterReplicatedIndexRequest = 9; + MsgVoterReplicatedIndexResponse = 10; + // Message means that `from` is tombstone. Leader can then update removed_records. + MsgGcPeerRequest = 11; + MsgGcPeerResponse = 12; + MsgFlushMemtable = 13; + MsgRefreshBuckets = 14; +} + +message FlushMemtable { + uint64 region_id = 1; +} + +message RefreshBuckets { + uint64 version = 1; + repeated bytes keys = 2; + repeated uint64 sizes = 3; +} + +message CheckGcPeer { + // The region ID who triggers the check and wait for report. It should be + // the ID of RaftMessage.from. + uint64 from_region_id = 1; + // The region ID to be checked if should be destroyed. + uint64 check_region_id = 2; + // The epoch of the region to be checked. + metapb.RegionEpoch check_region_epoch = 3; + // The peer to be checked. + metapb.Peer check_peer = 4; } message ExtraMessage { ExtraMessageType type = 1; - uint64 premerge_commit = 2; + // It's merge related index. In `WantRollbackMerge`, it's prepare merge index. In + // `MsgGcPeerRequest`, it's the commit merge index. In `MsgVoterReplicatedIndexRequest` + // it's the voter_replicated_index. + uint64 index = 2; + // In `MsgCheckStalePeerResponse`, it's the peers that receiver can continue to query. repeated metapb.Peer check_peers = 3; + bool wait_data = 4; + // Flag for forcely wake up hibernate regions if true. + bool forcely_awaken = 5; + CheckGcPeer check_gc_peer = 6; + FlushMemtable flush_memtable = 7; + // Used by `MsgAvailabilityRequest` and `MsgAvailabilityResponse` in v2. + AvailabilityContext availability_context = 8; + // notice the peer to refresh buckets version + RefreshBuckets refresh_buckets = 9; } diff --git a/proto/recoverdatapb.proto b/proto/recoverdatapb.proto new file mode 100644 index 0000000..de7a1b5 --- /dev/null +++ b/proto/recoverdatapb.proto @@ -0,0 +1,82 @@ +syntax = "proto3"; +package recover_data; + +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; + +// request to read region meata from a store +message ReadRegionMetaRequest { + uint64 store_id = 1; +} + +message Error { + string msg = 1; +} + +message RegionMeta { + uint64 region_id = 1; + uint64 peer_id = 2; + uint64 last_log_term = 3; + uint64 last_index = 4; + uint64 commit_index = 5; + uint64 version = 6; + bool tombstone = 7; //reserved, it may be used in late phase for peer check + bytes start_key = 8; + bytes end_key = 9; +} + +// command to store for recover region +message RecoverRegionRequest { + uint64 region_id = 1; + bool as_leader = 2; // force region_id as leader + bool tombstone = 3; // set Peer to tombstoned in late phase +} + +message RecoverRegionResponse { + Error error = 1; + uint64 store_id = 2; +} + +// wait apply to last index + message WaitApplyRequest { + uint64 store_id = 1; + } + + message WaitApplyResponse { + Error error = 1; + } + +// resolve data by resolved_ts +message ResolveKvDataRequest { + uint64 resolved_ts = 1; +} + +message ResolveKvDataResponse { + Error error = 1; + uint64 store_id = 2; + uint64 resolved_key_count = 3; // reserved for summary of restore + // cursor of delete key.commit_ts, reserved for progress of restore + // progress is (current_commit_ts - resolved_ts) / (backup_ts - resolved_ts) x 100% + uint64 current_commit_ts = 4; +} + +// a recovery workflow likes +// 1. BR read ReadRegionMeta to get all region meta +// 2. BR send recover region to tikv, e.g assign leader and wait leader apply to last index +// 3. BR wait all regions in tikv to apply to last index (no write during the recovery) +// 4. BR resolved kv data +service RecoverData { + // read region meta to ready region meta + rpc ReadRegionMeta(ReadRegionMetaRequest) returns (stream RegionMeta) {} + // execute the recovery command + rpc RecoverRegion(stream RecoverRegionRequest) returns (RecoverRegionResponse) {} + // wait all region apply to last index + rpc WaitApply(WaitApplyRequest) returns (WaitApplyResponse) {} + // execute delete data from kv db + rpc ResolveKvData(ResolveKvDataRequest) returns (stream ResolveKvDataResponse) {} +} \ No newline at end of file diff --git a/proto/replication_modepb.proto b/proto/replication_modepb.proto index 9bfb881..c60005b 100644 --- a/proto/replication_modepb.proto +++ b/proto/replication_modepb.proto @@ -17,10 +17,12 @@ message ReplicationStatus { enum DRAutoSyncState { // Raft logs need to sync between different DCs SYNC = 0; + // Wait for switching to ASYNC. Stop sync raft logs between DCs. + ASYNC_WAIT = 1; // Raft logs need to sync to majority peers - ASYNC = 1; + ASYNC = 2; // Switching from ASYNC to SYNC mode - SYNC_RECOVER = 2; + SYNC_RECOVER = 3; } // The status of dr-autosync mode. @@ -32,6 +34,10 @@ message DRAutoSync { uint64 state_id = 3; // Duration to wait before switching to SYNC by force (in seconds) int32 wait_sync_timeout_hint = 4; + // Stores should only sync messages with available stores when state is ASYNC or ASYNC_WAIT. + repeated uint64 available_stores = 5; + // Stores should forbid region split. + bool pause_region_split = 6; } enum RegionReplicationState { @@ -49,3 +55,8 @@ message RegionReplicationStatus { // Unique ID of the state, it increases after each state transfer. uint64 state_id = 2; } + +message StoreDRAutoSyncStatus { + DRAutoSyncState state = 1; + uint64 state_id = 2; +} diff --git a/proto/resource_manager.proto b/proto/resource_manager.proto new file mode 100644 index 0000000..aa7de74 --- /dev/null +++ b/proto/resource_manager.proto @@ -0,0 +1,233 @@ +syntax = "proto3"; +package resource_manager; + +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; + +service ResourceManager { + rpc ListResourceGroups(ListResourceGroupsRequest) returns (ListResourceGroupsResponse) {} + + rpc GetResourceGroup(GetResourceGroupRequest) returns (GetResourceGroupResponse) {} + + rpc AddResourceGroup(PutResourceGroupRequest) returns (PutResourceGroupResponse) {} + + rpc ModifyResourceGroup(PutResourceGroupRequest) returns (PutResourceGroupResponse) {} + + rpc DeleteResourceGroup(DeleteResourceGroupRequest) returns (DeleteResourceGroupResponse) {} + + rpc AcquireTokenBuckets(stream TokenBucketsRequest) returns (stream TokenBucketsResponse) {} +} + +message ListResourceGroupsRequest{} + +message ListResourceGroupsResponse{ + Error error = 1; + repeated ResourceGroup groups = 2; +} + +message GetResourceGroupRequest { + string resource_group_name = 1; +} + +message GetResourceGroupResponse{ + Error error = 1; + ResourceGroup group = 2; +} + +message DeleteResourceGroupRequest { + string resource_group_name = 1; +} + +message DeleteResourceGroupResponse{ + Error error = 1; + string body = 2; +} + +message PutResourceGroupRequest { + ResourceGroup group = 1; +} + +message PutResourceGroupResponse { + Error error = 1; + string body = 2; +} + +message TokenBucketsRequest { + repeated TokenBucketRequest requests = 1; + uint64 target_request_period_ms = 2; + uint64 client_unique_id = 3; +} + +message TokenBucketRequest { + message RequestRU { + repeated RequestUnitItem request_r_u = 1; + } + message RequestRawResource { + repeated RawResourceItem request_raw_resource = 1; + } + + string resource_group_name = 1; + oneof request { + // RU mode, group settings with WRU/RRU etc resource abstract unit. + RequestRU ru_items = 2; + // Raw mode, group settings with CPU/IO etc resource unit. + RequestRawResource raw_resource_items = 3; + } + // Aggregate statistics in group level. + Consumption consumption_since_last_request = 4; + // label background request. + bool is_background = 5; + bool is_tiflash = 6; +} + +message TokenBucketsResponse { + Error error = 1; + repeated TokenBucketResponse responses = 2; +} + +message TokenBucketResponse { + string resource_group_name = 1; + // RU mode + repeated GrantedRUTokenBucket granted_r_u_tokens = 2; + // Raw mode + repeated GrantedRawResourceTokenBucket granted_resource_tokens = 3; +} + +message GrantedRUTokenBucket { + RequestUnitType type = 1; + TokenBucket granted_tokens = 2; + int64 trickle_time_ms = 3; +} + +message GrantedRawResourceTokenBucket { + RawResourceType type = 1; + TokenBucket granted_tokens = 2; + int64 trickle_time_ms = 3; +} + +enum RequestUnitType { + RU = 0; +} + +enum RawResourceType { + CPU = 0; + IOReadFlow = 1; + IOWriteFlow = 2; +} + +message Consumption { + double r_r_u = 1; + double w_r_u = 2; + double read_bytes = 3; + double write_bytes = 4; + double total_cpu_time_ms = 5; + double sql_layer_cpu_time_ms = 6; + double kv_read_rpc_count = 7; + double kv_write_rpc_count = 8; +} + +message RequestUnitItem { + RequestUnitType type = 1; + double value = 2; +} + +message RawResourceItem { + RawResourceType type = 1; + double value = 2; +} + +enum GroupMode { + Unknown = 0; + RUMode = 1; + RawMode = 2; +} + +// ResourceGroup the settings definitions. +message ResourceGroup { + string name = 1; + GroupMode mode = 2; + // Used in RU mode, group settings with WRU/RRU etc resource abstract unit. + GroupRequestUnitSettings r_u_settings = 3; + // Used in Raw mode, group settings with CPU/IO etc resource unit. + GroupRawResourceSettings raw_resource_settings = 4; + // The task scheduling priority + uint32 priority = 5; + // Runaway queries settings + RunawaySettings runaway_settings = 6; + BackgroundSettings background_settings = 7; +} + +message GroupRequestUnitSettings { + TokenBucket r_u = 1; +} + +message GroupRawResourceSettings { + TokenBucket cpu = 1; + TokenBucket io_read = 2; + TokenBucket io_write = 3; +} + +message TokenBucket { + TokenLimitSettings settings = 1; + // Once used to reconfigure, the tokens is delta tokens. + double tokens = 2; +} + +message TokenLimitSettings { + uint64 fill_rate = 1; + int64 burst_limit = 2; + double max_tokens = 3; +} + +message Error { + string message = 1; +} + +enum RunawayAction { + NoneAction = 0; + DryRun = 1; // do nothing + CoolDown = 2; // deprioritize the task + Kill = 3; // kill the task +} + +enum RunawayWatchType { + NoneWatch = 0; + Exact = 1; + Similar = 2; + Plan = 3; +} + +message RunawayRule { + uint64 exec_elapsed_time_ms = 1; +} + +message RunawayWatch { + // how long would the watch last + int64 lasting_duration_ms = 1; + RunawayWatchType type = 2; +} + +message RunawaySettings { + RunawayRule rule = 1; + RunawayAction action = 2; + RunawayWatch watch = 3; +} + +message BackgroundSettings { + repeated string job_types = 1; +} + +message Participant { + // name is the unique name of the resource manager participant. + string name = 1; + // id is the unique id of the resource manager participant. + uint64 id = 2; + // listen_urls is the serivce endpoint list in the url format. + // listen_urls[0] is primary service endpoint. + repeated string listen_urls = 3; +} diff --git a/proto/resource_usage_agent.proto b/proto/resource_usage_agent.proto index e1a3093..4250ec2 100644 --- a/proto/resource_usage_agent.proto +++ b/proto/resource_usage_agent.proto @@ -14,49 +14,11 @@ option java_package = "org.tikv.kvproto"; // ResourceUsageAgent is the service for storing resource usage records. service ResourceUsageAgent { - // DEPRECATED: We now use `Report` to report not only CPU time. - // - // Report the CPU time records. By default, the records with the same - // resource group tag will be batched by minute. - rpc ReportCPUTime(stream CPUTimeRecord) returns (EmptyResponse) {} - // Report the resource usage records. By default, the records with the same // resource group tag will be batched by minute. rpc Report(stream ResourceUsageRecord) returns (EmptyResponse) {} } -message CPUTimeRecord { - bytes resource_group_tag = 1; - - // The following 2 repeated zipped together represents a List<(UnixTimestamp, CPUTime)> - - // UNIX timestamp in second. - repeated uint64 record_list_timestamp_sec = 2; - // The value can be greater than 1000ms if the requests are running parallelly. - repeated uint32 record_list_cpu_time_ms = 3; - -} - -message ResourceUsageRecord { - bytes resource_group_tag = 1; - - // The following repeated zipped together represents a List<(UnixTimestamp, Record)> - - // UNIX timestamp in second. - repeated uint64 record_list_timestamp_sec = 2; - - // The value can be greater than 1000ms if the requests are running parallelly. - repeated uint32 record_list_cpu_time_ms = 3; - - // The number of reads of keys associated with resource_group_tag. - repeated uint32 record_list_read_keys = 4; - - // The number of writes of keys associated with resource_group_tag. - repeated uint32 record_list_write_keys = 5; -} - -message EmptyResponse {} - // TiKV implements ResourceMeteringPubSub service for clients to subscribe to resource metering records. service ResourceMeteringPubSub { // Clients subscribe to resource metering records through this RPC, and TiKV periodically (e.g. per minute) @@ -65,3 +27,24 @@ service ResourceMeteringPubSub { } message ResourceMeteringRequest {} + +message EmptyResponse {} + +message ResourceUsageRecord { + oneof record_oneof { + GroupTagRecord record = 1; + } +} + +// GroupTagRecord is a set of resource usage data grouped by resource_group_tag. +message GroupTagRecord { + bytes resource_group_tag = 1; + repeated GroupTagRecordItem items = 2; +} + +message GroupTagRecordItem { + uint64 timestamp_sec = 1; + uint32 cpu_time_ms = 2; + uint32 read_keys = 3; + uint32 write_keys = 4; +} diff --git a/proto/schedulingpb.proto b/proto/schedulingpb.proto new file mode 100644 index 0000000..83b7de4 --- /dev/null +++ b/proto/schedulingpb.proto @@ -0,0 +1,197 @@ +syntax = "proto3"; +package schedulingpb; + +import "pdpb.proto"; +import "gogoproto/gogo.proto"; +import "rustproto.proto"; +import "metapb.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"; + +service Scheduling { + rpc StoreHeartbeat(StoreHeartbeatRequest) returns (StoreHeartbeatResponse) {} + + rpc RegionHeartbeat(stream RegionHeartbeatRequest) returns (stream RegionHeartbeatResponse) {} + + rpc SplitRegions(SplitRegionsRequest) returns (SplitRegionsResponse) {} + + rpc ScatterRegions(ScatterRegionsRequest) returns (ScatterRegionsResponse) {} + + rpc GetOperator(GetOperatorRequest) returns (GetOperatorResponse) {} + + rpc AskBatchSplit(AskBatchSplitRequest) returns (AskBatchSplitResponse) {} +} + +message RequestHeader { + // cluster_id is the ID of the cluster which be sent to. + uint64 cluster_id = 1; + // sender_id is the ID of the sender server. + uint64 sender_id = 2; +} + +message ResponseHeader { + // cluster_id is the ID of the cluster which sent the response. + uint64 cluster_id = 1; + Error error = 2; +} + +enum ErrorType { + OK = 0; + UNKNOWN = 1; + NOT_BOOTSTRAPPED = 2; + ALREADY_BOOTSTRAPPED = 3; + INVALID_VALUE = 4; + CLUSTER_MISMATCHED = 5; +} + +message Error { + ErrorType type = 1; + string message = 2; +} + +message Participant { + // name is the unique name of the scheduling participant. + string name = 1; + // id is the unique id of the scheduling participant. + uint64 id = 2; + // listen_urls is the serivce endpoint list in the url format. + // listen_urls[0] is primary service endpoint. + repeated string listen_urls = 3; +} + +message StoreHeartbeatRequest { + RequestHeader header = 1; + pdpb.StoreStats stats = 2; +} + +message StoreHeartbeatResponse { + ResponseHeader header = 1; + string cluster_version = 2; +} + +message RegionHeartbeatRequest { + RequestHeader header = 1; + + metapb.Region region = 2; + // Leader Peer sending the heartbeat. + metapb.Peer leader = 3; + // Term is the term of raft group. + uint64 term = 4; + // Leader considers that these peers are down. + repeated pdpb.PeerStats down_peers = 5; + // Pending peers are the peers that the leader can't consider as + // working followers. + repeated metapb.Peer pending_peers = 6; + // Bytes read/written during this period. + uint64 bytes_written = 7; + uint64 bytes_read = 8; + // Keys read/written during this period. + uint64 keys_written = 9; + uint64 keys_read = 10; + // Approximate region size. + uint64 approximate_size = 11; + // Approximate number of keys. + uint64 approximate_keys = 12; + // QueryStats reported write query stats, and there are read query stats in store heartbeat + pdpb.QueryStats query_stats = 13; + // Actually reported time interval + pdpb.TimeInterval interval = 14; +} + +message RegionHeartbeatResponse { + ResponseHeader header = 1; + + // ID of the region + uint64 region_id = 2; + metapb.RegionEpoch region_epoch = 3; + // Leader of the region at the moment of the corresponding request was made. + metapb.Peer target_peer = 4; + // Notice, Pd only allows handling reported epoch >= current pd's. + // Leader peer reports region status with RegionHeartbeatRequest + // to pd regularly, pd will determine whether this region + // should do ChangePeer or not. + // E,g, max peer number is 3, region A, first only peer 1 in A. + // 1. Pd region state -> Peers (1), ConfVer (1). + // 2. Leader peer 1 reports region state to pd, pd finds the + // peer number is < 3, so first changes its current region + // state -> Peers (1, 2), ConfVer (1), and returns ChangePeer Adding 2. + // 3. Leader does ChangePeer, then reports Peers (1, 2), ConfVer (2), + // pd updates its state -> Peers (1, 2), ConfVer (2). + // 4. Leader may report old Peers (1), ConfVer (1) to pd before ConfChange + // finished, pd stills responses ChangePeer Adding 2, of course, we must + // guarantee the second ChangePeer can't be applied in TiKV. + pdpb.ChangePeer change_peer = 5; + // Pd can return transfer_leader to let TiKV does leader transfer itself. + pdpb.TransferLeader transfer_leader = 6; + pdpb.Merge merge = 7; + // PD sends split_region to let TiKV split a region into two regions. + pdpb.SplitRegion split_region = 8; + // Multiple change peer operations atomically. + // Note: PD can use both ChangePeer and ChangePeerV2 at the same time + // (not in the same RegionHeartbeatResponse). + // Now, PD use ChangePeerV2 in following scenarios: + // 1. replacing peers + // 2. demoting voter directly + pdpb.ChangePeerV2 change_peer_v2 = 9; + pdpb.BatchSwitchWitness switch_witnesses = 10; +} + +message ScatterRegionsRequest { + RequestHeader header = 1; + // If group is defined, the regions with the same group would be scattered as a whole group. + // If not defined, the regions would be scattered in a cluster level. + string group = 2; + + // If regions_id is defined, the region_id would be ignored. + repeated uint64 regions_id = 3; + uint64 retry_limit = 4; + bool skip_store_limit = 5; +} + +message ScatterRegionsResponse { + ResponseHeader header = 1; + uint64 finished_percentage = 2; +} + +message SplitRegionsRequest { + RequestHeader header = 1; + repeated bytes split_keys = 2; + uint64 retry_limit = 3; +} + +message SplitRegionsResponse { + ResponseHeader header = 1; + uint64 finished_percentage = 2; + repeated uint64 regions_id = 3; +} + +message GetOperatorRequest { + RequestHeader header = 1; + uint64 region_id = 2; +} + +message GetOperatorResponse { + ResponseHeader header = 1; + uint64 region_id = 2; + bytes desc = 3; + pdpb.OperatorStatus status = 4; + bytes kind = 5; +} + +message AskBatchSplitRequest { + RequestHeader header = 1; + + metapb.Region region = 2; + uint32 split_count = 3; +} + +message AskBatchSplitResponse { + ResponseHeader header = 1; + + repeated pdpb.SplitID ids = 2; +} diff --git a/proto/span.proto b/proto/span.proto deleted file mode 100644 index 16bfbbc..0000000 --- a/proto/span.proto +++ /dev/null @@ -1,29 +0,0 @@ -syntax = "proto3"; -package span; - -message SpanSet { - uint64 start_time_ns = 1; - uint64 cycles_per_sec = 2; - repeated Span spans = 3; - uint64 create_time_ns = 4; -} - -message Root {} -message Parent { uint64 id = 1; } -message Continue { uint64 id = 1; } - -message Link { - oneof link { - Root root = 1; - Parent parent = 2; - Continue continue = 3; - } -} - -message Span { - uint64 id = 1; - Link link = 2; - uint64 begin_cycles = 3; - uint64 end_cycles = 4; - uint32 event = 5; -} diff --git a/proto/tikvpb.proto b/proto/tikvpb.proto index c3fd6ad..f48e9d1 100644 --- a/proto/tikvpb.proto +++ b/proto/tikvpb.proto @@ -5,6 +5,7 @@ import "coprocessor.proto"; import "kvrpcpb.proto"; import "mpp.proto"; import "raft_serverpb.proto"; +import "disaggregated.proto"; import "gogoproto/gogo.proto"; import "rustproto.proto"; @@ -36,6 +37,8 @@ service Tikv { rpc KvResolveLock(kvrpcpb.ResolveLockRequest) returns (kvrpcpb.ResolveLockResponse) {} rpc KvGC(kvrpcpb.GCRequest) returns (kvrpcpb.GCResponse) {} rpc KvDeleteRange(kvrpcpb.DeleteRangeRequest) returns (kvrpcpb.DeleteRangeResponse) {} + rpc KvPrepareFlashbackToVersion(kvrpcpb.PrepareFlashbackToVersionRequest) returns (kvrpcpb.PrepareFlashbackToVersionResponse) {} + rpc KvFlashbackToVersion(kvrpcpb.FlashbackToVersionRequest) returns (kvrpcpb.FlashbackToVersionResponse) {} // Raw commands; no transaction support. rpc RawGet(kvrpcpb.RawGetRequest) returns (kvrpcpb.RawGetResponse) {} @@ -73,6 +76,7 @@ service Tikv { rpc Raft(stream raft_serverpb.RaftMessage) returns (raft_serverpb.Done) {} rpc BatchRaft(stream BatchRaftMessage) returns (raft_serverpb.Done) {} rpc Snapshot(stream raft_serverpb.SnapshotChunk) returns (raft_serverpb.Done) {} + rpc TabletSnapshot(stream raft_serverpb.TabletSnapshotRequest) returns (stream raft_serverpb.TabletSnapshotResponse) {} // Sent from PD or TiDB to a TiKV node. rpc SplitRegion (kvrpcpb.SplitRegionRequest) returns (kvrpcpb.SplitRegionResponse) {} @@ -91,6 +95,7 @@ service Tikv { rpc CancelMPPTask(mpp.CancelTaskRequest) returns (mpp.CancelTaskResponse) {} rpc EstablishMPPConnection(mpp.EstablishMPPConnectionRequest) returns (stream mpp.MPPDataPacket) {} rpc IsAlive(mpp.IsAliveRequest) returns (mpp.IsAliveResponse) {} + rpc ReportMPPTaskStatus(mpp.ReportTaskStatusRequest) returns (mpp.ReportTaskStatusResponse) {} /// CheckLeader sends all information (includes region term and epoch) to other stores. /// Once a store receives a request, it checks term and epoch for each region, and sends the regions whose @@ -104,6 +109,31 @@ service Tikv { /// Get the information about lock waiting from TiKV. rpc GetLockWaitInfo(kvrpcpb.GetLockWaitInfoRequest) returns (kvrpcpb.GetLockWaitInfoResponse); + + /// Compact a specified key range. This request is not restricted to raft leaders and will not be replicated. + /// It only compacts data on this node. + /// TODO: Currently this RPC is designed to be only compatible with TiFlash. + /// Shall be move out in https://github.com/pingcap/kvproto/issues/912 + rpc Compact(kvrpcpb.CompactRequest) returns (kvrpcpb.CompactResponse); + /// Get the information about history lock waiting from TiKV. + rpc GetLockWaitHistory(kvrpcpb.GetLockWaitHistoryRequest) returns (kvrpcpb.GetLockWaitHistoryResponse); + + /// Get system table from TiFlash + rpc GetTiFlashSystemTable(kvrpcpb.TiFlashSystemTableRequest) returns (kvrpcpb.TiFlashSystemTableResponse) {} + + // These are for TiFlash disaggregated architecture + /// Try to lock a S3 object, atomically + rpc tryAddLock(disaggregated.TryAddLockRequest) returns (disaggregated.TryAddLockResponse) {}; + /// Try to delete a S3 object, atomically + rpc tryMarkDelete(disaggregated.TryMarkDeleteRequest) returns (disaggregated.TryMarkDeleteResponse) {}; + /// Build the disaggregated task on TiFlash write node + rpc EstablishDisaggTask(disaggregated.EstablishDisaggTaskRequest) returns (disaggregated.EstablishDisaggTaskResponse) {} + /// Cancel the disaggregated task on TiFlash write node + rpc CancelDisaggTask(disaggregated.CancelDisaggTaskRequest) returns (disaggregated.CancelDisaggTaskResponse) {} + /// Exchange page data between TiFlash write node and compute node + rpc FetchDisaggPages(disaggregated.FetchDisaggPagesRequest) returns (stream disaggregated.PagesPacket) {} + /// Compute node get configuration from Write node + rpc GetDisaggConfig(disaggregated.GetDisaggConfigRequest) returns (disaggregated.GetDisaggConfigResponse) {} } message BatchCommandsRequest { @@ -146,6 +176,9 @@ message BatchCommandsRequest { kvrpcpb.RawCoprocessorRequest RawCoprocessor = 34; + kvrpcpb.FlashbackToVersionRequest FlashbackToVersion = 35; + kvrpcpb.PrepareFlashbackToVersionRequest PrepareFlashbackToVersion = 36; + // For some test cases. BatchCommandsEmptyRequest Empty = 255; } @@ -197,6 +230,9 @@ message BatchCommandsResponse { kvrpcpb.RawCoprocessorResponse RawCoprocessor = 34; + kvrpcpb.FlashbackToVersionResponse FlashbackToVersion = 35; + kvrpcpb.PrepareFlashbackToVersionResponse PrepareFlashbackToVersion = 36; + // For some test cases. BatchCommandsEmptyResponse Empty = 255; } diff --git a/proto/tracepb.proto b/proto/tracepb.proto new file mode 100644 index 0000000..c229454 --- /dev/null +++ b/proto/tracepb.proto @@ -0,0 +1,60 @@ +syntax = "proto3"; +package tracepb; + +option java_package = "org.tikv.kvproto"; + +service TraceRecordPubSub { + // Subscribe the Trace records generated on this service. The service will periodically (e.g. per minute) + // publishes Trace records to clients via gRPC stream. + rpc Subscribe(TraceRecordRequest) returns (stream TraceRecord) {} +} + +message TraceRecordRequest {} + +message TraceRecord { + oneof record_oneof { + Report report = 1; + NotifyCollect notify_collect = 2; + } +} + +message RemoteParentSpan { + // A unique id to identify the request. It's usually a UUID. + uint64 trace_id = 1; + // The span of remote caller that is awaiting the request. + uint64 span_id = 2; +} + +// The context of the request to be traced. +message TraceContext { + repeated RemoteParentSpan remote_parent_spans = 1; + // Report the trace records only if the duration of handling the request exceeds the threshold. + uint32 duration_threshold_ms = 2; +} + +// Report the spans collected when handling a request on a service. +message Report { + repeated RemoteParentSpan remote_parent_spans = 1; + repeated Span spans = 2; +} + +// Notify the subscriber to persis the spans of the trace. +message NotifyCollect { + uint64 trace_id = 1; +} + +message Span { + // The unique span id within the spans with the same `trace_id`. + // The most significant 32 bits should be random number generated by each service instance. + uint64 span_id = 1; + uint64 parent_id = 2; + uint64 begin_unix_ns = 3; + uint64 duration_ns = 4; + string event = 5; + repeated Property properties = 6; +} + +message Property { + string key = 1; + string value = 2; +} diff --git a/proto/tsopb.proto b/proto/tsopb.proto new file mode 100644 index 0000000..061bcad --- /dev/null +++ b/proto/tsopb.proto @@ -0,0 +1,126 @@ +syntax = "proto3"; +package tsopb; + +import "pdpb.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"; + +service TSO { + rpc Tso(stream TsoRequest) returns (stream TsoResponse) {} + // Find the keyspace group that the keyspace belongs to by keyspace id. + rpc FindGroupByKeyspaceID (FindGroupByKeyspaceIDRequest) returns (FindGroupByKeyspaceIDResponse) {} + // Get the minimum timestamp across all keyspace groups served by the TSO server who receives + // and handle the request. If the TSO server/pod is not serving any keyspace group, return + // an empty timestamp, and the client needs to skip the empty timestamps when collecting + // the min timestamp from all TSO servers/pods. + rpc GetMinTS (GetMinTSRequest) returns (GetMinTSResponse) {} +} + +message RequestHeader { + // cluster_id is the ID of the cluster which be sent to. + uint64 cluster_id = 1; + // sender_id is the ID of the sender server. + uint64 sender_id = 2; + + // keyspace_id is the unique id of the tenant/keyspace. + uint32 keyspace_id = 3; + // keyspace_group_id is the unique id of the keyspace group to which the tenant/keyspace belongs. + uint32 keyspace_group_id = 4; +} + +message ResponseHeader { + // cluster_id is the ID of the cluster which sent the response. + uint64 cluster_id = 1; + Error error = 2; + + // keyspace_id is the unique id of the tenant/keyspace as the response receiver. + uint32 keyspace_id = 3; + // keyspace_group_id is the unique id of the keyspace group to which the tenant/keyspace belongs. + uint32 keyspace_group_id = 4; +} + +enum ErrorType { + OK = 0; + UNKNOWN = 1; + NOT_BOOTSTRAPPED = 2; + ALREADY_BOOTSTRAPPED = 3; + INVALID_VALUE = 4; + CLUSTER_MISMATCHED = 5; +} + +message Error { + ErrorType type = 1; + string message = 2; +} + +message TsoRequest { + RequestHeader header = 1; + + uint32 count = 2; + string dc_location = 3; +} + +message TsoResponse { + ResponseHeader header = 1; + + uint32 count = 2; + pdpb.Timestamp timestamp = 3; +} + +message Participant { + // name is the unique name of the TSO participant. + string name = 1; + // id is the unique id of the TSO participant. + uint64 id = 2; + // listen_urls is the serivce endpoint list in the url format. + // listen_urls[0] is primary service endpoint. + repeated string listen_urls = 3; +} + +message KeyspaceGroupMember { + string address = 1; + bool is_primary = 2; +} + +message SplitState { + uint32 split_source = 1; +} + +message KeyspaceGroup { + uint32 id = 1; + string user_kind = 2; + SplitState split_state = 3; + repeated KeyspaceGroupMember members = 4; +} + +message FindGroupByKeyspaceIDRequest { + RequestHeader header = 1; + uint32 keyspace_id = 2; +} + +message FindGroupByKeyspaceIDResponse { + ResponseHeader header = 1; + KeyspaceGroup keyspace_group = 2; +} + +message GetMinTSRequest { + RequestHeader header = 1; + string dc_location = 2; +} + +message GetMinTSResponse { + ResponseHeader header = 1; + pdpb.Timestamp timestamp = 2; + // the count of keyspace group primaries that the TSO server/pod is serving + uint32 keyspace_groups_serving = 3; + // the total count of keyspace groups + uint32 keyspace_groups_total = 4; +} diff --git a/src/common/errors.rs b/src/common/errors.rs index 29622c9..5e7f630 100644 --- a/src/common/errors.rs +++ b/src/common/errors.rs @@ -50,6 +50,7 @@ pub enum Error { /// Wraps a `grpcio::Error`. #[error("gRPC error: {0}")] Grpc(#[from] tonic::transport::Error), + /// Wraps a `reqwest::Error`. /// Wraps a `grpcio::Error`. #[error("gRPC api error: {0}")] GrpcAPI(#[from] tonic::Status), @@ -108,6 +109,8 @@ pub enum Error { inner: Box, success_keys: Vec>, }, + #[error("Keyspace not found: {0}")] + KeyspaceNotFound(String), } impl From for Error { diff --git a/src/common/security.rs b/src/common/security.rs index 483759c..89e074b 100644 --- a/src/common/security.rs +++ b/src/common/security.rs @@ -8,10 +8,10 @@ use std::time::Duration; use log::info; use regex::Regex; -use tonic::transport::Certificate; use tonic::transport::Channel; use tonic::transport::ClientTlsConfig; use tonic::transport::Identity; +use tonic::transport::{Certificate, Endpoint}; use crate::internal_err; use crate::Result; @@ -77,28 +77,41 @@ impl SecurityManager { where Factory: FnOnce(Channel) -> Client, { - let addr = "http://".to_string() + &SCHEME_REG.replace(addr, ""); - info!("connect to rpc server at endpoint: {:?}", addr); - - let mut builder = Channel::from_shared(addr)? - .tcp_keepalive(Some(Duration::from_secs(10))) - .keep_alive_timeout(Duration::from_secs(3)); - - if !self.ca.is_empty() { - let tls = ClientTlsConfig::new() - .ca_certificate(Certificate::from_pem(&self.ca)) - .identity(Identity::from_pem( - &self.cert, - load_pem_file("private key", &self.key)?, - )); - builder = builder.tls_config(tls)?; + let channel = if !self.ca.is_empty() { + self.tls_channel(addr).await? + } else { + self.default_channel(addr).await? }; - - let ch = builder.connect().await?; + let ch = channel.connect().await?; Ok(factory(ch)) } + + async fn tls_channel(&self, addr: &str) -> Result { + let addr = "https://".to_string() + &SCHEME_REG.replace(addr, ""); + let builder = self.endpoint(addr.to_string())?; + let tls = ClientTlsConfig::new() + .ca_certificate(Certificate::from_pem(&self.ca)) + .identity(Identity::from_pem( + &self.cert, + load_pem_file("private key", &self.key)?, + )); + let builder = builder.tls_config(tls)?; + Ok(builder) + } + + async fn default_channel(&self, addr: &str) -> Result { + let addr = "http://".to_string() + &SCHEME_REG.replace(addr, ""); + self.endpoint(addr) + } + + fn endpoint(&self, addr: String) -> Result { + let endpoint = Channel::from_shared(addr)? + .tcp_keepalive(Some(Duration::from_secs(10))) + .keep_alive_timeout(Duration::from_secs(3)); + Ok(endpoint) + } } #[cfg(test)] diff --git a/src/config.rs b/src/config.rs index 1be273c..79dad85 100644 --- a/src/config.rs +++ b/src/config.rs @@ -19,6 +19,7 @@ pub struct Config { pub cert_path: Option, pub key_path: Option, pub timeout: Duration, + pub keyspace: Option, } const DEFAULT_REQUEST_TIMEOUT: Duration = Duration::from_secs(2); @@ -30,6 +31,7 @@ impl Default for Config { cert_path: None, key_path: None, timeout: DEFAULT_REQUEST_TIMEOUT, + keyspace: None, } } } @@ -83,4 +85,21 @@ impl Config { self.timeout = timeout; self } + + /// Set to use default keyspace. + /// + /// Server should enable `storage.api-version = 2` to use this feature. + #[must_use] + pub fn with_default_keyspace(self) -> Self { + self.with_keyspace("DEFAULT") + } + + /// Set the use keyspace for the client. + /// + /// Server should enable `storage.api-version = 2` to use this feature. + #[must_use] + pub fn with_keyspace(mut self, keyspace: &str) -> Self { + self.keyspace = Some(keyspace.to_owned()); + self + } } diff --git a/src/generated/autoid.rs b/src/generated/autoid.rs new file mode 100644 index 0000000..503024a --- /dev/null +++ b/src/generated/autoid.rs @@ -0,0 +1,179 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AutoIdRequest { + #[prost(int64, tag = "1")] + pub db_id: i64, + #[prost(int64, tag = "2")] + pub tbl_id: i64, + #[prost(bool, tag = "3")] + pub is_unsigned: bool, + #[prost(uint64, tag = "4")] + pub n: u64, + #[prost(int64, tag = "5")] + pub increment: i64, + #[prost(int64, tag = "6")] + pub offset: i64, + #[prost(uint32, tag = "7")] + pub keyspace_id: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AutoIdResponse { + #[prost(int64, tag = "1")] + pub min: i64, + #[prost(int64, tag = "2")] + pub max: i64, + #[prost(bytes = "vec", tag = "3")] + pub errmsg: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RebaseRequest { + #[prost(int64, tag = "1")] + pub db_id: i64, + #[prost(int64, tag = "2")] + pub tbl_id: i64, + #[prost(bool, tag = "3")] + pub is_unsigned: bool, + #[prost(int64, tag = "4")] + pub base: i64, + #[prost(bool, tag = "5")] + pub force: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RebaseResponse { + #[prost(bytes = "vec", tag = "1")] + pub errmsg: ::prost::alloc::vec::Vec, +} +/// Generated client implementations. +pub mod auto_id_alloc_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct AutoIdAllocClient { + inner: tonic::client::Grpc, + } + impl AutoIdAllocClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl AutoIdAllocClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> AutoIdAllocClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + AutoIdAllocClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn alloc_auto_id( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/autoid.AutoIDAlloc/AllocAutoID", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("autoid.AutoIDAlloc", "AllocAutoID")); + self.inner.unary(req, path, codec).await + } + pub async fn rebase( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/autoid.AutoIDAlloc/Rebase", + ); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("autoid.AutoIDAlloc", "Rebase")); + self.inner.unary(req, path, codec).await + } + } +} diff --git a/src/generated/backup.rs b/src/generated/backup.rs index fad59c4..a283075 100644 --- a/src/generated/backup.rs +++ b/src/generated/backup.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// The message save the metadata of a backup. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -64,6 +65,36 @@ pub struct BackupMeta { /// API version implies the encode of the key and value. #[prost(enumeration = "super::kvrpcpb::ApiVersion", tag = "18")] pub api_version: i32, + /// the placement policy info in backup cluster. we assume the policy won't be too much for one cluster. + #[prost(message, repeated, tag = "19")] + pub policies: ::prost::alloc::vec::Vec, + /// new_collations_enabled specifies the config `new_collations_enabled_on_first_bootstrap` in tidb. + #[prost(string, tag = "20")] + pub new_collations_enabled: ::prost::alloc::string::String, + /// If in txn kv mode, `schemas` will be ignored, the backup data's range is as same as normal backup. + #[prost(bool, tag = "21")] + pub is_txn_kv: bool, + /// maintain the id mapping from upstream cluster to downstream cluster. + #[prost(message, repeated, tag = "22")] + pub db_maps: ::prost::alloc::vec::Vec, + #[prost(enumeration = "BackupMode", tag = "23")] + pub mode: i32, + /// record the backup range and the correspond SST files when using file-copy backup. + #[prost(message, repeated, tag = "24")] + pub ranges: ::prost::alloc::vec::Vec, + /// record the size of the backup data files and meta files + #[prost(uint64, tag = "25")] + pub backup_size: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct BackupRange { + #[prost(bytes = "vec", tag = "1")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "2")] + pub end_key: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub files: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -111,12 +142,21 @@ pub struct MetaFile { /// A set of files that contains RawRanges. #[prost(message, repeated, tag = "4")] pub raw_ranges: ::prost::alloc::vec::Vec, + /// A set of files that contains BackupRanges. + #[prost(message, repeated, tag = "6")] + pub backup_ranges: ::prost::alloc::vec::Vec, /// A set of files that contains DDLs. #[prost(bytes = "vec", repeated, tag = "5")] pub ddls: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct PlacementPolicy { + #[prost(bytes = "vec", tag = "1")] + pub info: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct Schema { #[prost(bytes = "vec", tag = "1")] pub db: ::prost::alloc::vec::Vec, @@ -137,6 +177,34 @@ pub struct Schema { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct IdMap { + #[prost(int64, tag = "1")] + pub upstream_id: i64, + #[prost(int64, tag = "2")] + pub downstream_id: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PitrTableMap { + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + #[prost(message, optional, tag = "2")] + pub id_map: ::core::option::Option, + #[prost(message, repeated, tag = "3")] + pub partitions: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PitrDbMap { + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + #[prost(message, optional, tag = "2")] + pub id_map: ::core::option::Option, + #[prost(message, repeated, tag = "3")] + pub tables: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct RawRange { #[prost(bytes = "vec", tag = "1")] pub start_key: ::prost::alloc::vec::Vec, @@ -218,6 +286,56 @@ pub struct BackupRequest { /// The cipher_info is Used to encrypt sst #[prost(message, optional, tag = "14")] pub cipher_info: ::core::option::Option, + /// 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. + /// 1. "v2": the generated SST files are encoded with api-v2, can be restored + /// to TiKV clusters whose api version is set to v2. + #[prost(enumeration = "super::kvrpcpb::ApiVersion", tag = "15")] + pub dst_api_version: i32, + /// 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. + #[prost(message, repeated, tag = "16")] + pub sub_ranges: ::prost::alloc::vec::Vec, + /// 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. + #[prost(bool, tag = "17")] + pub replica_read: bool, + #[prost(enumeration = "BackupMode", tag = "18")] + pub mode: i32, + /// 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. + #[prost(string, tag = "19")] + pub unique_id: ::prost::alloc::string::String, + #[prost(message, optional, tag = "20")] + pub context: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StreamBackupTaskInfo { + /// The storage for backup, parsed by BR. + #[prost(message, optional, tag = "1")] + pub storage: ::core::option::Option, + /// The time range for backing up. + #[prost(uint64, tag = "2")] + pub start_ts: u64, + #[prost(uint64, tag = "3")] + pub end_ts: u64, + /// Misc meta datas. + /// The name of the task, also the ID of the task. + #[prost(string, tag = "4")] + pub name: ::prost::alloc::string::String, + /// The table filter of the task. + /// Only for displaying the task info. + #[prost(string, repeated, tag = "5")] + pub table_filter: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + /// compression type + #[prost(enumeration = "CompressionType", tag = "6")] + pub compression_type: i32, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -285,6 +403,18 @@ pub struct S3 { pub force_path_style: bool, #[prost(string, tag = "11")] pub sse_kms_key_id: ::prost::alloc::string::String, + #[prost(string, tag = "12")] + pub role_arn: ::prost::alloc::string::String, + #[prost(string, tag = "13")] + pub external_id: ::prost::alloc::string::String, + #[prost(bool, tag = "14")] + pub object_lock_enabled: bool, + #[prost(string, tag = "15")] + pub session_token: ::prost::alloc::string::String, + #[prost(string, tag = "16")] + pub provider: ::prost::alloc::string::String, + #[prost(string, tag = "17")] + pub profile: ::prost::alloc::string::String, } /// GCS storage backend saves files into google cloud storage. #[allow(clippy::derive_partial_eq_without_eq)] @@ -309,6 +439,17 @@ pub struct Gcs { #[prost(string, tag = "6")] pub credentials_blob: ::prost::alloc::string::String, } +/// The encryption algorithm must be AES256. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AzureCustomerKey { + /// A Base64-encoded AES-256 encryption key value. + #[prost(string, tag = "1")] + pub encryption_key: ::prost::alloc::string::String, + /// The Base64-encoded SHA256 of the encryption key. + #[prost(string, tag = "2")] + pub encryption_key_sha256: ::prost::alloc::string::String, +} /// AzureBlobStorage storage backend saves files into azure blob storage. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -328,15 +469,43 @@ pub struct AzureBlobStorage { /// if empty, try to read account_name from the node's environment variable $AZURE_STORAGE_ACCOUNT. #[prost(string, tag = "5")] pub account_name: ::prost::alloc::string::String, - /// 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. + /// There are 3 kinds of credentials, and the priority order is + /// `SAS > Shared key > Azure AD (env) > Shared key (env)`. /// - /// See + /// 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 /// /// Otherwise, if empty, try to read shared key from the node's environment variable $AZURE_STORAGE_KEY. #[prost(string, tag = "6")] pub shared_key: ::prost::alloc::string::String, + /// 2. Use Azure AD (Azure Active Directory) to access the azure blob + /// + /// See + /// + /// 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 + #[prost(string, tag = "8")] + pub access_sig: ::prost::alloc::string::String, + /// Server Side Encryption, 2 types in total: + /// + /// 1. Specify an encryption scope for uploaded blobs. + /// + /// See + #[prost(string, tag = "9")] + pub encryption_scope: ::prost::alloc::string::String, + /// 2. Provide an encryption key on a request to blob storage. + /// + /// See + #[prost(message, optional, tag = "10")] + pub encryption_key: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -386,6 +555,60 @@ pub struct BackupResponse { pub end_key: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "4")] pub files: ::prost::alloc::vec::Vec, + /// API version implies the encode of the key and value. + #[prost(enumeration = "super::kvrpcpb::ApiVersion", tag = "5")] + pub api_version: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CleanupRequest { + /// unique_id represents the unique handle of the whole backup predecure. + /// it generated in prepare request and corrosponed to one specific backup. + #[prost(string, tag = "1")] + pub unique_id: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CleanupResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(bool, tag = "2")] + pub success: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PrepareRequest { + /// whether save state to the storage. + #[prost(bool, tag = "1")] + pub save_to_storage: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PrepareResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + /// 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. + #[prost(string, tag = "2")] + pub unique_id: ::prost::alloc::string::String, + #[prost(uint64, tag = "3")] + pub collect_file_count: u64, + #[prost(uint64, tag = "4")] + pub collect_file_size: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CheckAdminRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CheckAdminResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub region: ::core::option::Option, + #[prost(bool, tag = "3")] + pub has_pending_admin: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -415,7 +638,125 @@ pub struct ExternalStorageSaveRequest { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ExternalStorageSaveResponse {} -/// sst files compression algorithm +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Metadata { + /// deprecated after(in) v6.3.0 TiKV cluster + #[prost(message, repeated, tag = "1")] + pub files: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "6")] + pub file_groups: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "2")] + pub store_id: i64, + #[prost(uint64, tag = "3")] + pub resolved_ts: u64, + #[prost(uint64, tag = "4")] + pub max_ts: u64, + #[prost(uint64, tag = "5")] + pub min_ts: u64, + #[prost(enumeration = "MetaVersion", tag = "7")] + pub meta_version: i32, +} +/// DataFileGroup is the merged file info in log-backup +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DataFileGroup { + /// Path of the file. + #[prost(string, tag = "1")] + pub path: ::prost::alloc::string::String, + /// Partitions of the file. + #[prost(message, repeated, tag = "2")] + pub data_files_info: ::prost::alloc::vec::Vec, + /// / Below are extra information of the file, for better filtering files. + /// The min ts of the keys in the file. + #[prost(uint64, tag = "3")] + pub min_ts: u64, + /// The max ts of the keys in the file. + #[prost(uint64, tag = "4")] + pub max_ts: u64, + /// The resolved ts of the region when saving the file. + #[prost(uint64, tag = "5")] + pub min_resolved_ts: u64, + /// The file length after compressed. + #[prost(uint64, tag = "6")] + pub length: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DataFileInfo { + /// SHA256 of the file. + #[prost(bytes = "vec", tag = "1")] + pub sha256: ::prost::alloc::vec::Vec, + /// Path of the file. + #[prost(string, tag = "2")] + pub path: ::prost::alloc::string::String, + #[prost(int64, tag = "3")] + pub number_of_entries: i64, + /// / Below are extra information of the file, for better filtering files. + /// The min ts of the keys in the file. + #[prost(uint64, tag = "4")] + pub min_ts: u64, + /// The max ts of the keys in the file. + #[prost(uint64, tag = "5")] + pub max_ts: u64, + /// The resolved ts of the region when saving the file. + #[prost(uint64, tag = "6")] + pub resolved_ts: u64, + /// The region of the file. + #[prost(int64, tag = "7")] + pub region_id: i64, + /// The key range of the file. + /// Encoded and starts with 'z'(internal key). + #[prost(bytes = "vec", tag = "8")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "9")] + pub end_key: ::prost::alloc::vec::Vec, + /// The column family of the file. + #[prost(string, tag = "10")] + pub cf: ::prost::alloc::string::String, + /// The operation type of the file. + #[prost(enumeration = "FileType", tag = "11")] + pub r#type: i32, + /// Whether the data file contains meta keys(m prefixed keys) only. + #[prost(bool, tag = "12")] + pub is_meta: bool, + /// The table ID of the file contains, when `is_meta` is true, would be ignored. + #[prost(int64, tag = "13")] + pub table_id: i64, + /// The file length. + #[prost(uint64, tag = "14")] + pub length: u64, + /// The minimal begin ts in default cf if this file is write cf. + #[prost(uint64, tag = "15")] + pub min_begin_ts_in_default_cf: u64, + /// Offset of the partition. compatible with V1 and V2. + #[prost(uint64, tag = "16")] + pub range_offset: u64, + /// The range length of the merged file, if it exists. + #[prost(uint64, tag = "17")] + pub range_length: u64, + /// The compression type for the file. + #[prost(enumeration = "CompressionType", tag = "18")] + pub compression_type: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StreamBackupError { + /// the unix epoch time (in millisecs) of the time the error reported. + #[prost(uint64, tag = "1")] + pub happen_at: u64, + /// the unified error code of the error. + #[prost(string, tag = "2")] + pub error_code: ::prost::alloc::string::String, + /// the user-friendly error message. + #[prost(string, tag = "3")] + pub error_message: ::prost::alloc::string::String, + /// the store id of who issues the error. + #[prost(uint64, tag = "4")] + pub store_id: u64, +} +/// sst files or log files compression algorithm +/// for log files, unknown means not use compression algorithm #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum CompressionType { @@ -448,6 +789,92 @@ impl CompressionType { } } } +/// BackupMpde represents the mode of this whole backup request to the cluster. +/// and we need to store it in `backupmeta`. +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub 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, +} +impl BackupMode { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + BackupMode::Scan => "SCAN", + BackupMode::File => "FILE", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "SCAN" => Some(Self::Scan), + "FILE" => Some(Self::File), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum MetaVersion { + V1 = 0, + V2 = 1, +} +impl MetaVersion { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + MetaVersion::V1 => "V1", + MetaVersion::V2 => "V2", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "V1" => Some(Self::V1), + "V2" => Some(Self::V2), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum FileType { + Delete = 0, + Put = 1, +} +impl FileType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + FileType::Delete => "Delete", + FileType::Put => "Put", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "Delete" => Some(Self::Delete), + "Put" => Some(Self::Put), + _ => None, + } + } +} /// Generated client implementations. pub mod backup_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] @@ -555,6 +982,82 @@ pub mod backup_client { req.extensions_mut().insert(GrpcMethod::new("backup.Backup", "backup")); self.inner.server_streaming(req, path, codec).await } + /// 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. + pub async fn check_pending_admin_op( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/backup.Backup/CheckPendingAdminOp", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("backup.Backup", "CheckPendingAdminOp")); + self.inner.server_streaming(req, path, codec).await + } + /// 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. + pub async fn prepare( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/backup.Backup/prepare"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("backup.Backup", "prepare")); + self.inner.unary(req, path, codec).await + } + /// 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 + pub async fn cleanup( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/backup.Backup/cleanup"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("backup.Backup", "cleanup")); + self.inner.unary(req, path, codec).await + } } } /// Generated client implementations. diff --git a/src/generated/cdcpb.rs b/src/generated/cdcpb.rs index 0cf47c5..e80572e 100644 --- a/src/generated/cdcpb.rs +++ b/src/generated/cdcpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Header { @@ -45,6 +46,8 @@ pub struct Error { pub compatibility: ::core::option::Option, #[prost(message, optional, tag = "6")] pub cluster_id_mismatch: ::core::option::Option, + #[prost(message, optional, tag = "7")] + pub server_is_busy: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -97,6 +100,14 @@ pub mod event { pub value: ::prost::alloc::vec::Vec, #[prost(bytes = "vec", tag = "7")] pub old_value: ::prost::alloc::vec::Vec, + /// expire_ts_unix_secs is used for RawKV (see `ChangeDataRequest.KvApi`), + /// and represents the expiration time of this row. + /// Absolute time, seconds since Unix epoch. + #[prost(uint64, tag = "8")] + pub expire_ts_unix_secs: u64, + /// The source of this row. + #[prost(uint64, tag = "9")] + pub txn_source: u64, } /// Nested message and enum types in `Row`. pub mod row { @@ -230,6 +241,7 @@ pub mod event { LongTxn(LongTxn), } } +/// NOTE: events and resolved_ts won't appear simultaneously in one ChangeDataEvent. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ChangeDataEvent { @@ -246,6 +258,8 @@ pub struct ResolvedTs { pub regions: ::prost::alloc::vec::Vec, #[prost(uint64, tag = "2")] pub ts: u64, + #[prost(uint64, tag = "3")] + pub request_id: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -263,11 +277,18 @@ pub struct ChangeDataRequest { #[prost(bytes = "vec", tag = "6")] pub end_key: ::prost::alloc::vec::Vec, /// Used for CDC to identify events corresponding to different requests. + /// Generally in one call stream, a region can be subscribed multiple + /// times with different `request_id`s. #[prost(uint64, tag = "7")] pub request_id: u64, #[prost(enumeration = "super::kvrpcpb::ExtraOp", tag = "8")] pub extra_op: i32, - #[prost(oneof = "change_data_request::Request", tags = "9, 10")] + #[prost(enumeration = "change_data_request::KvApi", tag = "11")] + pub kv_api: i32, + /// Whether to filter out the value write by cdc itself. + #[prost(bool, tag = "12")] + pub filter_loop: bool, + #[prost(oneof = "change_data_request::Request", tags = "9, 10, 13")] pub request: ::core::option::Option, } /// Nested message and enum types in `ChangeDataRequest`. @@ -277,10 +298,54 @@ pub mod change_data_request { pub struct Register {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] + pub struct Deregister {} + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Message)] pub struct NotifyTxnStatus { #[prost(message, repeated, tag = "1")] pub txn_status: ::prost::alloc::vec::Vec, } + /// KvAPI specifies to capture data written by different KV API. + /// See more details in + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + ::prost::Enumeration + )] + #[repr(i32)] + pub enum KvApi { + TiDb = 0, + RawKv = 1, + TxnKv = 2, + } + impl KvApi { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + KvApi::TiDb => "TiDB", + KvApi::RawKv => "RawKV", + KvApi::TxnKv => "TxnKV", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "TiDB" => Some(Self::TiDb), + "RawKV" => Some(Self::RawKv), + "TxnKV" => Some(Self::TxnKv), + _ => None, + } + } + } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum Request { @@ -291,6 +356,8 @@ pub mod change_data_request { /// min_commit_ts so that the resolved_ts can be advanced. #[prost(message, tag = "10")] NotifyTxnStatus(NotifyTxnStatus), + #[prost(message, tag = "13")] + Deregister(Deregister), } } /// Generated client implementations. @@ -403,5 +470,35 @@ pub mod change_data_client { .insert(GrpcMethod::new("cdcpb.ChangeData", "EventFeed")); self.inner.streaming(req, path, codec).await } + /// EventFeedV2 is like EventFeed, with some new changes: + /// + /// * clients send requested features in HTTP/2 headers; + /// * if servers meets unsupported feature request, + /// it can fail the stream with an UNIMPLEMENTED error. + pub async fn event_feed_v2( + &mut self, + request: impl tonic::IntoStreamingRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/cdcpb.ChangeData/EventFeedV2", + ); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert(GrpcMethod::new("cdcpb.ChangeData", "EventFeedV2")); + self.inner.streaming(req, path, codec).await + } } } diff --git a/src/generated/configpb.rs b/src/generated/configpb.rs index 4888629..58f3c34 100644 --- a/src/generated/configpb.rs +++ b/src/generated/configpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Status { diff --git a/src/generated/coprocessor.rs b/src/generated/coprocessor.rs index 44fffbc..907f969 100644 --- a/src/generated/coprocessor.rs +++ b/src/generated/coprocessor.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// \[start, end) #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -34,6 +35,15 @@ pub struct Request { /// paging_size is 0 when it's disabled, otherwise, it should be a positive number. #[prost(uint64, tag = "10")] pub paging_size: u64, + /// tasks stores the batched coprocessor tasks sent to the same tikv store. + #[prost(message, repeated, tag = "11")] + pub tasks: ::prost::alloc::vec::Vec, + /// This is the session id between a client and tidb + #[prost(uint64, tag = "12")] + pub connection_id: u64, + /// This is the session alias between a client and tidb + #[prost(string, tag = "13")] + pub connection_alias: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -62,8 +72,13 @@ pub struct Response { pub cache_last_version: u64, #[prost(bool, tag = "9")] pub can_be_cached: bool, - #[prost(message, repeated, tag = "10")] - pub spans: ::prost::alloc::vec::Vec, + /// Contains the latest buckets version of the region. + /// Clients should query PD to update buckets in cache if its is stale. + #[prost(uint64, tag = "12")] + pub latest_buckets_version: u64, + /// StoreBatchTaskResponse is the collection of batch task responses. + #[prost(message, repeated, tag = "13")] + pub batch_responses: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -77,6 +92,14 @@ pub struct RegionInfo { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct TableRegions { + #[prost(int64, tag = "1")] + pub physical_table_id: i64, + #[prost(message, repeated, tag = "2")] + pub regions: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct BatchRequest { #[prost(message, optional, tag = "1")] pub context: ::core::option::Option, @@ -91,6 +114,17 @@ pub struct BatchRequest { /// Any schema-ful storage to validate schema correctness if necessary. #[prost(int64, tag = "6")] pub schema_ver: i64, + /// Used for partition table scan + #[prost(message, repeated, tag = "7")] + pub table_regions: ::prost::alloc::vec::Vec, + #[prost(string, tag = "8")] + pub log_id: ::prost::alloc::string::String, + /// This is the session id between a client and tidb + #[prost(uint64, tag = "9")] + pub connection_id: u64, + /// This is the session alias between a client and tidb + #[prost(string, tag = "10")] + pub connection_alias: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -104,3 +138,33 @@ pub struct BatchResponse { #[prost(message, repeated, tag = "4")] pub retry_regions: ::prost::alloc::vec::Vec, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StoreBatchTask { + #[prost(uint64, tag = "1")] + pub region_id: u64, + #[prost(message, optional, tag = "2")] + pub region_epoch: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub peer: ::core::option::Option, + #[prost(message, repeated, tag = "4")] + pub ranges: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "5")] + pub task_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StoreBatchTaskResponse { + #[prost(bytes = "vec", tag = "1")] + pub data: ::prost::alloc::vec::Vec, + #[prost(message, optional, tag = "2")] + pub region_error: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub locked: ::core::option::Option, + #[prost(string, tag = "4")] + pub other_error: ::prost::alloc::string::String, + #[prost(uint64, tag = "5")] + pub task_id: u64, + #[prost(message, optional, tag = "6")] + pub exec_details_v2: ::core::option::Option, +} diff --git a/src/generated/deadlock.rs b/src/generated/deadlock.rs index ed760b9..ef9c61e 100644 --- a/src/generated/deadlock.rs +++ b/src/generated/deadlock.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct WaitForEntriesRequest {} diff --git a/src/generated/debugpb.rs b/src/generated/debugpb.rs index c42c63e..dbe0d3e 100644 --- a/src/generated/debugpb.rs +++ b/src/generated/debugpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetRequest { @@ -239,6 +240,124 @@ pub struct GetAllRegionsInStoreResponse { #[prost(uint64, repeated, tag = "1")] pub regions: ::prost::alloc::vec::Vec, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResetToVersionRequest { + #[prost(uint64, tag = "1")] + pub ts: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResetToVersionResponse {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetRangePropertiesRequest { + #[prost(bytes = "vec", tag = "1")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "2")] + pub end_key: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetRangePropertiesResponse { + #[prost(message, repeated, tag = "1")] + pub properties: ::prost::alloc::vec::Vec< + get_range_properties_response::RangeProperty, + >, +} +/// Nested message and enum types in `GetRangePropertiesResponse`. +pub mod get_range_properties_response { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct RangeProperty { + #[prost(string, tag = "1")] + pub key: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub value: ::prost::alloc::string::String, + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlashbackToVersionRequest { + #[prost(message, optional, tag = "1")] + pub context: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub version: u64, + #[prost(uint64, tag = "3")] + pub region_id: u64, + #[prost(bytes = "vec", tag = "4")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "5")] + pub end_key: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "6")] + pub start_ts: u64, + #[prost(uint64, tag = "7")] + pub commit_ts: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlashbackToVersionResponse { + #[prost(string, tag = "1")] + pub error: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetRegionReadProgressRequest { + #[prost(uint64, tag = "1")] + pub region_id: u64, + /// when set to true, print a log of the locks with min start_ts in the resolver. + #[prost(bool, tag = "2")] + pub log_locks: bool, + /// only print locks whose start_ts >= min_start_ts. Can be used to find certain transaction. + #[prost(uint64, tag = "3")] + pub min_start_ts: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetRegionReadProgressResponse { + /// below are from region_read_progress module + #[prost(uint64, tag = "1")] + pub safe_ts: u64, + #[prost(uint64, tag = "2")] + pub applied_index: u64, + #[prost(uint64, tag = "3")] + pub pending_front_applied_index: u64, + #[prost(uint64, tag = "4")] + pub pending_front_ts: u64, + #[prost(uint64, tag = "5")] + pub pending_back_applied_index: u64, + #[prost(uint64, tag = "6")] + pub pending_back_ts: u64, + #[prost(bool, tag = "7")] + pub region_read_progress_paused: bool, + #[prost(uint64, tag = "8")] + pub duration_to_last_update_safe_ts_ms: u64, + #[prost(uint64, tag = "9")] + pub duration_to_last_consume_leader_ms: u64, + #[prost(bool, tag = "10")] + pub region_read_progress_exist: bool, + #[prost(uint64, tag = "18")] + pub read_state_ts: u64, + #[prost(uint64, tag = "19")] + pub read_state_apply_index: u64, + #[prost(bool, tag = "20")] + pub discard: bool, + /// below are from resolved-ts module + #[prost(uint64, tag = "11")] + pub resolved_ts: u64, + #[prost(uint64, tag = "12")] + pub resolver_tracked_index: u64, + #[prost(bool, tag = "13")] + pub resolver_exist: bool, + #[prost(bool, tag = "14")] + pub resolver_stopped: bool, + #[prost(uint64, tag = "16")] + pub num_locks: u64, + #[prost(uint64, tag = "17")] + pub num_transactions: u64, + #[prost(string, tag = "15")] + pub error: ::prost::alloc::string::String, +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum Db { @@ -845,5 +964,109 @@ pub mod debug_client { .insert(GrpcMethod::new("debugpb.Debug", "GetAllRegionsInStore")); self.inner.unary(req, path, codec).await } + /// Make this TiKV node return to the status on this node to certain ts. + pub async fn reset_to_version( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/debugpb.Debug/ResetToVersion", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("debugpb.Debug", "ResetToVersion")); + self.inner.unary(req, path, codec).await + } + /// Get range properties + pub async fn get_range_properties( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/debugpb.Debug/GetRangeProperties", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("debugpb.Debug", "GetRangeProperties")); + self.inner.unary(req, path, codec).await + } + /// Flashback given key range to a specified version. + pub async fn flashback_to_version( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/debugpb.Debug/FlashbackToVersion", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("debugpb.Debug", "FlashbackToVersion")); + self.inner.unary(req, path, codec).await + } + /// GetRegionReadProgress returns the some useful info in RegionReadProgress + pub async fn get_region_read_progress( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/debugpb.Debug/GetRegionReadProgress", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("debugpb.Debug", "GetRegionReadProgress")); + self.inner.unary(req, path, codec).await + } } } diff --git a/src/generated/diagnosticspb.rs b/src/generated/diagnosticspb.rs index 9127e73..edb1dd4 100644 --- a/src/generated/diagnosticspb.rs +++ b/src/generated/diagnosticspb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct SearchLogRequest { diff --git a/src/generated/disaggregated.rs b/src/generated/disaggregated.rs new file mode 100644 index 0000000..0b24ba7 --- /dev/null +++ b/src/generated/disaggregated.rs @@ -0,0 +1,265 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct S3LockResult { + #[prost(oneof = "s3_lock_result::Error", tags = "1, 2, 3")] + pub error: ::core::option::Option, +} +/// Nested message and enum types in `S3LockResult`. +pub mod s3_lock_result { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Error { + #[prost(message, tag = "1")] + Success(super::Success), + #[prost(message, tag = "2")] + NotOwner(super::NotOwner), + #[prost(message, tag = "3")] + Conflict(super::Conflict), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Success {} +/// Error caused by S3GC owner changed +/// client should retry +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct NotOwner {} +/// Error caused by concurrency conflict, +/// request cancel +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Conflict { + #[prost(string, tag = "1")] + pub reason: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TryAddLockRequest { + /// The data file key to add lock + #[prost(bytes = "vec", tag = "1")] + pub data_file_key: ::prost::alloc::vec::Vec, + /// The lock store id + #[prost(uint64, tag = "3")] + pub lock_store_id: u64, + /// The upload sequence number of lock store + #[prost(uint64, tag = "4")] + pub lock_seq: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TryAddLockResponse { + #[prost(message, optional, tag = "1")] + pub result: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TryMarkDeleteRequest { + /// The data file key to be marked as deleted + #[prost(bytes = "vec", tag = "1")] + pub data_file_key: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TryMarkDeleteResponse { + #[prost(message, optional, tag = "1")] + pub result: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetDisaggConfigRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DisaggS3Config { + #[prost(string, tag = "1")] + pub bucket: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub root: ::prost::alloc::string::String, + #[prost(string, tag = "3")] + pub endpoint: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetDisaggConfigResponse { + #[prost(message, optional, tag = "1")] + pub s3_config: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DisaggTaskMeta { + /// start ts of a query + #[prost(uint64, tag = "1")] + pub start_ts: u64, + /// gather_id + query_ts + server_id + local_query_id to represent a global unique query. + /// + /// used to distinguish different gathers in the mpp query + #[prost(int64, tag = "9")] + pub gather_id: i64, + /// timestamp when start to execute query, used for TiFlash miniTSO schedule. + #[prost(uint64, tag = "2")] + pub query_ts: u64, + /// TiDB server id + #[prost(uint64, tag = "3")] + pub server_id: u64, + /// unique local query_id if tidb don't restart. + #[prost(uint64, tag = "4")] + pub local_query_id: u64, + /// if task id is -1 , it indicates a tidb task. + #[prost(int64, tag = "5")] + pub task_id: i64, + /// the exectuor id + #[prost(string, tag = "6")] + pub executor_id: ::prost::alloc::string::String, + /// keyspace id of the request + #[prost(uint32, tag = "7")] + pub keyspace_id: u32, + /// API version of the request + #[prost(enumeration = "super::kvrpcpb::ApiVersion", tag = "8")] + pub api_version: i32, + /// This is the session id between a client and tidb + #[prost(uint64, tag = "10")] + pub connection_id: u64, + /// This is the session alias between a client and tidb + #[prost(string, tag = "11")] + pub connection_alias: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DisaggReadError { + #[prost(int32, tag = "1")] + pub code: i32, + #[prost(string, tag = "2")] + pub msg: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct EstablishDisaggTaskError { + #[prost(oneof = "establish_disagg_task_error::Errors", tags = "1, 2, 99")] + pub errors: ::core::option::Option, +} +/// Nested message and enum types in `EstablishDisaggTaskError`. +pub mod establish_disagg_task_error { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Errors { + #[prost(message, tag = "1")] + ErrorRegion(super::ErrorRegion), + #[prost(message, tag = "2")] + ErrorLocked(super::ErrorLocked), + #[prost(message, tag = "99")] + ErrorOther(super::ErrorOther), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ErrorRegion { + #[prost(string, tag = "1")] + pub msg: ::prost::alloc::string::String, + /// The read node needs to update its region cache about these regions. + #[prost(uint64, repeated, tag = "2")] + pub region_ids: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ErrorLocked { + #[prost(string, tag = "1")] + pub msg: ::prost::alloc::string::String, + /// The read node needs to resolve these locks. + #[prost(message, repeated, tag = "2")] + pub locked: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ErrorOther { + #[prost(int32, tag = "1")] + pub code: i32, + #[prost(string, tag = "2")] + pub msg: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct EstablishDisaggTaskRequest { + #[prost(message, optional, tag = "1")] + pub meta: ::core::option::Option, + /// target address of this task. + #[prost(string, tag = "2")] + pub address: ::prost::alloc::string::String, + /// The write node needs to ensure that subsequent + /// FetchDisaggPagesRequest can be processed within timeout_s. + /// unit: seconds + #[prost(int64, tag = "3")] + pub timeout_s: i64, + /// The key ranges, Region meta that read node need to execute TableScan + #[prost(message, repeated, tag = "4")] + pub regions: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "5")] + pub schema_ver: i64, + /// Used for PartitionTableScan + #[prost(message, repeated, tag = "6")] + pub table_regions: ::prost::alloc::vec::Vec, + /// The encoded TableScan/PartitionTableScan + Selection. + #[prost(bytes = "vec", tag = "7")] + pub encoded_plan: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct EstablishDisaggTaskResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + /// Write node maintains a snapshot with a lease time. + /// Read node should read the delta pages + /// (ColumnFileInMemory and ColumnFileTiny) + /// along with this store_id and snapshot_id. + /// + /// metapb.Store.id + #[prost(uint64, tag = "3")] + pub store_id: u64, + #[prost(message, optional, tag = "4")] + pub snapshot_id: ::core::option::Option, + /// Serialized disaggregated tasks (per physical table) + #[prost(bytes = "vec", repeated, tag = "5")] + pub tables: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CancelDisaggTaskRequest { + #[prost(message, optional, tag = "1")] + pub meta: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CancelDisaggTaskResponse {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FetchDisaggPagesRequest { + /// The snapshot id to fetch pages + #[prost(message, optional, tag = "1")] + pub snapshot_id: ::core::option::Option, + #[prost(int64, tag = "2")] + pub table_id: i64, + #[prost(uint64, tag = "3")] + pub segment_id: u64, + /// It must be a subset of the delta pages ids returned + /// in EstablishDisaggTaskResponse.segments + #[prost(uint64, repeated, tag = "4")] + pub page_ids: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PagesPacket { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + /// Serialized column file data + /// + /// * ColumnFilePersisted alone with its schema, page data, field offsets + #[prost(bytes = "vec", repeated, tag = "2")] + pub pages: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, + /// * ColumnFileInMemory alone with its serialized block + #[prost(bytes = "vec", repeated, tag = "3")] + pub chunks: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, + /// Return tipb.SelectResponse.execution_summaries in the + /// last packet + #[prost(bytes = "vec", repeated, tag = "4")] + pub summaries: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, +} diff --git a/src/generated/disk_usage.rs b/src/generated/disk_usage.rs index 6f4292a..4b2c1fc 100644 --- a/src/generated/disk_usage.rs +++ b/src/generated/disk_usage.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum DiskUsage { diff --git a/src/generated/encryptionpb.rs b/src/generated/encryptionpb.rs index 26c2be8..f7d7a92 100644 --- a/src/generated/encryptionpb.rs +++ b/src/generated/encryptionpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// General encryption metadata for any data type. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -141,6 +142,7 @@ pub enum EncryptionMethod { Aes128Ctr = 2, Aes192Ctr = 3, Aes256Ctr = 4, + Sm4Ctr = 5, } impl EncryptionMethod { /// String value of the enum field names used in the ProtoBuf definition. @@ -154,6 +156,7 @@ impl EncryptionMethod { EncryptionMethod::Aes128Ctr => "AES128_CTR", EncryptionMethod::Aes192Ctr => "AES192_CTR", EncryptionMethod::Aes256Ctr => "AES256_CTR", + EncryptionMethod::Sm4Ctr => "SM4_CTR", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -164,6 +167,7 @@ impl EncryptionMethod { "AES128_CTR" => Some(Self::Aes128Ctr), "AES192_CTR" => Some(Self::Aes192Ctr), "AES256_CTR" => Some(Self::Aes256Ctr), + "SM4_CTR" => Some(Self::Sm4Ctr), _ => None, } } diff --git a/src/generated/enginepb.rs b/src/generated/enginepb.rs index 486cefe..b084107 100644 --- a/src/generated/enginepb.rs +++ b/src/generated/enginepb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CommandRequestHeader { diff --git a/src/generated/eraftpb.rs b/src/generated/eraftpb.rs index c2920dc..6724cb2 100644 --- a/src/generated/eraftpb.rs +++ b/src/generated/eraftpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// The entry is a type of change that needs to be applied. It contains two data fields. /// While the fields are built into the model; their usage is determined by the entry_type. /// @@ -77,7 +78,14 @@ pub struct Message { #[prost(bytes = "vec", tag = "12")] pub context: ::prost::alloc::vec::Vec, #[prost(uint64, tag = "14")] - pub priority: u64, + pub deprecated_priority: u64, + /// If this new field is not set, then use the above old field; otherwise + /// use the new field. When broadcasting request vote, both fields are + /// set if the priority is larger than 0. This change is not a fully + /// compatible change, but it makes minimal impact that only new priority + /// is not recognized by the old nodes during rolling update. + #[prost(int64, tag = "15")] + pub priority: i64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/src/generated/errorpb.rs b/src/generated/errorpb.rs index 77a5405..4ed789b 100644 --- a/src/generated/errorpb.rs +++ b/src/generated/errorpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// NotLeader is the error variant that tells a request be handle by raft leader /// is sent to raft follower or learner. #[allow(clippy::derive_partial_eq_without_eq)] @@ -10,6 +11,25 @@ pub struct NotLeader { #[prost(message, optional, tag = "2")] pub leader: ::core::option::Option, } +/// IsWitness is the error variant that tells a request be handle by witness +/// which should be forbidden and retry. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct IsWitness { + /// The requested region ID + #[prost(uint64, tag = "1")] + pub region_id: u64, +} +/// BucketVersionNotMatch is the error variant that tells the request buckets version is not match. +/// client should update the buckets version and retry. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct BucketVersionNotMatch { + #[prost(uint64, tag = "1")] + pub version: u64, + #[prost(bytes = "vec", repeated, tag = "2")] + pub keys: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, +} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DiskFull { @@ -87,6 +107,11 @@ pub struct ServerIsBusy { /// The suggested backoff time #[prost(uint64, tag = "2")] pub backoff_ms: u64, + #[prost(uint32, tag = "3")] + pub estimated_wait_ms: u32, + /// Current applied_index at the leader, may be used in replica read. + #[prost(uint64, tag = "4")] + pub applied_index: u64, } /// StaleCommand is the error variant that tells the command is stale, that is, /// the current request term is lower than current raft term. @@ -149,6 +174,39 @@ pub struct DataIsNotReady { #[prost(uint64, tag = "3")] pub safe_ts: u64, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RecoveryInProgress { + /// The requested region ID + #[prost(uint64, tag = "1")] + pub region_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlashbackInProgress { + /// The requested region ID + #[prost(uint64, tag = "1")] + pub region_id: u64, + #[prost(uint64, tag = "2")] + pub flashback_start_ts: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlashbackNotPrepared { + /// The requested region ID + #[prost(uint64, tag = "1")] + pub region_id: u64, +} +/// MismatchPeerId is the error variant that tells the request is sent to wrong peer. +/// Client receives this error should reload the region info and retry. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MismatchPeerId { + #[prost(uint64, tag = "1")] + pub request_peer_id: u64, + #[prost(uint64, tag = "2")] + pub store_peer_id: u64, +} /// Error wraps all region errors, indicates an error encountered by a request. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -184,4 +242,25 @@ pub struct Error { pub region_not_initialized: ::core::option::Option, #[prost(message, optional, tag = "15")] pub disk_full: ::core::option::Option, + /// Online recovery is still in performing, reject writes to avoid potential issues + #[prost(message, optional, tag = "16")] + pub recovery_in_progress: ::core::option::Option, + /// Flashback is still in performing, reject any read or write to avoid potential issues. + /// NOTICE: this error is non-retryable, the request should fail ASAP when it meets this error. + #[prost(message, optional, tag = "17")] + pub flashback_in_progress: ::core::option::Option, + /// If the second phase flashback request is sent to a region that is not prepared for the flashback, + /// this error will be returned. + /// NOTICE: this error is non-retryable, the client should retry the first phase flashback request when it meets this error. + #[prost(message, optional, tag = "18")] + pub flashback_not_prepared: ::core::option::Option, + /// IsWitness is the error variant that tells a request be handle by witness + /// which should be forbidden and retry. + #[prost(message, optional, tag = "19")] + pub is_witness: ::core::option::Option, + #[prost(message, optional, tag = "20")] + pub mismatch_peer_id: ::core::option::Option, + /// BucketVersionNotMatch is the error variant that tells the request buckets version is not match. + #[prost(message, optional, tag = "21")] + pub bucket_version_not_match: ::core::option::Option, } diff --git a/src/generated/gcpb.rs b/src/generated/gcpb.rs new file mode 100644 index 0000000..ef87aeb --- /dev/null +++ b/src/generated/gcpb.rs @@ -0,0 +1,352 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RequestHeader { + /// cluster_id is the ID of the cluster which be sent to. + #[prost(uint64, tag = "1")] + pub cluster_id: u64, + /// sender_id is the ID of the sender server, also member ID or etcd ID. + #[prost(uint64, tag = "2")] + pub sender_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResponseHeader { + /// cluster_id is the ID of the cluster which sent the response. + #[prost(uint64, tag = "1")] + pub cluster_id: u64, + #[prost(message, optional, tag = "2")] + pub error: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Error { + #[prost(enumeration = "ErrorType", tag = "1")] + pub r#type: i32, + #[prost(string, tag = "2")] + pub message: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct KeySpace { + #[prost(bytes = "vec", tag = "1")] + pub space_id: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "2")] + pub gc_safe_point: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListKeySpacesRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + /// set with_gc_safe_point to true to also receive gc safe point for each key space + #[prost(bool, tag = "2")] + pub with_gc_safe_point: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListKeySpacesResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub key_spaces: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetMinServiceSafePointRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub space_id: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetMinServiceSafePointResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub safe_point: u64, + /// revision here is to safeguard the validity of the obtained min, + /// preventing cases where new services register their safe points after min is obtained by gc worker + #[prost(int64, tag = "3")] + pub revision: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateGcSafePointRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub space_id: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "3")] + pub safe_point: u64, + /// here client need to provide the revision obtained from GetMinServiceSafePoint, + /// so server can check if it's still valid + #[prost(int64, tag = "4")] + pub revision: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateGcSafePointResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + /// update will be successful if revision is valid and new safepoint > old safe point + /// if failed, previously obtained min might be incorrect, should retry from GetMinServiceSafePoint + #[prost(bool, tag = "2")] + pub succeeded: bool, + #[prost(uint64, tag = "3")] + pub new_safe_point: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateServiceSafePointRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub space_id: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "3")] + pub service_id: ::prost::alloc::vec::Vec, + /// safe point will be set to expire on (PD Server time + TTL) + /// pass in a ttl \< 0 to remove target safe point + /// pass in MAX_INT64 to set a safe point that never expire + #[prost(int64, tag = "4")] + pub ttl: i64, + #[prost(uint64, tag = "5")] + pub safe_point: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateServiceSafePointResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + /// update will be successful if ttl \< 0 (a removal request) + /// or if new safe point >= old safe point and new safe point >= gc safe point + #[prost(bool, tag = "2")] + pub succeeded: bool, + #[prost(uint64, tag = "3")] + pub gc_safe_point: u64, + #[prost(uint64, tag = "4")] + pub old_safe_point: u64, + #[prost(uint64, tag = "5")] + pub new_safe_point: u64, +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ErrorType { + Ok = 0, + Unknown = 1, + NotBootstrapped = 2, + /// revision supplied does not match the current etcd revision + RevisionMismatch = 3, + /// if the proposed safe point is earlier than old safe point or gc safe point + SafepointRollback = 4, +} +impl ErrorType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ErrorType::Ok => "OK", + ErrorType::Unknown => "UNKNOWN", + ErrorType::NotBootstrapped => "NOT_BOOTSTRAPPED", + ErrorType::RevisionMismatch => "REVISION_MISMATCH", + ErrorType::SafepointRollback => "SAFEPOINT_ROLLBACK", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "OK" => Some(Self::Ok), + "UNKNOWN" => Some(Self::Unknown), + "NOT_BOOTSTRAPPED" => Some(Self::NotBootstrapped), + "REVISION_MISMATCH" => Some(Self::RevisionMismatch), + "SAFEPOINT_ROLLBACK" => Some(Self::SafepointRollback), + _ => None, + } + } +} +/// Generated client implementations. +pub mod gc_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct GcClient { + inner: tonic::client::Grpc, + } + impl GcClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl GcClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> GcClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + GcClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn list_key_spaces( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/gcpb.GC/ListKeySpaces"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("gcpb.GC", "ListKeySpaces")); + self.inner.unary(req, path, codec).await + } + pub async fn get_min_service_safe_point( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/gcpb.GC/GetMinServiceSafePoint", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("gcpb.GC", "GetMinServiceSafePoint")); + self.inner.unary(req, path, codec).await + } + pub async fn update_gc_safe_point( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/gcpb.GC/UpdateGCSafePoint", + ); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("gcpb.GC", "UpdateGCSafePoint")); + self.inner.unary(req, path, codec).await + } + pub async fn update_service_safe_point( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/gcpb.GC/UpdateServiceSafePoint", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("gcpb.GC", "UpdateServiceSafePoint")); + self.inner.unary(req, path, codec).await + } + } +} diff --git a/src/generated/google.api.rs b/src/generated/google.api.rs index 76cd332..bb9f720 100644 --- a/src/generated/google.api.rs +++ b/src/generated/google.api.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// Defines the HTTP configuration for an API service. It contains a list of /// \[HttpRule\]\[google.api.HttpRule\], each specifying the mapping of an RPC method /// to one or more HTTP REST API methods. diff --git a/src/generated/import_kvpb.rs b/src/generated/import_kvpb.rs index 7c681ad..104b434 100644 --- a/src/generated/import_kvpb.rs +++ b/src/generated/import_kvpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct SwitchModeRequest { diff --git a/src/generated/import_sstpb.rs b/src/generated/import_sstpb.rs index fcd9f5d..4f8a127 100644 --- a/src/generated/import_sstpb.rs +++ b/src/generated/import_sstpb.rs @@ -1,14 +1,48 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SuspendImportRpcRequest { + /// whether to suspend new imports. + #[prost(bool, tag = "1")] + pub should_suspend_imports: bool, + /// the duration of import service suspension + /// when should_deny_imports is false, + /// this won't take effect. + #[prost(uint64, tag = "2")] + pub duration_in_secs: u64, + /// The identifier for the caller. + #[prost(string, tag = "3")] + pub caller: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SuspendImportRpcResponse { + /// The last state before this RPC. + #[prost(bool, tag = "1")] + pub already_suspended: bool, +} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct SwitchModeRequest { #[prost(enumeration = "SwitchMode", tag = "1")] pub mode: i32, + #[prost(message, repeated, tag = "2")] + pub ranges: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct SwitchModeResponse {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetModeRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetModeResponse { + #[prost(enumeration = "SwitchMode", tag = "1")] + pub mode: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct Range { #[prost(bytes = "vec", tag = "1")] pub start: ::prost::alloc::vec::Vec, @@ -111,6 +145,8 @@ pub struct CompactRequest { pub range: ::core::option::Option, #[prost(int32, tag = "2")] pub output_level: i32, + #[prost(message, optional, tag = "3")] + pub context: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -118,6 +154,13 @@ pub struct CompactResponse {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DownloadRequest { + /// Map represents the map of \. + /// We'll generate all SSTMeta into one SST File. + #[prost(map = "string, message", tag = "1")] + pub ssts: ::std::collections::HashMap<::prost::alloc::string::String, SstMeta>, + /// resolved_ts is used to merge related SST Files. + #[prost(uint64, tag = "3")] + pub resolved_ts: u64, /// The SST meta used to identify the downloaded file. /// Must be the same among all nodes in the same Raft group. /// Note: the "crc32" and "cf_name" fields are ignored in this request, @@ -144,11 +187,21 @@ pub struct DownloadRequest { pub rewrite_rule: ::core::option::Option, #[prost(message, optional, tag = "14")] pub storage_backend: ::core::option::Option, + /// The identity for the stroage backend. + /// When this field presents, the storage would be cached. + /// If there is a cached storage, TiKV would use it driectly. + #[prost(string, tag = "17")] + pub storage_cache_id: ::prost::alloc::string::String, #[prost(bool, tag = "15")] pub is_raw_kv: bool, /// cipher_info is used to decrypt sst when download sst #[prost(message, optional, tag = "16")] pub cipher_info: ::core::option::Option, + /// The type of the download request. + #[prost(enumeration = "DownloadRequestType", tag = "18")] + pub request_type: i32, + #[prost(message, optional, tag = "19")] + pub context: ::core::option::Option, } /// For now it is just used for distinguishing the error of the request with the error /// of gRPC, add more concrete types if it is necessary later. @@ -157,6 +210,9 @@ pub struct DownloadRequest { pub struct Error { #[prost(string, tag = "1")] pub message: ::prost::alloc::string::String, + /// We meet some internal errors of the store. + #[prost(message, optional, tag = "2")] + pub store_error: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -179,6 +235,10 @@ pub struct DownloadResponse { /// The actual length of the rewritten SST file. #[prost(uint64, tag = "5")] pub length: u64, + /// This field only return when file-copy backup enabled. + /// Because it will merge many SST files in a download request. + #[prost(message, repeated, tag = "6")] + pub ssts: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -250,6 +310,8 @@ pub struct WriteBatch { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct WriteRequest { + #[prost(message, optional, tag = "3")] + pub context: ::core::option::Option, #[prost(oneof = "write_request::Chunk", tags = "1, 2")] pub chunk: ::core::option::Option, } @@ -279,10 +341,19 @@ pub struct RawWriteBatch { pub ttl: u64, #[prost(message, repeated, tag = "2")] pub pairs: ::prost::alloc::vec::Vec, + /// To be compatible with the key encoding of API V2. + /// This field should be generated from the client instead of the server, + /// since the message will be send to all the replicas of a region. + /// Otherwise, the underlying data generated by the server would be inconsistent which is hard to scale + /// for other features like MVCC over RawKV. + #[prost(uint64, tag = "3")] + pub ts: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct RawWriteRequest { + #[prost(message, optional, tag = "3")] + pub context: ::core::option::Option, #[prost(oneof = "raw_write_request::Chunk", tags = "1, 2")] pub chunk: ::core::option::Option, } @@ -351,6 +422,109 @@ pub struct DuplicateDetectResponse { #[prost(message, repeated, tag = "3")] pub pairs: ::prost::alloc::vec::Vec, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct KvMeta { + /// The file name of the KV file. + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + /// file offset, sometimes only need to get a part of data from the merged file + #[prost(uint64, tag = "11")] + pub range_offset: u64, + /// file length for check. + #[prost(uint64, tag = "2")] + pub length: u64, + /// range length of the merged file, if it exists. + #[prost(uint64, tag = "12")] + pub range_length: u64, + /// tell us which cf should apply. WRITE_CF or DEFAULT_CF e.g. + #[prost(string, tag = "3")] + pub cf: ::prost::alloc::string::String, + /// is_delete represents whether we should delete the kv in tikv. + /// it may not be too much delete file. only rollBack operation will generate delete kv file. + #[prost(bool, tag = "4")] + pub is_delete: bool, + /// the key ts space being smaller than start_ts can be filter. + #[prost(uint64, tag = "10")] + pub start_ts: u64, + /// the key ts space large than restore_ts can be filter. + #[prost(uint64, tag = "5")] + pub restore_ts: u64, + #[prost(bytes = "vec", tag = "6")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "7")] + pub end_key: ::prost::alloc::vec::Vec, + /// used for checksum when download kv file. + #[prost(bytes = "vec", tag = "8")] + pub sha256: ::prost::alloc::vec::Vec, + /// the key ts space less than start_snapshot_ts can be filter. + /// Deprecated: this field 'start_snapshot_ts' is replaced by the field 'start_ts'. + #[prost(uint64, tag = "9")] + pub start_snapshot_ts: u64, + /// the compression type for the file. + #[prost(enumeration = "super::backup::CompressionType", tag = "13")] + pub compression_type: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ApplyRequest { + /// The meta of the KV file. + #[prost(message, optional, tag = "1")] + pub meta: ::core::option::Option, + #[prost(message, repeated, tag = "12")] + pub metas: ::prost::alloc::vec::Vec, + /// Performs a key prefix rewrite after downloading the file. + /// All keys in the files will be rewritten as: + /// + /// new_key = new_key_prefix + old_key\[len(old_key_prefix)..\] + /// + /// When used for TiDB, rewriting the prefix changes the table ID. Please + /// note that key-rewrite is applied on the origin keys in encoded + /// representation. + /// + /// You need to ensure that the keys before and after rewriting are in the + /// same order, otherwise the RPC request will fail. + #[prost(message, optional, tag = "2")] + pub rewrite_rule: ::core::option::Option, + #[prost(message, repeated, tag = "13")] + pub rewrite_rules: ::prost::alloc::vec::Vec, + /// The identity for the stroage backend. + /// When this field presents, the storage would be cached. + /// If there is a cached storage, TiKV would use it driectly. + #[prost(string, tag = "5")] + pub storage_cache_id: ::prost::alloc::string::String, + #[prost(message, optional, tag = "3")] + pub storage_backend: ::core::option::Option, + /// context represents region info and it used to build raft commands. + #[prost(message, optional, tag = "4")] + pub context: ::core::option::Option, + /// cipher_info is used to decrypt kv file when download file. + #[prost(message, optional, tag = "11")] + pub cipher_info: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ApplyResponse { + /// The actual key range (after rewrite) of the downloaded file. The range is + /// inclusive in both ends. + #[prost(message, optional, tag = "1")] + pub range: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub error: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ClearRequest { + /// clear files in import directory with given prefix. + #[prost(string, tag = "1")] + pub prefix: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ClearResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum SwitchMode { @@ -377,6 +551,34 @@ impl SwitchMode { } } } +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum DownloadRequestType { + /// For the compatibility with old version of TiDBs + Legacy = 0, + /// For the TiDBs with newer versions that support keyspace feature. + Keyspace = 1, +} +impl DownloadRequestType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + DownloadRequestType::Legacy => "Legacy", + DownloadRequestType::Keyspace => "Keyspace", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "Legacy" => Some(Self::Legacy), + "Keyspace" => Some(Self::Keyspace), + _ => None, + } + } +} /// Generated client implementations. pub mod import_sst_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] @@ -504,6 +706,32 @@ pub mod import_sst_client { .insert(GrpcMethod::new("import_sstpb.ImportSST", "SwitchMode")); self.inner.unary(req, path, codec).await } + /// Get import mode(normal/import). + pub async fn get_mode( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/import_sstpb.ImportSST/GetMode", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("import_sstpb.ImportSST", "GetMode")); + self.inner.unary(req, path, codec).await + } /// Upload an SST file to a server. pub async fn upload( &mut self, @@ -727,5 +955,77 @@ pub mod import_sst_client { .insert(GrpcMethod::new("import_sstpb.ImportSST", "DuplicateDetect")); self.inner.server_streaming(req, path, codec).await } + /// Apply download & apply increment kv files to TiKV. + pub async fn apply( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/import_sstpb.ImportSST/Apply", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("import_sstpb.ImportSST", "Apply")); + self.inner.unary(req, path, codec).await + } + /// ClearFiles clear applied file after restore succeed. + pub async fn clear_files( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/import_sstpb.ImportSST/ClearFiles", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("import_sstpb.ImportSST", "ClearFiles")); + self.inner.unary(req, path, codec).await + } + /// Suspend ingest for data listeners don't support catching import data. + pub async fn suspend_import_rpc( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/import_sstpb.ImportSST/SuspendImportRPC", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("import_sstpb.ImportSST", "SuspendImportRPC")); + self.inner.unary(req, path, codec).await + } } } diff --git a/src/generated/keyspacepb.rs b/src/generated/keyspacepb.rs new file mode 100644 index 0000000..3cf798d --- /dev/null +++ b/src/generated/keyspacepb.rs @@ -0,0 +1,308 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct KeyspaceMeta { + #[prost(uint32, tag = "1")] + pub id: u32, + #[prost(string, tag = "2")] + pub name: ::prost::alloc::string::String, + #[prost(enumeration = "KeyspaceState", tag = "3")] + pub state: i32, + #[prost(int64, tag = "4")] + pub created_at: i64, + #[prost(int64, tag = "5")] + pub state_changed_at: i64, + #[prost(map = "string, string", tag = "7")] + pub config: ::std::collections::HashMap< + ::prost::alloc::string::String, + ::prost::alloc::string::String, + >, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct LoadKeyspaceRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(string, tag = "2")] + pub name: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct LoadKeyspaceResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub keyspace: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WatchKeyspacesRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WatchKeyspacesResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub keyspaces: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateKeyspaceStateRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint32, tag = "2")] + pub id: u32, + #[prost(enumeration = "KeyspaceState", tag = "3")] + pub state: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateKeyspaceStateResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub keyspace: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetAllKeyspacesRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint32, tag = "2")] + pub start_id: u32, + #[prost(uint32, tag = "3")] + pub limit: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetAllKeyspacesResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub keyspaces: ::prost::alloc::vec::Vec, +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum KeyspaceState { + Enabled = 0, + Disabled = 1, + Archived = 2, + Tombstone = 3, +} +impl KeyspaceState { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + KeyspaceState::Enabled => "ENABLED", + KeyspaceState::Disabled => "DISABLED", + KeyspaceState::Archived => "ARCHIVED", + KeyspaceState::Tombstone => "TOMBSTONE", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "ENABLED" => Some(Self::Enabled), + "DISABLED" => Some(Self::Disabled), + "ARCHIVED" => Some(Self::Archived), + "TOMBSTONE" => Some(Self::Tombstone), + _ => None, + } + } +} +/// Generated client implementations. +pub mod keyspace_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + /// Keyspace provides services to manage keyspaces. + #[derive(Debug, Clone)] + pub struct KeyspaceClient { + inner: tonic::client::Grpc, + } + impl KeyspaceClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl KeyspaceClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> KeyspaceClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + KeyspaceClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn load_keyspace( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/keyspacepb.Keyspace/LoadKeyspace", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("keyspacepb.Keyspace", "LoadKeyspace")); + self.inner.unary(req, path, codec).await + } + /// WatchKeyspaces first return all current keyspaces' metadata as its first response. + /// Then, it returns responses containing keyspaces that had their metadata changed. + pub async fn watch_keyspaces( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/keyspacepb.Keyspace/WatchKeyspaces", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("keyspacepb.Keyspace", "WatchKeyspaces")); + self.inner.server_streaming(req, path, codec).await + } + pub async fn update_keyspace_state( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/keyspacepb.Keyspace/UpdateKeyspaceState", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("keyspacepb.Keyspace", "UpdateKeyspaceState")); + self.inner.unary(req, path, codec).await + } + pub async fn get_all_keyspaces( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/keyspacepb.Keyspace/GetAllKeyspaces", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("keyspacepb.Keyspace", "GetAllKeyspaces")); + self.inner.unary(req, path, codec).await + } + } +} diff --git a/src/generated/kvrpcpb.rs b/src/generated/kvrpcpb.rs index 12aef31..ec03912 100644 --- a/src/generated/kvrpcpb.rs +++ b/src/generated/kvrpcpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// A transactional get command. Lookup a value for `key` in the transaction with /// starting timestamp = `version`. #[allow(clippy::derive_partial_eq_without_eq)] @@ -97,8 +98,9 @@ pub struct PrewriteRequest { #[prost(bool, tag = "6")] pub skip_constraint_check: bool, /// For pessimistic transaction, some mutations don't need to be locked, for example, non-unique index key. - #[prost(bool, repeated, tag = "7")] - pub is_pessimistic_lock: ::prost::alloc::vec::Vec, + /// Keys with deferred constraint checks are not locked. + #[prost(enumeration = "prewrite_request::PessimisticAction", repeated, tag = "7")] + pub pessimistic_actions: ::prost::alloc::vec::Vec, /// How many keys this transaction involves in this region. #[prost(uint64, tag = "8")] pub txn_size: u64, @@ -126,6 +128,69 @@ pub struct PrewriteRequest { /// The level of assertion to use on this prewrte request. #[prost(enumeration = "AssertionLevel", tag = "15")] pub assertion_level: i32, + /// for_update_ts constriants that should be checked when prewriting a pessimistic transaction. + /// See + #[prost(message, repeated, tag = "16")] + pub for_update_ts_constraints: ::prost::alloc::vec::Vec< + prewrite_request::ForUpdateTsConstraint, + >, +} +/// Nested message and enum types in `PrewriteRequest`. +pub mod prewrite_request { + /// for_update_ts constriants that should be checked when prewriting a pessimistic transaction. + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct ForUpdateTsConstraint { + /// The index of key in the prewrite request that should be checked. + #[prost(uint32, tag = "1")] + pub index: u32, + /// The expected for_update_ts of the pessimistic lock of the key. + #[prost(uint64, tag = "2")] + pub expected_for_update_ts: u64, + } + /// What kind of checks need to be performed for keys in a pessimistic transaction. + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + ::prost::Enumeration + )] + #[repr(i32)] + pub enum PessimisticAction { + /// The key needn't be locked and no extra write conflict checks are needed. + SkipPessimisticCheck = 0, + /// The key should have been locked at the time of prewrite. + DoPessimisticCheck = 1, + /// The key doesn't need a pessimistic lock. But we need to do data constraint checks. + DoConstraintCheck = 2, + } + impl PessimisticAction { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + PessimisticAction::SkipPessimisticCheck => "SKIP_PESSIMISTIC_CHECK", + PessimisticAction::DoPessimisticCheck => "DO_PESSIMISTIC_CHECK", + PessimisticAction::DoConstraintCheck => "DO_CONSTRAINT_CHECK", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "SKIP_PESSIMISTIC_CHECK" => Some(Self::SkipPessimisticCheck), + "DO_PESSIMISTIC_CHECK" => Some(Self::DoPessimisticCheck), + "DO_CONSTRAINT_CHECK" => Some(Self::DoConstraintCheck), + _ => None, + } + } + } } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -144,6 +209,9 @@ pub struct PrewriteResponse { /// transaction is not 1PC, the value will be 0. #[prost(uint64, tag = "4")] pub one_pc_commit_ts: u64, + /// Execution details about the request processing. + #[prost(message, optional, tag = "5")] + pub exec_details_v2: ::core::option::Option, } /// Lock a set of keys to prepare to write to them. #[allow(clippy::derive_partial_eq_without_eq)] @@ -194,6 +262,35 @@ pub struct PessimisticLockRequest { /// repsonse will be empty while the `not_founds` field still indicates the keys' existence. #[prost(bool, tag = "12")] pub check_existence: bool, + /// TiKV lock the record only when it exists + #[prost(bool, tag = "13")] + pub lock_only_if_exists: bool, + /// Specifies the behavior when the request is woken up after wating for lock of another transaction. + #[prost(enumeration = "PessimisticLockWakeUpMode", tag = "14")] + pub wake_up_mode: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PessimisticLockKeyResult { + #[prost(enumeration = "PessimisticLockKeyResultType", tag = "1")] + pub r#type: i32, + #[prost(bytes = "vec", tag = "2")] + pub value: ::prost::alloc::vec::Vec, + #[prost(bool, tag = "3")] + pub existence: bool, + /// We allow a key be locked when there is write conflict (latest commit_ts > for_update_ts). + /// In this case, the key is semantically locked by a newer for_update_ts. + /// For each requested key, the field is non-zero if the key is locked with write conflict, and it + /// equals to the commit_ts of the latest version of the specified key. The for_update_ts field + /// of the lock that's actually written to TiKV will also be this value. At the same time, + /// `value` and `existence` will be returned regardless to how `return_values` and + /// `check_existence` are set. + #[prost(uint64, tag = "4")] + pub locked_with_conflict_ts: u64, + /// Hint the client that resolving lock is not needed for this lock. For `PessimisticLock` + /// requests only. + #[prost(bool, tag = "11")] + pub skip_resolving_lock: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -211,13 +308,21 @@ pub struct PessimisticLockResponse { pub value: ::prost::alloc::vec::Vec, /// The values is set if 'return_values' is true in the request and no error. /// If 'force' is true, this field is not used. + /// Only used when `wake_up_mode` is `WakeUpModeNormal`. #[prost(bytes = "vec", repeated, tag = "5")] pub values: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, /// Indicates whether the values at the same index is correspond to an existing key. /// In legacy TiKV, this field is not used even 'force' is false. In that case, an empty value indicates /// two possible situations: (1) the key does not exist. (2) the key exists but the value is empty. + /// Only used when `wake_up_mode` is `WakeUpModeNormal`. #[prost(bool, repeated, tag = "6")] pub not_founds: ::prost::alloc::vec::Vec, + /// Execution details about the request processing. + #[prost(message, optional, tag = "7")] + pub exec_details_v2: ::core::option::Option, + /// Results of the request. Only used when `wake_up_mode` is `WakeUpModeForceLock`. + #[prost(message, repeated, tag = "8")] + pub results: ::prost::alloc::vec::Vec, } /// Unlock keys locked using `PessimisticLockRequest`. #[allow(clippy::derive_partial_eq_without_eq)] @@ -239,6 +344,9 @@ pub struct PessimisticRollbackResponse { pub region_error: ::core::option::Option, #[prost(message, repeated, tag = "2")] pub errors: ::prost::alloc::vec::Vec, + /// Execution details about the request processing. + #[prost(message, optional, tag = "3")] + pub exec_details_v2: ::core::option::Option, } /// Used to update the lock_ttl of a psessimistic and/or large transaction to prevent it from been killed. #[allow(clippy::derive_partial_eq_without_eq)] @@ -266,6 +374,9 @@ pub struct TxnHeartBeatResponse { /// The TTL actually set on the requested lock. #[prost(uint64, tag = "3")] pub lock_ttl: u64, + /// Execution details about the request processing. + #[prost(message, optional, tag = "4")] + pub exec_details_v2: ::core::option::Option, } /// CheckTxnStatusRequest checks the status of a transaction. /// If the transaction is rollbacked/committed, return that result. @@ -304,6 +415,13 @@ pub struct CheckTxnStatusRequest { /// it's still uncertain. #[prost(bool, tag = "8")] pub resolving_pessimistic_lock: bool, + /// Whether it's needed to check if the lock on the key (if any) is the primary lock. + /// This is for handling some corner cases when a pessimistic transaction changes its primary + /// (see for details). This field is necessary + /// because the old versions of clients cannot handle some results returned from TiKV correctly. + /// For new versions, this field should always be set to true. + #[prost(bool, tag = "9")] + pub verify_is_primary: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -325,6 +443,9 @@ pub struct CheckTxnStatusResponse { pub action: i32, #[prost(message, optional, tag = "6")] pub lock_info: ::core::option::Option, + /// Execution details about the request processing. + #[prost(message, optional, tag = "7")] + pub exec_details_v2: ::core::option::Option, } /// Part of the async commit protocol, checks for locks on all supplied keys. If a lock is missing, /// does not have a successful status, or belongs to another transaction, TiKV will leave a rollback @@ -356,6 +477,9 @@ pub struct CheckSecondaryLocksResponse { /// locks have been committed, it will be zero. #[prost(uint64, tag = "4")] pub commit_ts: u64, + /// Execution details about the request processing. + #[prost(message, optional, tag = "5")] + pub exec_details_v2: ::core::option::Option, } /// The second phase of writing to TiKV. If there are no errors or conflicts, then this request /// commits a transaction so that its data can be read by other transactions. @@ -384,6 +508,9 @@ pub struct CommitResponse { /// If the commit ts is derived from min_commit_ts, this field should be set. #[prost(uint64, tag = "3")] pub commit_version: u64, + /// Execution details about the request processing. + #[prost(message, optional, tag = "4")] + pub exec_details_v2: ::core::option::Option, } /// Not yet implemented. #[allow(clippy::derive_partial_eq_without_eq)] @@ -478,6 +605,9 @@ pub struct BatchRollbackResponse { pub region_error: ::core::option::Option, #[prost(message, optional, tag = "2")] pub error: ::core::option::Option, + /// Execution details about the request processing. + #[prost(message, optional, tag = "3")] + pub exec_details_v2: ::core::option::Option, } /// Scan the database for locks. Used at the start of the GC process to find all /// old locks. @@ -509,6 +639,9 @@ pub struct ScanLockResponse { /// Info on all locks found by the scan. #[prost(message, repeated, tag = "3")] pub locks: ::prost::alloc::vec::Vec, + /// Execution details about the request processing. + #[prost(message, optional, tag = "4")] + pub exec_details_v2: ::core::option::Option, } /// For all keys locked by the transaction identified by `start_version`, either /// commit or rollback the transaction and unlock the key. @@ -536,6 +669,9 @@ pub struct ResolveLockResponse { pub region_error: ::core::option::Option, #[prost(message, optional, tag = "2")] pub error: ::core::option::Option, + /// Execution details about the request processing. + #[prost(message, optional, tag = "3")] + pub exec_details_v2: ::core::option::Option, } /// Request TiKV to garbage collect all non-current data older than `safe_point`. #[allow(clippy::derive_partial_eq_without_eq)] @@ -555,7 +691,6 @@ pub struct GcResponse { pub error: ::core::option::Option, } /// Delete a range of data from TiKV. -/// This message should not be used. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteRangeRequest { @@ -579,6 +714,64 @@ pub struct DeleteRangeResponse { #[prost(string, tag = "2")] pub error: ::prost::alloc::string::String, } +/// Preparing the flashback for a region/key range will "lock" the region +/// so that there is no any read, write or schedule operation could be proposed before +/// the actual flashback operation. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PrepareFlashbackToVersionRequest { + #[prost(message, optional, tag = "1")] + pub context: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "3")] + pub end_key: ::prost::alloc::vec::Vec, + /// The `start_ts` which we will use to write a lock to prevent + /// the `resolved_ts` from advancing during the whole process. + #[prost(uint64, tag = "4")] + pub start_ts: u64, + /// The TS version which the data will flashback to later. + #[prost(uint64, tag = "5")] + pub version: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PrepareFlashbackToVersionResponse { + #[prost(message, optional, tag = "1")] + pub region_error: ::core::option::Option, + #[prost(string, tag = "2")] + pub error: ::prost::alloc::string::String, +} +/// Flashback the region to a specific point with the given `version`, please +/// make sure the region is "locked" by `PrepareFlashbackToVersionRequest` first, +/// otherwise this request will fail. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlashbackToVersionRequest { + #[prost(message, optional, tag = "1")] + pub context: ::core::option::Option, + /// The TS version which the data should flashback to. + #[prost(uint64, tag = "2")] + pub version: u64, + #[prost(bytes = "vec", tag = "3")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "4")] + pub end_key: ::prost::alloc::vec::Vec, + /// The `start_ts` and `commit_ts` which the newly written MVCC version will use. + /// Please make sure the `start_ts` is the same one in `PrepareFlashbackToVersionRequest`. + #[prost(uint64, tag = "5")] + pub start_ts: u64, + #[prost(uint64, tag = "6")] + pub commit_ts: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlashbackToVersionResponse { + #[prost(message, optional, tag = "1")] + pub region_error: ::core::option::Option, + #[prost(string, tag = "2")] + pub error: ::prost::alloc::string::String, +} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct RawGetRequest { @@ -652,13 +845,16 @@ pub struct RawBatchPutRequest { pub pairs: ::prost::alloc::vec::Vec, #[prost(string, tag = "3")] pub cf: ::prost::alloc::string::String, + #[deprecated] + #[prost(uint64, tag = "4")] + pub ttl: u64, + #[prost(bool, tag = "5")] + pub for_cas: bool, /// The time-to-live for each keys in seconds, and if the length of `ttls` /// is exactly one, the ttl will be applied to all keys. Otherwise, the length /// mismatch between `ttls` and `pairs` will return an error. - #[prost(uint64, repeated, tag = "4")] + #[prost(uint64, repeated, tag = "6")] pub ttls: ::prost::alloc::vec::Vec, - #[prost(bool, tag = "5")] - pub for_cas: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1024,6 +1220,63 @@ pub struct Context { /// transactions are committed and theirs commit_ts \<= read request's start_ts. #[prost(uint64, repeated, tag = "22")] pub committed_locks: ::prost::alloc::vec::Vec, + /// The informantion to trace a request sent to TiKV. + #[prost(message, optional, tag = "23")] + pub trace_context: ::core::option::Option, + /// The source of the request, will be used as the tag of the metrics reporting. + /// This field can be set for any requests that require to report metrics with any extra labels. + #[prost(string, tag = "24")] + pub request_source: ::prost::alloc::string::String, + /// The source of the current transaction. + #[prost(uint64, tag = "25")] + pub txn_source: u64, + /// If `busy_threshold_ms` is given, TiKV can reject the request and return a `ServerIsBusy` + /// error before processing if the estimated waiting duration exceeds the threshold. + #[prost(uint32, tag = "27")] + pub busy_threshold_ms: u32, + /// Some information used for resource control. + #[prost(message, optional, tag = "28")] + pub resource_control_context: ::core::option::Option, + /// The keyspace that the request is sent to. + /// NOTE: This field is only meaningful while the api_version is V2. + #[prost(uint32, tag = "32")] + pub keyspace_id: u32, + /// The buckets version that the request is sent to. + /// NOTE: This field is only meaningful while enable buckets. + #[prost(uint64, tag = "33")] + pub buckets_version: u64, + /// It tells us where the request comes from in TiDB. If it isn't from TiDB, leave it blank. + /// This is for tests only and thus can be safely changed/removed without affecting compatibility. + #[prost(message, optional, tag = "34")] + pub source_stmt: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResourceControlContext { + /// It's used to identify which resource group the request belongs to. + #[prost(string, tag = "1")] + pub resource_group_name: ::prost::alloc::string::String, + /// The resource consumption of the resource group that have completed at all TiKVs between the previous request to this TiKV and current request. + /// It's used as penalty to make the local resource scheduling on one TiKV takes the gloabl resource consumption into consideration. + #[prost(message, optional, tag = "2")] + pub penalty: ::core::option::Option, + /// This priority would override the original priority of the resource group for the request. + /// Used to deprioritize the runaway queries. + #[prost(uint64, tag = "3")] + pub override_priority: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SourceStmt { + #[prost(uint64, tag = "1")] + pub start_ts: u64, + #[prost(uint64, tag = "2")] + pub connection_id: u64, + #[prost(uint64, tag = "3")] + pub stmt_id: u64, + /// session alias set by user + #[prost(string, tag = "4")] + pub session_alias: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1050,6 +1303,11 @@ pub struct LockInfo { pub min_commit_ts: u64, #[prost(bytes = "vec", repeated, tag = "10")] pub secondaries: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, + /// The time elapsed since last update of lock wait info when waiting. + /// It's used in timeout errors. 0 means unknown or not applicable. + /// It can be used to help the client decide whether to try resolving the lock. + #[prost(uint64, tag = "11")] + pub duration_to_last_update_ms: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1084,6 +1342,9 @@ pub struct KeyError { /// Assertion of a `Mutation` is evaluated as a failure. #[prost(message, optional, tag = "10")] pub assertion_failed: ::core::option::Option, + /// CheckTxnStatus is sent to a lock that's not the primary. + #[prost(message, optional, tag = "11")] + pub primary_mismatch: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1098,6 +1359,64 @@ pub struct WriteConflict { pub primary: ::prost::alloc::vec::Vec, #[prost(uint64, tag = "5")] pub conflict_commit_ts: u64, + #[prost(enumeration = "write_conflict::Reason", tag = "6")] + pub reason: i32, +} +/// Nested message and enum types in `WriteConflict`. +pub mod write_conflict { + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + ::prost::Enumeration + )] + #[repr(i32)] + pub enum Reason { + Unknown = 0, + /// in optimistic transactions. + Optimistic = 1, + /// a lock acquisition request waits for a lock and awakes, or meets a newer version of data, let TiDB retry. + PessimisticRetry = 2, + /// the transaction itself has been rolled back when it tries to prewrite. + SelfRolledBack = 3, + /// RcCheckTs failure by meeting a newer version, let TiDB retry. + RcCheckTs = 4, + /// write conflict found in lazy uniqueness check in pessimistic transactions. + LazyUniquenessCheck = 5, + } + impl Reason { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + Reason::Unknown => "Unknown", + Reason::Optimistic => "Optimistic", + Reason::PessimisticRetry => "PessimisticRetry", + Reason::SelfRolledBack => "SelfRolledBack", + Reason::RcCheckTs => "RcCheckTs", + Reason::LazyUniquenessCheck => "LazyUniquenessCheck", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "Unknown" => Some(Self::Unknown), + "Optimistic" => Some(Self::Optimistic), + "PessimisticRetry" => Some(Self::PessimisticRetry), + "SelfRolledBack" => Some(Self::SelfRolledBack), + "RcCheckTs" => Some(Self::RcCheckTs), + "LazyUniquenessCheck" => Some(Self::LazyUniquenessCheck), + _ => None, + } + } + } } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1160,21 +1479,54 @@ pub struct AssertionFailed { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct PrimaryMismatch { + #[prost(message, optional, tag = "1")] + pub lock_info: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct TimeDetail { /// Off-cpu wall time elapsed in TiKV side. Usually this includes queue waiting time and - /// other kind of waitings in series. - #[prost(int64, tag = "1")] - pub wait_wall_time_ms: i64, + /// other kind of waitings in series. (Wait time in the raftstore is not included.) + #[prost(uint64, tag = "1")] + pub wait_wall_time_ms: u64, /// Off-cpu and on-cpu wall time elapsed to actually process the request payload. It does not /// include `wait_wall_time`. /// This field is very close to the CPU time in most cases. Some wait time spend in RocksDB /// cannot be excluded for now, like Mutex wait time, which is included in this field, so that /// this field is called wall time instead of CPU time. - #[prost(int64, tag = "2")] - pub process_wall_time_ms: i64, + #[prost(uint64, tag = "2")] + pub process_wall_time_ms: u64, /// KV read wall Time means the time used in key/value scan and get. - #[prost(int64, tag = "3")] - pub kv_read_wall_time_ms: i64, + #[prost(uint64, tag = "3")] + pub kv_read_wall_time_ms: u64, + /// Total wall clock time spent on this RPC in TiKV . + #[prost(uint64, tag = "4")] + pub total_rpc_wall_time_ns: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TimeDetailV2 { + /// Off-cpu wall time elapsed in TiKV side. Usually this includes queue waiting time and + /// other kind of waitings in series. (Wait time in the raftstore is not included.) + #[prost(uint64, tag = "1")] + pub wait_wall_time_ns: u64, + /// Off-cpu and on-cpu wall time elapsed to actually process the request payload. It does not + /// include `wait_wall_time` and `suspend_wall_time`. + /// This field is very close to the CPU time in most cases. Some wait time spend in RocksDB + /// cannot be excluded for now, like Mutex wait time, which is included in this field, so that + /// this field is called wall time instead of CPU time. + #[prost(uint64, tag = "2")] + pub process_wall_time_ns: u64, + /// Cpu wall time elapsed that task is waiting in queue. + #[prost(uint64, tag = "3")] + pub process_suspend_wall_time_ns: u64, + /// KV read wall Time means the time used in key/value scan and get. + #[prost(uint64, tag = "4")] + pub kv_read_wall_time_ns: u64, + /// Total wall clock time spent on this RPC in TiKV . + #[prost(uint64, tag = "5")] + pub total_rpc_wall_time_ns: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1234,6 +1586,21 @@ pub struct ScanDetailV2 { /// Total number of bytes from block reads. #[prost(uint64, tag = "7")] pub rocksdb_block_read_byte: u64, + /// Total time used for block reads. + #[prost(uint64, tag = "9")] + pub rocksdb_block_read_nanos: u64, + /// Time used for getting a raftstore snapshot (including proposing read index, leader confirmation and getting the RocksDB snapshot). + #[prost(uint64, tag = "10")] + pub get_snapshot_nanos: u64, + /// Time used for proposing read index from read pool to store pool, equals 0 when performing lease read. + #[prost(uint64, tag = "11")] + pub read_index_propose_wait_nanos: u64, + /// Time used for leader confirmation, equals 0 when performing lease read. + #[prost(uint64, tag = "12")] + pub read_index_confirm_wait_nanos: u64, + /// Time used for read pool scheduling. + #[prost(uint64, tag = "13")] + pub read_pool_schedule_wait_nanos: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1249,11 +1616,73 @@ pub struct ExecDetails { #[derive(Clone, PartialEq, ::prost::Message)] pub struct ExecDetailsV2 { /// Available when ctx.record_time_stat = true or meet slow query. + /// deprecated. Should use `time_detail_v2` instead. #[prost(message, optional, tag = "1")] pub time_detail: ::core::option::Option, /// Available when ctx.record_scan_stat = true or meet slow query. #[prost(message, optional, tag = "2")] pub scan_detail_v2: ::core::option::Option, + /// Raftstore writing durations of the request. Only available for some write requests. + #[prost(message, optional, tag = "3")] + pub write_detail: ::core::option::Option, + /// Available when ctx.record_time_stat = true or meet slow query. + #[prost(message, optional, tag = "4")] + pub time_detail_v2: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WriteDetail { + /// Wait duration in the store loop. + #[prost(uint64, tag = "1")] + pub store_batch_wait_nanos: u64, + /// Wait duration before sending proposal to peers. + #[prost(uint64, tag = "2")] + pub propose_send_wait_nanos: u64, + /// Total time spent on persisting the log. + #[prost(uint64, tag = "3")] + pub persist_log_nanos: u64, + /// Wait time until the Raft log write leader begins to write. + #[prost(uint64, tag = "4")] + pub raft_db_write_leader_wait_nanos: u64, + /// Time spent on synchronizing the Raft log to the disk. + #[prost(uint64, tag = "5")] + pub raft_db_sync_log_nanos: u64, + /// Time spent on writing the Raft log to the Raft memtable. + #[prost(uint64, tag = "6")] + pub raft_db_write_memtable_nanos: u64, + /// Time waiting for peers to confirm the proposal (counting from the instant when the leader sends the proposal message). + #[prost(uint64, tag = "7")] + pub commit_log_nanos: u64, + /// Wait duration in the apply loop. + #[prost(uint64, tag = "8")] + pub apply_batch_wait_nanos: u64, + /// Total time spend to applying the log. + #[prost(uint64, tag = "9")] + pub apply_log_nanos: u64, + /// Wait time until the KV RocksDB lock is acquired. + #[prost(uint64, tag = "10")] + pub apply_mutex_lock_nanos: u64, + /// Wait time until becoming the KV RocksDB write leader. + #[prost(uint64, tag = "11")] + pub apply_write_leader_wait_nanos: u64, + /// Time spent on writing the KV DB WAL to the disk. + #[prost(uint64, tag = "12")] + pub apply_write_wal_nanos: u64, + /// Time spent on writing to the memtable of the KV RocksDB. + #[prost(uint64, tag = "13")] + pub apply_write_memtable_nanos: u64, + /// Time spent on waiting in the latch. + #[prost(uint64, tag = "14")] + pub latch_wait_nanos: u64, + /// Processing time in the transaction layer. + #[prost(uint64, tag = "15")] + pub process_nanos: u64, + /// Wait time because of the scheduler flow control or quota limiter throttling. + #[prost(uint64, tag = "16")] + pub throttle_nanos: u64, + /// Wait time in the waiter manager for pessimistic locking. + #[prost(uint64, tag = "17")] + pub pessimistic_lock_wait_nanos: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1294,6 +1723,10 @@ pub struct MvccWrite { pub has_gc_fence: bool, #[prost(uint64, tag = "7")] pub gc_fence: u64, + #[prost(uint64, tag = "8")] + pub last_change_ts: u64, + #[prost(uint64, tag = "9")] + pub versions_to_last_change: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1326,6 +1759,10 @@ pub struct MvccLock { pub secondaries: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, #[prost(uint64, repeated, tag = "10")] pub rollback_ts: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "11")] + pub last_change_ts: u64, + #[prost(uint64, tag = "12")] + pub versions_to_last_change: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1480,6 +1917,24 @@ pub struct GetLockWaitInfoResponse { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetLockWaitHistoryRequest { + /// TODO: There may need some filter options to be used on conditional querying, e.g., finding + /// the lock waiting status for some specified transaction. + #[prost(message, optional, tag = "1")] + pub context: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetLockWaitHistoryResponse { + #[prost(message, optional, tag = "1")] + pub region_error: ::core::option::Option, + #[prost(string, tag = "2")] + pub error: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "3")] + pub entries: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct RawCoprocessorRequest { #[prost(message, optional, tag = "1")] pub context: ::core::option::Option, @@ -1529,20 +1984,171 @@ pub struct RawChecksumResponse { #[prost(uint64, tag = "5")] pub total_bytes: u64, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CompactError { + #[prost(oneof = "compact_error::Error", tags = "1, 2, 3, 4")] + pub error: ::core::option::Option, +} +/// Nested message and enum types in `CompactError`. +pub mod compact_error { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Error { + #[prost(message, tag = "1")] + ErrInvalidStartKey(super::CompactErrorInvalidStartKey), + #[prost(message, tag = "2")] + ErrPhysicalTableNotExist(super::CompactErrorPhysicalTableNotExist), + #[prost(message, tag = "3")] + ErrCompactInProgress(super::CompactErrorCompactInProgress), + #[prost(message, tag = "4")] + ErrTooManyPendingTasks(super::CompactErrorTooManyPendingTasks), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CompactErrorInvalidStartKey {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CompactErrorPhysicalTableNotExist {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CompactErrorCompactInProgress {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CompactErrorTooManyPendingTasks {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CompactRequest { + /// If specified, the compaction will start from this start key. + /// If unspecified, the compaction will start from beginning. + /// NOTE 1: The start key should be never manually constructed. You should always use a key + /// returned in CompactResponse. + /// NOTE 2: the compaction range will be always restricted by physical_table_id. + #[prost(bytes = "vec", tag = "1")] + pub start_key: ::prost::alloc::vec::Vec, + /// The physical table that will be compacted. + /// + /// TODO: this is information that TiKV doesn't need to know. + /// See + #[prost(int64, tag = "2")] + pub physical_table_id: i64, + /// The logical table id of the compaction. When receiving parallel requests with the same + /// logical table id, err_compact_in_progress will be returned. + /// + /// TODO: this is information that TiKV doesn't need to know. + /// See + #[prost(int64, tag = "3")] + pub logical_table_id: i64, + /// API version of the request + #[prost(enumeration = "ApiVersion", tag = "7")] + pub api_version: i32, + /// Keyspace of the table located in. + #[prost(uint32, tag = "8")] + pub keyspace_id: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CompactResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + /// The compaction is done incrementally. If there are more data to compact, this field + /// will be set. The client can request to compact more data according to the `compacted_end_key`. + #[prost(bool, tag = "2")] + pub has_remaining: bool, + #[prost(bytes = "vec", tag = "3")] + pub compacted_start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "4")] + pub compacted_end_key: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TiFlashSystemTableRequest { + #[prost(string, tag = "1")] + pub sql: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TiFlashSystemTableResponse { + #[prost(bytes = "vec", tag = "1")] + pub data: ::prost::alloc::vec::Vec, +} +/// Used to specify the behavior when a pessimistic lock request is woken up after waiting for another +/// lock. +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum PessimisticLockWakeUpMode { + /// When woken up, returns WriteConflict error to the client and the client should retry if necessary. + /// In this mode, results of `return_values` or `check_existence` will be set to `values` and `not_founds` + /// fields of the PessimisticLockResponse, which is compatible with old versions. + WakeUpModeNormal = 0, + /// When woken up, continue trying to lock the key. This implicitly enables the `allow_lock_with_conflict` + /// behavior, which means, allow acquiring the lock even if there is WriteConflict on the key. + /// In this mode, `return_values` or `check_existence` fields of PessimisticLockResponse won't be used, and + /// all results are carried in the `results` field. + WakeUpModeForceLock = 1, +} +impl PessimisticLockWakeUpMode { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + PessimisticLockWakeUpMode::WakeUpModeNormal => "WakeUpModeNormal", + PessimisticLockWakeUpMode::WakeUpModeForceLock => "WakeUpModeForceLock", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "WakeUpModeNormal" => Some(Self::WakeUpModeNormal), + "WakeUpModeForceLock" => Some(Self::WakeUpModeForceLock), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum PessimisticLockKeyResultType { + LockResultNormal = 0, + LockResultLockedWithConflict = 1, + LockResultFailed = 2, +} +impl PessimisticLockKeyResultType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + PessimisticLockKeyResultType::LockResultNormal => "LockResultNormal", + PessimisticLockKeyResultType::LockResultLockedWithConflict => { + "LockResultLockedWithConflict" + } + PessimisticLockKeyResultType::LockResultFailed => "LockResultFailed", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "LockResultNormal" => Some(Self::LockResultNormal), + "LockResultLockedWithConflict" => Some(Self::LockResultLockedWithConflict), + "LockResultFailed" => Some(Self::LockResultFailed), + _ => None, + } + } +} /// The API version the server and the client is using. /// See more details in #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum ApiVersion { - /// Mainly for TxnKV and not safe to use RawKV along with TxnKV. - /// - /// V1 server only accepts V1 requests. Except that the V1 raw requests with TTL - /// will be rejected. + /// `V1` is mainly for TiDB & TxnKV, and is not safe to use RawKV along with the others. + /// V1 server only accepts V1 requests. V1 raw requests with TTL will be rejected. V1 = 0, - /// Only RawKV is available, and then 8 bytes representing the unix timestamp in - /// seconds for expiring time will be append to the value of all RawKV kv pairs. - /// - /// --- + /// ## `V1TTL` is only available to RawKV, and 8 bytes representing the unix timestamp in + /// seconds for expiring time will be append to the value of all RawKV entries. For example: /// /// ## \| User value | Expire Ts | /// @@ -1551,31 +2157,40 @@ pub enum ApiVersion { /// V1TTL server only accepts V1 raw requests. /// V1 client should not use `V1TTL` in request. V1 client should always send `V1`. V1ttl = 1, - /// TxnKV keys start with `x{keyspace id}`, `m`, or `t`. + /// `V2` use new encoding for RawKV & TxnKV to support more features. /// - /// RawKV keys must be in `default` CF and all start with `r{keyspace id}` prefix, - /// where the keyspace id is in varint format (little endian), whose bytes expect - /// the last one always sets the most significant bit to 1. + /// Key Encoding: + /// TiDB: start with `m` or `t`, the same as `V1`. + /// TxnKV: prefix with `x`, encoded as `MCE( x{keyspace id} + {user key} ) + timestamp`. + /// RawKV: prefix with `r`, encoded as `MCE( r{keyspace id} + {user key} ) + timestamp`. + /// Where the `{keyspace id}` is fixed-length of 3 bytes in network byte order. + /// Besides, RawKV entires must be in `default` CF. /// - /// The last byte in the raw value must be a meta flag. For example: - /// - /// --- + /// ## Value Encoding: + /// TiDB & TxnKV: the same as `V1`. + /// RawKV: `{user value} + {optional fields} + {meta flag}`. The last byte in the + /// raw value must be meta flags. For example: /// /// ## \| User value | Meta flags | /// /// ## \| 0x12 0x34 0x56 | 0x00 (0b00000000) | /// - /// As shown in the example below, the least significant bit of the meta flag - /// indicates whether the value contains 8 bytes expire ts at the very left to the - /// meta flags. + /// ## Bit 0 of meta flags is for TTL. If set, the value contains 8 bytes expiring time as + /// unix timestamp in seconds at the very left to the meta flags. /// - /// --- - /// - /// ## \| User value | Expire Ts | Meta flags | + /// ## \| User value | Expiring time | Meta flags | /// /// ## \| 0x12 0x34 0x56 | 0x00 0x00 0x00 0x00 0x00 0x00 0xff 0xff | 0x01 (0b00000001) | /// - /// V2 server accpets V2 requests and V1 txn requests that statrts with TiDB key + /// ## Bit 1 is for deletion. If set, the entry is logical deleted. + /// + /// |Meta flags| + /// |----------| + /// |0x02 (0b00000010)| + /// + /// --- + /// + /// V2 server accpets V2 requests and V1 transactional requests that statrts with TiDB key /// prefix (`m` and `t`). V2 = 2, } @@ -1638,6 +2253,8 @@ pub enum IsolationLevel { Si = 0, /// RC = read committed Rc = 1, + /// RC read and it's needed to check if there exists more recent versions. + RcCheckTs = 2, } impl IsolationLevel { /// String value of the enum field names used in the ProtoBuf definition. @@ -1648,6 +2265,7 @@ impl IsolationLevel { match self { IsolationLevel::Si => "SI", IsolationLevel::Rc => "RC", + IsolationLevel::RcCheckTs => "RCCheckTS", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1655,6 +2273,7 @@ impl IsolationLevel { match value { "SI" => Some(Self::Si), "RC" => Some(Self::Rc), + "RCCheckTS" => Some(Self::RcCheckTs), _ => None, } } diff --git a/src/generated/logbackup.rs b/src/generated/logbackup.rs new file mode 100644 index 0000000..51b3cf5 --- /dev/null +++ b/src/generated/logbackup.rs @@ -0,0 +1,199 @@ +// This file is @generated by prost-build. +/// The minimal information for identify a region. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RegionIdentity { + #[prost(uint64, tag = "1")] + pub id: u64, + /// We omitted epoch_conf_version because config change won't make range change. + #[prost(uint64, tag = "2")] + pub epoch_version: u64, +} +/// The last flush ts with region information. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RegionCheckpoint { + #[prost(message, optional, tag = "1")] + pub err: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub region: ::core::option::Option, + #[prost(uint64, tag = "3")] + pub checkpoint: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetLastFlushTsOfRegionRequest { + #[prost(message, repeated, tag = "1")] + pub regions: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetLastFlushTsOfRegionResponse { + #[prost(message, repeated, tag = "1")] + pub checkpoints: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SubscribeFlushEventRequest { + #[prost(string, tag = "1")] + pub client_id: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SubscribeFlushEventResponse { + #[prost(message, repeated, tag = "1")] + pub events: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlushEvent { + #[prost(bytes = "vec", tag = "1")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "2")] + pub end_key: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "3")] + pub checkpoint: u64, +} +/// Generated client implementations. +pub mod log_backup_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + /// The log backup service. + /// Generally, most essential interfaces of log backup (say, checkpoint management, task management) are + /// provided by adding some key in the embed etcd of PD. + /// This interface is mainly provided for the checkpoint advancer and debug usage. + #[derive(Debug, Clone)] + pub struct LogBackupClient { + inner: tonic::client::Grpc, + } + impl LogBackupClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl LogBackupClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> LogBackupClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + LogBackupClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn get_last_flush_ts_of_region( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/logbackup.LogBackup/GetLastFlushTSOfRegion", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new("logbackup.LogBackup", "GetLastFlushTSOfRegion"), + ); + self.inner.unary(req, path, codec).await + } + pub async fn subscribe_flush_event( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/logbackup.LogBackup/SubscribeFlushEvent", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("logbackup.LogBackup", "SubscribeFlushEvent")); + self.inner.server_streaming(req, path, codec).await + } + } +} diff --git a/src/generated/meta_storagepb.rs b/src/generated/meta_storagepb.rs new file mode 100644 index 0000000..1355ebd --- /dev/null +++ b/src/generated/meta_storagepb.rs @@ -0,0 +1,383 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Error { + #[prost(enumeration = "ErrorType", tag = "1")] + pub r#type: i32, + #[prost(string, tag = "2")] + pub message: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RequestHeader { + /// cluster_id is the ID of the cluster which be sent to. + #[prost(uint64, tag = "1")] + pub cluster_id: u64, + /// source is the source of the request. + #[prost(string, tag = "2")] + pub source: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResponseHeader { + /// cluster_id is the ID of the cluster which sent the response. + #[prost(uint64, tag = "1")] + pub cluster_id: u64, + #[prost(message, optional, tag = "2")] + pub error: ::core::option::Option, + #[prost(int64, tag = "3")] + pub revision: i64, +} +/// copied part of +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WatchRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "3")] + pub range_end: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "4")] + pub start_revision: i64, + #[prost(bool, tag = "5")] + pub prev_kv: bool, +} +/// copied part of +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WatchResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(int64, tag = "2")] + pub compact_revision: i64, + #[prost(message, repeated, tag = "3")] + pub events: ::prost::alloc::vec::Vec, +} +/// copied part of +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "3")] + pub range_end: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "4")] + pub limit: i64, + #[prost(int64, tag = "5")] + pub revision: i64, +} +/// copied part of +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub kvs: ::prost::alloc::vec::Vec, + #[prost(bool, tag = "3")] + pub more: bool, + #[prost(int64, tag = "4")] + pub count: i64, +} +/// copied part of +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PutRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "3")] + pub value: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "4")] + pub lease: i64, + #[prost(bool, tag = "5")] + pub prev_kv: bool, +} +/// copied part of +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PutResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub prev_kv: ::core::option::Option, +} +/// copied from etcd +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct KeyValue { + /// key is the key in bytes. An empty key is not allowed. + #[prost(bytes = "vec", tag = "1")] + pub key: ::prost::alloc::vec::Vec, + /// create_revision is the revision of last creation on this key. + #[prost(int64, tag = "2")] + pub create_revision: i64, + /// mod_revision is the revision of last modification on this key. + #[prost(int64, tag = "3")] + pub mod_revision: i64, + /// version is the version of the key. A deletion resets + /// the version to zero and any modification of the key + /// increases its version. + #[prost(int64, tag = "4")] + pub version: i64, + /// value is the value held by the key, in bytes. + #[prost(bytes = "vec", tag = "5")] + pub value: ::prost::alloc::vec::Vec, + /// lease is the ID of the lease that attached to key. + /// When the attached lease expires, the key will be deleted. + /// If lease is 0, then no lease is attached to the key. + #[prost(int64, tag = "6")] + pub lease: i64, +} +/// copied from etcd +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Event { + /// type is the kind of event. If type is a PUT, it indicates + /// new data has been stored to the key. If type is a DELETE, + /// it indicates the key was deleted. + #[prost(enumeration = "event::EventType", tag = "1")] + pub r#type: i32, + /// kv holds the KeyValue for the event. + /// A PUT event contains current kv pair. + /// A PUT event with kv.Version=1 indicates the creation of a key. + /// A DELETE/EXPIRE event contains the deleted key with + /// its modification revision set to the revision of deletion. + #[prost(message, optional, tag = "2")] + pub kv: ::core::option::Option, + /// prev_kv holds the key-value pair before the event happens. + #[prost(message, optional, tag = "3")] + pub prev_kv: ::core::option::Option, +} +/// Nested message and enum types in `Event`. +pub mod event { + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + ::prost::Enumeration + )] + #[repr(i32)] + pub enum EventType { + Put = 0, + Delete = 1, + } + impl EventType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + EventType::Put => "PUT", + EventType::Delete => "DELETE", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "PUT" => Some(Self::Put), + "DELETE" => Some(Self::Delete), + _ => None, + } + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ErrorType { + Ok = 0, + Unknown = 1, + /// required watch revision is smaller than current compact/min revision. + DataCompacted = 2, +} +impl ErrorType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ErrorType::Ok => "OK", + ErrorType::Unknown => "UNKNOWN", + ErrorType::DataCompacted => "DATA_COMPACTED", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "OK" => Some(Self::Ok), + "UNKNOWN" => Some(Self::Unknown), + "DATA_COMPACTED" => Some(Self::DataCompacted), + _ => None, + } + } +} +/// Generated client implementations. +pub mod meta_storage_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + /// MetaStorage is the meta storage service. + #[derive(Debug, Clone)] + pub struct MetaStorageClient { + inner: tonic::client::Grpc, + } + impl MetaStorageClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl MetaStorageClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> MetaStorageClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + MetaStorageClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn watch( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/meta_storagepb.MetaStorage/Watch", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("meta_storagepb.MetaStorage", "Watch")); + self.inner.server_streaming(req, path, codec).await + } + /// Get is the same as etcd Range which might be implemented in a more common way + /// so that we can use other storages to replace etcd in the future. + pub async fn get( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/meta_storagepb.MetaStorage/Get", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("meta_storagepb.MetaStorage", "Get")); + self.inner.unary(req, path, codec).await + } + pub async fn put( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/meta_storagepb.MetaStorage/Put", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("meta_storagepb.MetaStorage", "Put")); + self.inner.unary(req, path, codec).await + } + } +} diff --git a/src/generated/metapb.rs b/src/generated/metapb.rs index 3cec809..8839fb0 100644 --- a/src/generated/metapb.rs +++ b/src/generated/metapb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Cluster { @@ -53,6 +54,9 @@ pub struct Store { /// If the store is physically destroyed, which means it can never up again. #[prost(bool, tag = "12")] pub physically_destroyed: bool, + /// NodeState is used to replace StoreState which will be deprecated in the future. + #[prost(enumeration = "NodeState", tag = "13")] + pub node_state: i32, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -66,6 +70,46 @@ pub struct RegionEpoch { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct BucketStats { + /// total read in bytes of each bucket + #[prost(uint64, repeated, tag = "1")] + pub read_bytes: ::prost::alloc::vec::Vec, + /// total write in bytes of each bucket + #[prost(uint64, repeated, tag = "2")] + pub write_bytes: ::prost::alloc::vec::Vec, + /// total read qps of each bucket + #[prost(uint64, repeated, tag = "3")] + pub read_qps: ::prost::alloc::vec::Vec, + /// total write qps of each bucket + #[prost(uint64, repeated, tag = "4")] + pub write_qps: ::prost::alloc::vec::Vec, + /// total read keys of each bucket + #[prost(uint64, repeated, tag = "5")] + pub read_keys: ::prost::alloc::vec::Vec, + /// total write keys of each bucket + #[prost(uint64, repeated, tag = "6")] + pub write_keys: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Buckets { + #[prost(uint64, tag = "1")] + pub region_id: u64, + /// A hint indicate if keys have changed. + #[prost(uint64, tag = "2")] + pub version: u64, + /// keys of buckets, include start/end key of region + #[prost(bytes = "vec", repeated, tag = "3")] + pub keys: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, + /// bucket stats + #[prost(message, optional, tag = "4")] + pub stats: ::core::option::Option, + /// The period in milliseconds that stats are collected with in + #[prost(uint64, tag = "5")] + pub period_in_ms: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct Region { #[prost(uint64, tag = "1")] pub id: u64, @@ -84,6 +128,13 @@ pub struct Region { /// If encryption_meta is empty (i.e. nil), it means start_key and end_key are unencrypted. #[prost(message, optional, tag = "6")] pub encryption_meta: ::core::option::Option, + /// The flashback state indicates whether this region is in the flashback state. + /// TODO: only check by `flashback_start_ts` in the future. Keep for compatibility now. + #[prost(bool, tag = "7")] + pub is_in_flashback: bool, + /// The start_ts that the current flashback progress is using. + #[prost(uint64, tag = "8")] + pub flashback_start_ts: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -94,6 +145,8 @@ pub struct Peer { pub store_id: u64, #[prost(enumeration = "PeerRole", tag = "3")] pub role: i32, + #[prost(bool, tag = "4")] + pub is_witness: bool, } #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] @@ -124,6 +177,42 @@ impl StoreState { } } } +/// NodeState is going to replace StoreState to make the state concept more clear. +/// "Up" is devided into "Preparing" and "Serving" stages so that we can better describe the online process. +/// "Removing" is just like previous `Offline` which is more accurate. +/// "Removed" has the same meaning with `Tombstone`. +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum NodeState { + Preparing = 0, + Serving = 1, + Removing = 2, + Removed = 3, +} +impl NodeState { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + NodeState::Preparing => "Preparing", + NodeState::Serving => "Serving", + NodeState::Removing => "Removing", + NodeState::Removed => "Removed", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "Preparing" => Some(Self::Preparing), + "Serving" => Some(Self::Serving), + "Removing" => Some(Self::Removing), + "Removed" => Some(Self::Removed), + _ => None, + } + } +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum PeerRole { diff --git a/src/generated/mod.rs b/src/generated/mod.rs index 09a476d..da3c98e 100644 --- a/src/generated/mod.rs +++ b/src/generated/mod.rs @@ -1,3 +1,7 @@ +// This file is @generated by prost-build. +pub mod autoid { + include!("autoid.rs"); +} pub mod backup { include!("backup.rs"); } @@ -19,6 +23,9 @@ pub mod debugpb { pub mod diagnosticspb { include!("diagnosticspb.rs"); } +pub mod disaggregated { + include!("disaggregated.rs"); +} pub mod disk_usage { include!("disk_usage.rs"); } @@ -34,6 +41,9 @@ pub mod eraftpb { pub mod errorpb { include!("errorpb.rs"); } +pub mod gcpb { + include!("gcpb.rs"); +} pub mod google { pub mod api { include!("google.api.rs"); @@ -45,9 +55,18 @@ pub mod import_kvpb { pub mod import_sstpb { include!("import_sstpb.rs"); } +pub mod keyspacepb { + include!("keyspacepb.rs"); +} pub mod kvrpcpb { include!("kvrpcpb.rs"); } +pub mod logbackup { + include!("logbackup.rs"); +} +pub mod meta_storagepb { + include!("meta_storagepb.rs"); +} pub mod metapb { include!("metapb.rs"); } @@ -63,15 +82,27 @@ pub mod raft_cmdpb { pub mod raft_serverpb { include!("raft_serverpb.rs"); } +pub mod recover_data { + include!("recover_data.rs"); +} pub mod replication_modepb { include!("replication_modepb.rs"); } +pub mod resource_manager { + include!("resource_manager.rs"); +} pub mod resource_usage_agent { include!("resource_usage_agent.rs"); } -pub mod span { - include!("span.rs"); +pub mod schedulingpb { + include!("schedulingpb.rs"); } pub mod tikvpb { include!("tikvpb.rs"); } +pub mod tracepb { + include!("tracepb.rs"); +} +pub mod tsopb { + include!("tsopb.rs"); +} diff --git a/src/generated/mpp.rs b/src/generated/mpp.rs index 8834980..e68ab94 100644 --- a/src/generated/mpp.rs +++ b/src/generated/mpp.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// TaskMeta contains meta of a mpp plan, including query's ts and task address. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -14,6 +15,41 @@ pub struct TaskMeta { /// target address of this task. #[prost(string, tag = "4")] pub address: ::prost::alloc::string::String, + /// used to distinguish different gathers in the mpp query. + #[prost(uint64, tag = "5")] + pub gather_id: u64, + /// timestamp when start to execute query, used for TiFlash miniTSO schedule. + #[prost(uint64, tag = "6")] + pub query_ts: u64, + /// unique local query_id if tidb don't restart. So we can use gather_id + query_ts + local_query_id + server_id to represent a global unique query. + #[prost(uint64, tag = "7")] + pub local_query_id: u64, + /// TiDB server id + #[prost(uint64, tag = "8")] + pub server_id: u64, + /// mpp version + #[prost(int64, tag = "9")] + pub mpp_version: i64, + /// keyspace id of the request + #[prost(uint32, tag = "10")] + pub keyspace_id: u32, + /// coordinator_address of this query + #[prost(string, tag = "11")] + pub coordinator_address: ::prost::alloc::string::String, + /// Only when coordinator_address is not empty, this flag can be true. When set to true, TiFlash only report execution summary through ReportMPPTaskStatus service, don't include summaries in MppDataPacket + #[prost(bool, tag = "12")] + pub report_execution_summary: bool, + /// API version of the request + #[prost(enumeration = "super::kvrpcpb::ApiVersion", tag = "16")] + pub api_version: i32, + #[prost(string, tag = "17")] + pub resource_group_name: ::prost::alloc::string::String, + /// This is the session id between a client and tidb + #[prost(uint64, tag = "18")] + pub connection_id: u64, + /// This is the session alias between a client and tidb + #[prost(string, tag = "19")] + pub connection_alias: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -23,6 +59,8 @@ pub struct IsAliveRequest {} pub struct IsAliveResponse { #[prost(bool, tag = "1")] pub available: bool, + #[prost(int64, tag = "2")] + pub mpp_version: i64, } /// Dipsatch the task request to different tiflash servers. #[allow(clippy::derive_partial_eq_without_eq)] @@ -39,6 +77,9 @@ pub struct DispatchTaskRequest { /// If this task contains table scan, we still need their region info. #[prost(int64, tag = "5")] pub schema_ver: i64, + /// Used for partition table scan + #[prost(message, repeated, tag = "6")] + pub table_regions: ::prost::alloc::vec::Vec, } /// Get response of DispatchTaskRequest. #[allow(clippy::derive_partial_eq_without_eq)] @@ -64,6 +105,24 @@ pub struct CancelTaskResponse { #[prost(message, optional, tag = "1")] pub error: ::core::option::Option, } +/// ReportTaskStatus reports the execution status of a task. +/// when TiFlash reports status to TiDB, ReportTaskStatusRequest serialize tipb.TiFlashExecutionInfo into data; +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReportTaskStatusRequest { + #[prost(message, optional, tag = "1")] + pub meta: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub data: ::prost::alloc::vec::Vec, + #[prost(message, optional, tag = "3")] + pub error: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReportTaskStatusResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, +} /// build connection between different tasks. Data is sent by the tasks that are closer to the data sources. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -86,6 +145,11 @@ pub struct MppDataPacket { pub error: ::core::option::Option, #[prost(bytes = "vec", repeated, tag = "3")] pub chunks: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, + #[prost(uint64, repeated, tag = "4")] + pub stream_ids: ::prost::alloc::vec::Vec, + /// version of data packet format + #[prost(int64, tag = "5")] + pub version: i64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -94,4 +158,6 @@ pub struct Error { pub code: i32, #[prost(string, tag = "2")] pub msg: ::prost::alloc::string::String, + #[prost(int64, tag = "3")] + pub mpp_version: i64, } diff --git a/src/generated/pdpb.rs b/src/generated/pdpb.rs index 0755789..548b333 100644 --- a/src/generated/pdpb.rs +++ b/src/generated/pdpb.rs @@ -1,3 +1,69 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WatchGlobalConfigRequest { + #[prost(string, tag = "1")] + pub config_path: ::prost::alloc::string::String, + #[prost(int64, tag = "2")] + pub revision: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WatchGlobalConfigResponse { + #[prost(message, repeated, tag = "1")] + pub changes: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "2")] + pub revision: i64, + #[prost(message, optional, tag = "3")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StoreGlobalConfigRequest { + #[prost(message, repeated, tag = "1")] + pub changes: ::prost::alloc::vec::Vec, + #[prost(string, tag = "2")] + pub config_path: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StoreGlobalConfigResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct LoadGlobalConfigRequest { + #[prost(string, repeated, tag = "1")] + pub names: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(string, tag = "2")] + pub config_path: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct LoadGlobalConfigResponse { + #[prost(message, repeated, tag = "1")] + pub items: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "2")] + pub revision: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GlobalConfigItem { + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + /// this field 'value' is replaced by the field 'payload'. + #[prost(string, tag = "2")] + pub value: ::prost::alloc::string::String, + #[prost(message, optional, tag = "3")] + pub error: ::core::option::Option, + #[prost(enumeration = "EventType", tag = "4")] + pub kind: i32, + /// Since item value needs to support marshal of different struct types, + /// it should be set to bytes instead of string. + #[prost(bytes = "vec", tag = "5")] + pub payload: ::prost::alloc::vec::Vec, +} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct RequestHeader { @@ -107,6 +173,20 @@ pub struct AllocIdResponse { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct IsSnapshotRecoveringRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct IsSnapshotRecoveringResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bool, tag = "2")] + pub marked: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct GetStoreRequest { #[prost(message, optional, tag = "1")] pub header: ::core::option::Option, @@ -165,6 +245,8 @@ pub struct GetRegionRequest { pub header: ::core::option::Option, #[prost(bytes = "vec", tag = "2")] pub region_key: ::prost::alloc::vec::Vec, + #[prost(bool, tag = "3")] + pub need_buckets: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -182,6 +264,9 @@ pub struct GetRegionResponse { /// working followers. #[prost(message, repeated, tag = "6")] pub pending_peers: ::prost::alloc::vec::Vec, + /// buckets isn't nil if GetRegion.\* requests set need_buckets. + #[prost(message, optional, tag = "7")] + pub buckets: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -190,6 +275,8 @@ pub struct GetRegionByIdRequest { pub header: ::core::option::Option, #[prost(uint64, tag = "2")] pub region_id: u64, + #[prost(bool, tag = "3")] + pub need_buckets: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -311,6 +398,24 @@ pub struct GetMembersResponse { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetClusterInfoRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetClusterInfoResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(enumeration = "ServiceMode", repeated, tag = "2")] + pub service_modes: ::prost::alloc::vec::Vec, + /// If service mode is API_SVC_MODE, this field will be set to the + /// registered tso service addresses. + #[prost(string, repeated, tag = "3")] + pub tso_urls: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct PeerStats { #[prost(message, optional, tag = "1")] pub peer: ::core::option::Option, @@ -367,6 +472,10 @@ pub struct RegionHeartbeatRequest { /// which is calculated by cpu_time_delta/heartbeat_reported_interval. #[prost(uint64, tag = "17")] pub cpu_usage: u64, + /// (Serverless) Approximate size of key-value pairs for billing. + /// It's counted on size of user key & value (excluding metadata fields), before compression, and latest versions only. + #[prost(uint64, tag = "18")] + pub approximate_kv_size: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -407,6 +516,20 @@ pub struct SplitRegion { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct SwitchWitness { + #[prost(uint64, tag = "1")] + pub peer_id: u64, + #[prost(bool, tag = "2")] + pub is_witness: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct BatchSwitchWitness { + #[prost(message, repeated, tag = "1")] + pub switch_witnesses: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct RegionHeartbeatResponse { #[prost(message, optional, tag = "1")] pub header: ::core::option::Option, @@ -446,9 +569,13 @@ pub struct RegionHeartbeatResponse { /// Multiple change peer operations atomically. /// Note: PD can use both ChangePeer and ChangePeerV2 at the same time /// (not in the same RegionHeartbeatResponse). - /// Now, PD use ChangePeerV2 only for replacing peers. + /// Now, PD use ChangePeerV2 in following scenarios: + /// 1. replacing peers + /// 2. demoting voter directly #[prost(message, optional, tag = "9")] pub change_peer_v2: ::core::option::Option, + #[prost(message, optional, tag = "10")] + pub switch_witnesses: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -557,6 +684,10 @@ pub struct PeerStat { pub read_bytes: u64, #[prost(message, optional, tag = "4")] pub query_stats: ::core::option::Option, + #[prost(uint64, tag = "5")] + pub written_keys: u64, + #[prost(uint64, tag = "6")] + pub written_bytes: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -629,6 +760,55 @@ pub struct StoreStats { /// Damaged regions on the store that need to be removed by PD. #[prost(uint64, repeated, tag = "23")] pub damaged_regions_id: ::prost::alloc::vec::Vec, + /// If the apply worker is busy, namely high apply wait duration + #[prost(bool, tag = "24")] + pub is_apply_busy: bool, + /// Snapshot stats in the store + #[prost(message, repeated, tag = "25")] + pub snapshot_stats: ::prost::alloc::vec::Vec, + #[prost(message, optional, tag = "26")] + pub slow_trend: ::core::option::Option, + /// If the grpc server is paused. + #[prost(bool, tag = "27")] + pub is_grpc_paused: bool, + /// Total memory of the store in bytes. + #[prost(uint64, tag = "28")] + pub total_memory: u64, + /// Used memory of the store in bytes. + #[prost(uint64, tag = "29")] + pub used_memory: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SlowTrend { + #[prost(double, tag = "1")] + pub cause_value: f64, + #[prost(double, tag = "2")] + pub cause_rate: f64, + #[prost(double, tag = "3")] + pub result_value: f64, + #[prost(double, tag = "4")] + pub result_rate: f64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SnapshotStat { + #[prost(uint64, tag = "1")] + pub region_id: u64, + /// Generate snapshot duration + #[prost(uint64, tag = "2")] + pub generate_duration_sec: u64, + /// Send snapshot duration + #[prost(uint64, tag = "3")] + pub send_duration_sec: u64, + /// \|-- waiting --|-- generate --|-- send --| + /// \|-----------total duration---------------| + /// Total duration include waiting and executing duration + #[prost(uint64, tag = "4")] + pub total_duration_sec: u64, + /// Size is the transport data size + #[prost(uint64, tag = "5")] + pub transport_size: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -637,12 +817,19 @@ pub struct PeerReport { pub raft_state: ::core::option::Option, #[prost(message, optional, tag = "2")] pub region_state: ::core::option::Option, + #[prost(bool, tag = "3")] + pub is_force_leader: bool, + /// The peer has proposed but uncommitted commit merge. + #[prost(bool, tag = "4")] + pub has_commit_merge: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct StoreReport { #[prost(message, repeated, tag = "1")] pub peer_reports: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "2")] + pub step: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -651,19 +838,68 @@ pub struct StoreHeartbeatRequest { pub header: ::core::option::Option, #[prost(message, optional, tag = "2")] pub stats: ::core::option::Option, - /// Detailed store report that is only filled up on PD's demand for online unsafe recover. + /// Detailed store report that is only filled up on PD's demand for online unsafe recovery. #[prost(message, optional, tag = "3")] pub store_report: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub dr_autosync_status: ::core::option::Option< + super::replication_modepb::StoreDrAutoSyncStatus, + >, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DemoteFailedVoters { + #[prost(uint64, tag = "1")] + pub region_id: u64, + #[prost(message, repeated, tag = "2")] + pub failed_voters: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ForceLeader { + /// The store ids of the failed stores, TiKV uses it to decide if a peer is alive. + #[prost(uint64, repeated, tag = "1")] + pub failed_stores: ::prost::alloc::vec::Vec, + /// The region ids of the peer which is to be force leader. + #[prost(uint64, repeated, tag = "2")] + pub enter_force_leaders: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct RecoveryPlan { + /// Create empty regions to fill the key range hole. #[prost(message, repeated, tag = "1")] pub creates: ::prost::alloc::vec::Vec, + /// Update the meta of the regions, including peer lists, epoch and key range. + #[deprecated] #[prost(message, repeated, tag = "2")] pub updates: ::prost::alloc::vec::Vec, + /// Tombstone the peers on the store locally. #[prost(uint64, repeated, tag = "3")] - pub deletes: ::prost::alloc::vec::Vec, + pub tombstones: ::prost::alloc::vec::Vec, + /// Issue conf change that demote voters on failed stores to learners on the regions. + #[prost(message, repeated, tag = "4")] + pub demotes: ::prost::alloc::vec::Vec, + /// Make the peers to be force leaders. + #[prost(message, optional, tag = "5")] + pub force_leader: ::core::option::Option, + /// Step is an increasing number to note the round of recovery, + /// It should be filled in the corresponding store report. + #[prost(uint64, tag = "6")] + pub step: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AwakenRegions { + /// Awake all regions if abnormal_stores is empty. + #[prost(uint64, repeated, tag = "1")] + pub abnormal_stores: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ControlGrpc { + #[prost(enumeration = "ControlGrpcEvent", tag = "1")] + pub ctrl_event: i32, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -676,10 +912,22 @@ pub struct StoreHeartbeatResponse { >, #[prost(string, tag = "3")] pub cluster_version: ::prost::alloc::string::String, + /// Used by online unsafe recovery to request store report. + /// Now it's substituted by reusing recovery_plan field. PD will send a empty + /// recovery plan instead to request store report. + #[deprecated] #[prost(bool, tag = "4")] pub require_detailed_report: bool, + /// Operations of recovery. After the plan is executed, TiKV should attach the + /// store report in store heartbeat. #[prost(message, optional, tag = "5")] - pub plan: ::core::option::Option, + pub recovery_plan: ::core::option::Option, + /// Pd can return awaken_regions to let TiKV awaken hibernated regions itself. + #[prost(message, optional, tag = "6")] + pub awaken_regions: ::core::option::Option, + /// Pd can return operations to let TiKV forcely PAUSE | RESUME grpc server. + #[prost(message, optional, tag = "7")] + pub control_grpc: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -704,6 +952,8 @@ pub struct ScatterRegionRequest { pub regions_id: ::prost::alloc::vec::Vec, #[prost(uint64, tag = "7")] pub retry_limit: u64, + #[prost(bool, tag = "8")] + pub skip_store_limit: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -769,6 +1019,124 @@ pub struct UpdateServiceGcSafePointResponse { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetGcSafePointV2Request { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint32, tag = "2")] + pub keyspace_id: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetGcSafePointV2Response { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub safe_point: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WatchGcSafePointV2Request { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(int64, tag = "2")] + pub revision: i64, +} +/// SafePointEvent is for the rpc WatchGCSafePointV2. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SafePointEvent { + #[prost(uint32, tag = "1")] + pub keyspace_id: u32, + #[prost(uint64, tag = "2")] + pub safe_point: u64, + #[prost(enumeration = "EventType", tag = "3")] + pub r#type: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WatchGcSafePointV2Response { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub events: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "3")] + pub revision: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateGcSafePointV2Request { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint32, tag = "2")] + pub keyspace_id: u32, + #[prost(uint64, tag = "3")] + pub safe_point: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateGcSafePointV2Response { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub new_safe_point: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateServiceSafePointV2Request { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint32, tag = "2")] + pub keyspace_id: u32, + #[prost(bytes = "vec", tag = "3")] + pub service_id: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "4")] + pub safe_point: u64, + /// Safe point will be set to expire on (PD Server time + TTL), + /// pass in a ttl \< 0 to remove target safe point; + /// pass in MAX_INT64 to set a safe point that never expire. + /// This should be set by component that may crash unexpectedly so that it doesn't block + /// cluster garbage collection. + #[prost(int64, tag = "5")] + pub ttl: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateServiceSafePointV2Response { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bytes = "vec", tag = "2")] + pub service_id: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "3")] + pub ttl: i64, + #[prost(uint64, tag = "4")] + pub min_safe_point: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetAllGcSafePointV2Request { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GcSafePointV2 { + #[prost(uint32, tag = "1")] + pub keyspace_id: u32, + #[prost(uint64, tag = "2")] + pub gc_safe_point: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetAllGcSafePointV2Response { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub gc_safe_points: ::prost::alloc::vec::Vec, + #[prost(int64, tag = "3")] + pub revision: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct RegionStat { /// Bytes read/written during this period. #[prost(uint64, tag = "1")] @@ -795,6 +1163,18 @@ pub struct SyncRegionRequest { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct PeersStats { + #[prost(message, repeated, tag = "1")] + pub peers: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Peers { + #[prost(message, repeated, tag = "1")] + pub peers: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct SyncRegionResponse { #[prost(message, optional, tag = "1")] pub header: ::core::option::Option, @@ -808,6 +1188,13 @@ pub struct SyncRegionResponse { pub region_stats: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "5")] pub region_leaders: ::prost::alloc::vec::Vec, + /// the buckets informations without stats. + #[prost(message, repeated, tag = "6")] + pub buckets: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "16")] + pub down_peers: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "17")] + pub pending_peers: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -943,6 +1330,106 @@ pub struct QueryStats { #[prost(uint64, tag = "11")] pub rollback: u64, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReportBucketsRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub region_epoch: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub buckets: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReportBucketsResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReportMinResolvedTsRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub store_id: u64, + #[prost(uint64, tag = "3")] + pub min_resolved_ts: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReportMinResolvedTsResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SetExternalTimestampRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub timestamp: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SetExternalTimestampResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetExternalTimestampRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetExternalTimestampResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub timestamp: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetMinTsRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetMinTsResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub timestamp: ::core::option::Option, +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum EventType { + Put = 0, + Delete = 1, +} +impl EventType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + EventType::Put => "PUT", + EventType::Delete => "DELETE", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "PUT" => Some(Self::Put), + "DELETE" => Some(Self::Delete), + _ => None, + } + } +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum ErrorType { @@ -953,6 +1440,12 @@ pub enum ErrorType { AlreadyBootstrapped = 4, IncompatibleVersion = 5, RegionNotFound = 6, + GlobalConfigNotFound = 7, + DuplicatedEntry = 8, + EntryNotFound = 9, + InvalidValue = 10, + /// required watch revision is smaller than current compact/min revision. + DataCompacted = 11, } impl ErrorType { /// String value of the enum field names used in the ProtoBuf definition. @@ -968,6 +1461,11 @@ impl ErrorType { ErrorType::AlreadyBootstrapped => "ALREADY_BOOTSTRAPPED", ErrorType::IncompatibleVersion => "INCOMPATIBLE_VERSION", ErrorType::RegionNotFound => "REGION_NOT_FOUND", + ErrorType::GlobalConfigNotFound => "GLOBAL_CONFIG_NOT_FOUND", + ErrorType::DuplicatedEntry => "DUPLICATED_ENTRY", + ErrorType::EntryNotFound => "ENTRY_NOT_FOUND", + ErrorType::InvalidValue => "INVALID_VALUE", + ErrorType::DataCompacted => "DATA_COMPACTED", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -980,6 +1478,40 @@ impl ErrorType { "ALREADY_BOOTSTRAPPED" => Some(Self::AlreadyBootstrapped), "INCOMPATIBLE_VERSION" => Some(Self::IncompatibleVersion), "REGION_NOT_FOUND" => Some(Self::RegionNotFound), + "GLOBAL_CONFIG_NOT_FOUND" => Some(Self::GlobalConfigNotFound), + "DUPLICATED_ENTRY" => Some(Self::DuplicatedEntry), + "ENTRY_NOT_FOUND" => Some(Self::EntryNotFound), + "INVALID_VALUE" => Some(Self::InvalidValue), + "DATA_COMPACTED" => Some(Self::DataCompacted), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ServiceMode { + UnknownSvcMode = 0, + PdSvcMode = 1, + ApiSvcMode = 2, +} +impl ServiceMode { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ServiceMode::UnknownSvcMode => "UNKNOWN_SVC_MODE", + ServiceMode::PdSvcMode => "PD_SVC_MODE", + ServiceMode::ApiSvcMode => "API_SVC_MODE", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "UNKNOWN_SVC_MODE" => Some(Self::UnknownSvcMode), + "PD_SVC_MODE" => Some(Self::PdSvcMode), + "API_SVC_MODE" => Some(Self::ApiSvcMode), _ => None, } } @@ -1015,6 +1547,34 @@ impl CheckPolicy { } #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] +pub enum ControlGrpcEvent { + /// Pause TiKV grpc server. + Pause = 0, + /// Resume TiKV grpc server. + Resume = 1, +} +impl ControlGrpcEvent { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ControlGrpcEvent::Pause => "PAUSE", + ControlGrpcEvent::Resume => "RESUME", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "PAUSE" => Some(Self::Pause), + "RESUME" => Some(Self::Resume), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] pub enum OperatorStatus { Success = 0, Timeout = 1, @@ -1189,6 +1749,30 @@ pub mod pd_client { self.inner = self.inner.max_encoding_message_size(limit); self } + /// GetClusterInfo get the information of this cluster. It does not require + /// the cluster_id in request matchs the id of this cluster. + pub async fn get_cluster_info( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/pdpb.PD/GetClusterInfo"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "GetClusterInfo")); + self.inner.unary(req, path, codec).await + } /// GetMembers get the member list of this cluster. It does not require /// the cluster_id in request matchs the id of this cluster. pub async fn get_members( @@ -1301,6 +1885,31 @@ pub mod pd_client { req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "AllocID")); self.inner.unary(req, path, codec).await } + pub async fn is_snapshot_recovering( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/IsSnapshotRecovering", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("pdpb.PD", "IsSnapshotRecovering")); + self.inner.unary(req, path, codec).await + } pub async fn get_store( &mut self, request: impl tonic::IntoRequest, @@ -1726,6 +2335,128 @@ pub mod pd_client { .insert(GrpcMethod::new("pdpb.PD", "UpdateServiceGCSafePoint")); self.inner.unary(req, path, codec).await } + pub async fn get_gc_safe_point_v2( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/pdpb.PD/GetGCSafePointV2"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "GetGCSafePointV2")); + self.inner.unary(req, path, codec).await + } + pub async fn watch_gc_safe_point_v2( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/WatchGCSafePointV2", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("pdpb.PD", "WatchGCSafePointV2")); + self.inner.server_streaming(req, path, codec).await + } + pub async fn update_gc_safe_point_v2( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/UpdateGCSafePointV2", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("pdpb.PD", "UpdateGCSafePointV2")); + self.inner.unary(req, path, codec).await + } + pub async fn update_service_safe_point_v2( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/UpdateServiceSafePointV2", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("pdpb.PD", "UpdateServiceSafePointV2")); + self.inner.unary(req, path, codec).await + } + pub async fn get_all_gc_safe_point_v2( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/GetAllGCSafePointV2", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("pdpb.PD", "GetAllGCSafePointV2")); + self.inner.unary(req, path, codec).await + } pub async fn sync_regions( &mut self, request: impl tonic::IntoStreamingRequest, @@ -1863,5 +2594,199 @@ pub mod pd_client { req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "GetDCLocationInfo")); self.inner.unary(req, path, codec).await } + pub async fn store_global_config( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/StoreGlobalConfig", + ); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "StoreGlobalConfig")); + self.inner.unary(req, path, codec).await + } + pub async fn load_global_config( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/pdpb.PD/LoadGlobalConfig"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "LoadGlobalConfig")); + self.inner.unary(req, path, codec).await + } + pub async fn watch_global_config( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/WatchGlobalConfig", + ); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "WatchGlobalConfig")); + self.inner.server_streaming(req, path, codec).await + } + pub async fn report_buckets( + &mut self, + request: impl tonic::IntoStreamingRequest< + Message = super::ReportBucketsRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/pdpb.PD/ReportBuckets"); + let mut req = request.into_streaming_request(); + req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "ReportBuckets")); + self.inner.client_streaming(req, path, codec).await + } + pub async fn report_min_resolved_ts( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/ReportMinResolvedTS", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("pdpb.PD", "ReportMinResolvedTS")); + self.inner.unary(req, path, codec).await + } + pub async fn set_external_timestamp( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/SetExternalTimestamp", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("pdpb.PD", "SetExternalTimestamp")); + self.inner.unary(req, path, codec).await + } + pub async fn get_external_timestamp( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/pdpb.PD/GetExternalTimestamp", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("pdpb.PD", "GetExternalTimestamp")); + self.inner.unary(req, path, codec).await + } + /// Get the minimum timestamp across all keyspace groups from API server + /// TODO: Currently, we need to ask API server to get the minimum timestamp. + /// Once we support service discovery, we can remove it. + pub async fn get_min_ts( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/pdpb.PD/GetMinTS"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("pdpb.PD", "GetMinTS")); + self.inner.unary(req, path, codec).await + } } } diff --git a/src/generated/raft_cmdpb.rs b/src/generated/raft_cmdpb.rs index da10882..eb17807 100644 --- a/src/generated/raft_cmdpb.rs +++ b/src/generated/raft_cmdpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetRequest { @@ -192,6 +193,12 @@ pub struct SplitRequest { #[deprecated] #[prost(bool, tag = "4")] pub right_derive: bool, + /// It should be false iff the region split by user key such as split table or create partion table etc, + /// the new region's will not share the source region size, so it's region size is zero. + /// It should be true iff the region's load reaches the threshold such as size, keys, load check etc, + /// the new region's size will share the origin region, so it's region size is half of the source region. + #[prost(bool, tag = "5")] + pub share_source_region_size: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -210,6 +217,12 @@ pub struct BatchSplitRequest { /// other regions use new ids. #[prost(bool, tag = "2")] pub right_derive: bool, + /// It should be false iff the region split by user key such as split table or create partion table etc, + /// the new region's will not share the source region size, so it's region size is zero. + /// It should be true iff the region's load reaches the threshold such as size, keys, load check etc, + /// the new region's size will share the origin region, so it's region size is half of the source region. + #[prost(bool, tag = "3")] + pub share_source_region_size: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -224,6 +237,8 @@ pub struct CompactLogRequest { pub compact_index: u64, #[prost(uint64, tag = "2")] pub compact_term: u64, + #[prost(uint64, tag = "3")] + pub voter_replicated_index: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -271,6 +286,22 @@ pub struct PrepareMergeRequest { pub struct PrepareMergeResponse {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct PrepareFlashbackRequest { + /// The start_ts that the current flashback progress is using. + #[prost(uint64, tag = "1")] + pub start_ts: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PrepareFlashbackResponse {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FinishFlashbackRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FinishFlashbackResponse {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct CommitMergeRequest { #[prost(message, optional, tag = "1")] pub source: ::core::option::Option, @@ -278,6 +309,9 @@ pub struct CommitMergeRequest { pub commit: u64, #[prost(message, repeated, tag = "3")] pub entries: ::prost::alloc::vec::Vec, + /// Used in v2. When it's present, `source` and `commit` will not be set. + #[prost(message, optional, tag = "4")] + pub source_state: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -293,6 +327,29 @@ pub struct RollbackMergeRequest { pub struct RollbackMergeResponse {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct SwitchWitnessRequest { + #[prost(uint64, tag = "1")] + pub peer_id: u64, + #[prost(bool, tag = "2")] + pub is_witness: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct BatchSwitchWitnessRequest { + #[prost(message, repeated, tag = "1")] + pub switch_witnesses: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct BatchSwitchWitnessResponse {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateGcPeerRequest { + #[prost(uint64, repeated, tag = "1")] + pub peer_id: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct AdminRequest { #[prost(enumeration = "AdminCmdType", tag = "1")] pub cmd_type: i32, @@ -319,6 +376,14 @@ pub struct AdminRequest { pub change_peer_v2: ::core::option::Option, #[prost(message, optional, tag = "12")] pub compute_hash: ::core::option::Option, + #[prost(message, optional, tag = "13")] + pub prepare_flashback: ::core::option::Option, + #[prost(message, optional, tag = "14")] + pub finish_flashback: ::core::option::Option, + #[prost(message, optional, tag = "15")] + pub switch_witnesses: ::core::option::Option, + #[prost(message, optional, tag = "16")] + pub update_gc_peers: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -346,6 +411,13 @@ pub struct AdminResponse { pub splits: ::core::option::Option, #[prost(message, optional, tag = "11")] pub change_peer_v2: ::core::option::Option, + #[prost(message, optional, tag = "12")] + pub prepare_flashback: ::core::option::Option, + #[prost(message, optional, tag = "13")] + pub finish_flashback: ::core::option::Option, + /// UpdateGcPeer doesn't need to be responded. Avoid consuming a tag number. + #[prost(message, optional, tag = "14")] + pub switch_witnesses: ::core::option::Option, } /// For get the leader of the region. #[allow(clippy::derive_partial_eq_without_eq)] @@ -426,6 +498,10 @@ pub struct RaftRequestHeader { pub flags: u64, #[prost(bytes = "vec", tag = "11")] pub flag_data: ::prost::alloc::vec::Vec, + #[prost(enumeration = "super::kvrpcpb::CommandPri", tag = "12")] + pub priority: i32, + #[prost(string, tag = "13")] + pub resource_group_name: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -526,6 +602,11 @@ pub enum AdminCmdType { RollbackMerge = 9, BatchSplit = 10, ChangePeerV2 = 11, + PrepareFlashback = 12, + FinishFlashback = 13, + BatchSwitchWitness = 14, + /// Command that updates RegionLocalState.gc_peers + UpdateGcPeer = 15, } impl AdminCmdType { /// String value of the enum field names used in the ProtoBuf definition. @@ -546,6 +627,10 @@ impl AdminCmdType { AdminCmdType::RollbackMerge => "RollbackMerge", AdminCmdType::BatchSplit => "BatchSplit", AdminCmdType::ChangePeerV2 => "ChangePeerV2", + AdminCmdType::PrepareFlashback => "PrepareFlashback", + AdminCmdType::FinishFlashback => "FinishFlashback", + AdminCmdType::BatchSwitchWitness => "BatchSwitchWitness", + AdminCmdType::UpdateGcPeer => "UpdateGcPeer", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -563,6 +648,10 @@ impl AdminCmdType { "RollbackMerge" => Some(Self::RollbackMerge), "BatchSplit" => Some(Self::BatchSplit), "ChangePeerV2" => Some(Self::ChangePeerV2), + "PrepareFlashback" => Some(Self::PrepareFlashback), + "FinishFlashback" => Some(Self::FinishFlashback), + "BatchSwitchWitness" => Some(Self::BatchSwitchWitness), + "UpdateGcPeer" => Some(Self::UpdateGcPeer), _ => None, } } diff --git a/src/generated/raft_serverpb.rs b/src/generated/raft_serverpb.rs index e273d8c..f6b1f94 100644 --- a/src/generated/raft_serverpb.rs +++ b/src/generated/raft_serverpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct RaftMessage { @@ -55,6 +56,24 @@ pub struct SnapshotMeta { /// true means this snapshot is triggered for load balance #[prost(bool, tag = "2")] pub for_balance: bool, + /// true means this is an empty snapshot for witness + #[prost(bool, tag = "3")] + pub for_witness: bool, + /// the timestamp second to generate snapshot + #[prost(uint64, tag = "4")] + pub start: u64, + /// the duration of generating snapshot + #[prost(uint64, tag = "5")] + pub generate_duration_sec: u64, + /// the path of the tablet snapshot, it should only be used for v1 to receive + /// snapshot from v2 + #[prost(string, tag = "6")] + pub tablet_snap_path: ::prost::alloc::string::String, + /// A hint of the latest commit index on leader when sending snapshot. + /// It should only be used for v2 to send snapshot to v1. + /// See + #[prost(uint64, tag = "7")] + pub commit_index_hint: u64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -69,6 +88,99 @@ pub struct SnapshotChunk { pub struct Done {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct TabletSnapshotFileMeta { + #[prost(uint64, tag = "1")] + pub file_size: u64, + #[prost(string, tag = "2")] + pub file_name: ::prost::alloc::string::String, + /// Some block data. Unencrypted. + #[prost(bytes = "vec", tag = "3")] + pub head_chunk: ::prost::alloc::vec::Vec, + /// trailing data including checksum. Unencrypted. + #[prost(bytes = "vec", tag = "4")] + pub trailing_chunk: ::prost::alloc::vec::Vec, +} +/// Snapshot preview for server to decide whether skip some files. +/// Server should send back an `AcceptedSnapshotFile` to let client +/// keep sending specified files. Only SST files can be skipped, all +/// other files should always be sent. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TabletSnapshotPreview { + #[prost(message, repeated, tag = "1")] + pub metas: ::prost::alloc::vec::Vec, + /// There may be too many metas, use a flag to indicate all metas + /// are sent. + #[prost(bool, tag = "2")] + pub end: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TabletSnapshotFileChunk { + #[prost(uint64, tag = "1")] + pub file_size: u64, + #[prost(string, tag = "2")] + pub file_name: ::prost::alloc::string::String, + /// Encrypted. + #[prost(bytes = "vec", tag = "3")] + pub data: ::prost::alloc::vec::Vec, + /// Initial vector if encryption is enabled. + #[prost(bytes = "vec", tag = "4")] + pub iv: ::prost::alloc::vec::Vec, + #[prost(message, optional, tag = "5")] + pub key: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TabletSnapshotHead { + #[prost(message, optional, tag = "1")] + pub message: ::core::option::Option, + #[prost(bool, tag = "2")] + pub use_cache: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TabletSnapshotEnd { + /// Checksum of all data sent in `TabletSnapshotFileChunk.data` and + /// `TabletSnapshotFileChunk.file_name`. + #[prost(uint64, tag = "1")] + pub checksum: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TabletSnapshotRequest { + #[prost(oneof = "tablet_snapshot_request::Payload", tags = "1, 2, 3, 4")] + pub payload: ::core::option::Option, +} +/// Nested message and enum types in `TabletSnapshotRequest`. +pub mod tablet_snapshot_request { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Payload { + #[prost(message, tag = "1")] + Head(super::TabletSnapshotHead), + #[prost(message, tag = "2")] + Preview(super::TabletSnapshotPreview), + #[prost(message, tag = "3")] + Chunk(super::TabletSnapshotFileChunk), + #[prost(message, tag = "4")] + End(super::TabletSnapshotEnd), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AcceptedSnapshotFiles { + #[prost(string, repeated, tag = "1")] + pub file_name: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TabletSnapshotResponse { + #[prost(message, optional, tag = "1")] + pub files: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct KeyValue { #[prost(bytes = "vec", tag = "1")] pub key: ::prost::alloc::vec::Vec, @@ -88,6 +200,10 @@ pub struct RaftSnapshotData { pub version: u64, #[prost(message, optional, tag = "5")] pub meta: ::core::option::Option, + #[prost(message, repeated, tag = "6")] + pub removed_records: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "7")] + pub merged_records: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -101,6 +217,17 @@ pub struct StoreIdent { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct StoreRecoverState { + /// Used for TiKV start recovery when WAL of KVDB was disabled. + /// TiKV may read all relations between seqno and raft log index, and replay + /// all raft logs which corresponding seqno smaller than the seqno here. + /// After TiKV replays all raft logs and flushed KV data, the seqno here must + /// be updated. + #[prost(uint64, tag = "1")] + pub seqno: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct RaftLocalState { #[prost(message, optional, tag = "1")] pub hard_state: ::core::option::Option, @@ -133,6 +260,32 @@ pub struct MergeState { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct MergedRecord { + #[prost(uint64, tag = "1")] + pub source_region_id: u64, + #[prost(message, optional, tag = "2")] + pub source_epoch: ::core::option::Option, + /// Peers of source region when merge is committed. + #[prost(message, repeated, tag = "3")] + pub source_peers: ::prost::alloc::vec::Vec, + /// Removed peers (by confchange) of source region when merge is committed. + #[prost(message, repeated, tag = "9")] + pub source_removed_records: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "4")] + pub target_region_id: u64, + #[prost(message, optional, tag = "5")] + pub target_epoch: ::core::option::Option, + #[prost(message, repeated, tag = "6")] + pub target_peers: ::prost::alloc::vec::Vec, + /// Commit merge index. + #[prost(uint64, tag = "7")] + pub index: u64, + /// Prepare merge index. + #[prost(uint64, tag = "8")] + pub source_index: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct RegionLocalState { #[prost(enumeration = "PeerState", tag = "1")] pub state: i32, @@ -140,16 +293,105 @@ pub struct RegionLocalState { pub region: ::core::option::Option, #[prost(message, optional, tag = "3")] pub merge_state: ::core::option::Option, + /// The apply index corresponding to the storage when it's initialized. + #[prost(uint64, tag = "4")] + pub tablet_index: u64, + /// Raft doesn't guarantee peer will be removed in the end. In v1, peer finds + /// out its destiny by logs or broadcast; in v2, leader is responsible to + /// ensure removed peers are destroyed. + /// Note: only peers who has been part of this region can be in this list. + #[prost(message, repeated, tag = "5")] + pub removed_records: ::prost::alloc::vec::Vec, + /// Merged peer can't be deleted like gc peers. Instead, leader needs to + /// query target peer to decide whether source peer can be destroyed. + #[prost(message, repeated, tag = "6")] + pub merged_records: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RegionSequenceNumberRelation { + #[prost(uint64, tag = "1")] + pub region_id: u64, + #[prost(uint64, tag = "2")] + pub sequence_number: u64, + #[prost(message, optional, tag = "3")] + pub apply_state: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub region_state: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AvailabilityContext { + #[prost(uint64, tag = "1")] + pub from_region_id: u64, + #[prost(message, optional, tag = "2")] + pub from_region_epoch: ::core::option::Option, + #[prost(bool, tag = "3")] + pub unavailable: bool, + #[prost(bool, tag = "4")] + pub trimmed: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FlushMemtable { + #[prost(uint64, tag = "1")] + pub region_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RefreshBuckets { + #[prost(uint64, tag = "1")] + pub version: u64, + #[prost(bytes = "vec", repeated, tag = "2")] + pub keys: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, + #[prost(uint64, repeated, tag = "3")] + pub sizes: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CheckGcPeer { + /// The region ID who triggers the check and wait for report. It should be + /// the ID of RaftMessage.from. + #[prost(uint64, tag = "1")] + pub from_region_id: u64, + /// The region ID to be checked if should be destroyed. + #[prost(uint64, tag = "2")] + pub check_region_id: u64, + /// The epoch of the region to be checked. + #[prost(message, optional, tag = "3")] + pub check_region_epoch: ::core::option::Option, + /// The peer to be checked. + #[prost(message, optional, tag = "4")] + pub check_peer: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ExtraMessage { #[prost(enumeration = "ExtraMessageType", tag = "1")] pub r#type: i32, + /// It's merge related index. In `WantRollbackMerge`, it's prepare merge index. In + /// `MsgGcPeerRequest`, it's the commit merge index. In `MsgVoterReplicatedIndexRequest` + /// it's the voter_replicated_index. #[prost(uint64, tag = "2")] - pub premerge_commit: u64, + pub index: u64, + /// In `MsgCheckStalePeerResponse`, it's the peers that receiver can continue to query. #[prost(message, repeated, tag = "3")] pub check_peers: ::prost::alloc::vec::Vec, + #[prost(bool, tag = "4")] + pub wait_data: bool, + /// Flag for forcely wake up hibernate regions if true. + #[prost(bool, tag = "5")] + pub forcely_awaken: bool, + #[prost(message, optional, tag = "6")] + pub check_gc_peer: ::core::option::Option, + #[prost(message, optional, tag = "7")] + pub flush_memtable: ::core::option::Option, + /// Used by `MsgAvailabilityRequest` and `MsgAvailabilityResponse` in v2. + #[prost(message, optional, tag = "8")] + pub availability_context: ::core::option::Option, + /// notice the peer to refresh buckets version + #[prost(message, optional, tag = "9")] + pub refresh_buckets: ::core::option::Option, } #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] @@ -158,6 +400,11 @@ pub enum PeerState { Applying = 1, Tombstone = 2, Merging = 3, + /// Currently used for witness to non-witness conversion: When a witness + /// has just become a non-witness, we need to set and persist this state, + /// so that when the service restarts before applying snapshot, we can + /// actively request snapshot when initializing this peer. + Unavailable = 4, } impl PeerState { /// String value of the enum field names used in the ProtoBuf definition. @@ -170,6 +417,7 @@ impl PeerState { PeerState::Applying => "Applying", PeerState::Tombstone => "Tombstone", PeerState::Merging => "Merging", + PeerState::Unavailable => "Unavailable", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -179,6 +427,7 @@ impl PeerState { "Applying" => Some(Self::Applying), "Tombstone" => Some(Self::Tombstone), "Merging" => Some(Self::Merging), + "Unavailable" => Some(Self::Unavailable), _ => None, } } @@ -194,6 +443,16 @@ pub enum ExtraMessageType { /// to make sure they all agree to sleep. MsgHibernateRequest = 4, MsgHibernateResponse = 5, + MsgRejectRaftLogCausedByMemoryUsage = 6, + MsgAvailabilityRequest = 7, + MsgAvailabilityResponse = 8, + MsgVoterReplicatedIndexRequest = 9, + MsgVoterReplicatedIndexResponse = 10, + /// Message means that `from` is tombstone. Leader can then update removed_records. + MsgGcPeerRequest = 11, + MsgGcPeerResponse = 12, + MsgFlushMemtable = 13, + MsgRefreshBuckets = 14, } impl ExtraMessageType { /// String value of the enum field names used in the ProtoBuf definition. @@ -208,6 +467,21 @@ impl ExtraMessageType { ExtraMessageType::MsgCheckStalePeerResponse => "MsgCheckStalePeerResponse", ExtraMessageType::MsgHibernateRequest => "MsgHibernateRequest", ExtraMessageType::MsgHibernateResponse => "MsgHibernateResponse", + ExtraMessageType::MsgRejectRaftLogCausedByMemoryUsage => { + "MsgRejectRaftLogCausedByMemoryUsage" + } + ExtraMessageType::MsgAvailabilityRequest => "MsgAvailabilityRequest", + ExtraMessageType::MsgAvailabilityResponse => "MsgAvailabilityResponse", + ExtraMessageType::MsgVoterReplicatedIndexRequest => { + "MsgVoterReplicatedIndexRequest" + } + ExtraMessageType::MsgVoterReplicatedIndexResponse => { + "MsgVoterReplicatedIndexResponse" + } + ExtraMessageType::MsgGcPeerRequest => "MsgGcPeerRequest", + ExtraMessageType::MsgGcPeerResponse => "MsgGcPeerResponse", + ExtraMessageType::MsgFlushMemtable => "MsgFlushMemtable", + ExtraMessageType::MsgRefreshBuckets => "MsgRefreshBuckets", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -219,6 +493,21 @@ impl ExtraMessageType { "MsgCheckStalePeerResponse" => Some(Self::MsgCheckStalePeerResponse), "MsgHibernateRequest" => Some(Self::MsgHibernateRequest), "MsgHibernateResponse" => Some(Self::MsgHibernateResponse), + "MsgRejectRaftLogCausedByMemoryUsage" => { + Some(Self::MsgRejectRaftLogCausedByMemoryUsage) + } + "MsgAvailabilityRequest" => Some(Self::MsgAvailabilityRequest), + "MsgAvailabilityResponse" => Some(Self::MsgAvailabilityResponse), + "MsgVoterReplicatedIndexRequest" => { + Some(Self::MsgVoterReplicatedIndexRequest) + } + "MsgVoterReplicatedIndexResponse" => { + Some(Self::MsgVoterReplicatedIndexResponse) + } + "MsgGcPeerRequest" => Some(Self::MsgGcPeerRequest), + "MsgGcPeerResponse" => Some(Self::MsgGcPeerResponse), + "MsgFlushMemtable" => Some(Self::MsgFlushMemtable), + "MsgRefreshBuckets" => Some(Self::MsgRefreshBuckets), _ => None, } } diff --git a/src/generated/recover_data.rs b/src/generated/recover_data.rs new file mode 100644 index 0000000..3b7adb0 --- /dev/null +++ b/src/generated/recover_data.rs @@ -0,0 +1,292 @@ +// This file is @generated by prost-build. +/// request to read region meata from a store +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReadRegionMetaRequest { + #[prost(uint64, tag = "1")] + pub store_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Error { + #[prost(string, tag = "1")] + pub msg: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RegionMeta { + #[prost(uint64, tag = "1")] + pub region_id: u64, + #[prost(uint64, tag = "2")] + pub peer_id: u64, + #[prost(uint64, tag = "3")] + pub last_log_term: u64, + #[prost(uint64, tag = "4")] + pub last_index: u64, + #[prost(uint64, tag = "5")] + pub commit_index: u64, + #[prost(uint64, tag = "6")] + pub version: u64, + /// reserved, it may be used in late phase for peer check + #[prost(bool, tag = "7")] + pub tombstone: bool, + #[prost(bytes = "vec", tag = "8")] + pub start_key: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "9")] + pub end_key: ::prost::alloc::vec::Vec, +} +/// command to store for recover region +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RecoverRegionRequest { + #[prost(uint64, tag = "1")] + pub region_id: u64, + /// force region_id as leader + #[prost(bool, tag = "2")] + pub as_leader: bool, + /// set Peer to tombstoned in late phase + #[prost(bool, tag = "3")] + pub tombstone: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RecoverRegionResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub store_id: u64, +} +/// wait apply to last index +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WaitApplyRequest { + #[prost(uint64, tag = "1")] + pub store_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WaitApplyResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, +} +/// resolve data by resolved_ts +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResolveKvDataRequest { + #[prost(uint64, tag = "1")] + pub resolved_ts: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResolveKvDataResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub store_id: u64, + /// reserved for summary of restore + #[prost(uint64, tag = "3")] + pub resolved_key_count: u64, + /// cursor of delete key.commit_ts, reserved for progress of restore + /// progress is (current_commit_ts - resolved_ts) / (backup_ts - resolved_ts) x 100% + #[prost(uint64, tag = "4")] + pub current_commit_ts: u64, +} +/// Generated client implementations. +pub mod recover_data_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + /// a recovery workflow likes + /// + /// 1. BR read ReadRegionMeta to get all region meta + /// 1. BR send recover region to tikv, e.g assign leader and wait leader apply to last index + /// 1. BR wait all regions in tikv to apply to last index (no write during the recovery) + /// 1. BR resolved kv data + #[derive(Debug, Clone)] + pub struct RecoverDataClient { + inner: tonic::client::Grpc, + } + impl RecoverDataClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl RecoverDataClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> RecoverDataClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + RecoverDataClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + /// read region meta to ready region meta + pub async fn read_region_meta( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/recover_data.RecoverData/ReadRegionMeta", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("recover_data.RecoverData", "ReadRegionMeta")); + self.inner.server_streaming(req, path, codec).await + } + /// execute the recovery command + pub async fn recover_region( + &mut self, + request: impl tonic::IntoStreamingRequest< + Message = super::RecoverRegionRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/recover_data.RecoverData/RecoverRegion", + ); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert(GrpcMethod::new("recover_data.RecoverData", "RecoverRegion")); + self.inner.client_streaming(req, path, codec).await + } + /// wait all region apply to last index + pub async fn wait_apply( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/recover_data.RecoverData/WaitApply", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("recover_data.RecoverData", "WaitApply")); + self.inner.unary(req, path, codec).await + } + /// execute delete data from kv db + pub async fn resolve_kv_data( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/recover_data.RecoverData/ResolveKvData", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("recover_data.RecoverData", "ResolveKvData")); + self.inner.server_streaming(req, path, codec).await + } + } +} diff --git a/src/generated/replication_modepb.rs b/src/generated/replication_modepb.rs index 1f57b9e..3140fab 100644 --- a/src/generated/replication_modepb.rs +++ b/src/generated/replication_modepb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. /// The replication status sync from PD to TiKV. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -22,6 +23,12 @@ pub struct DrAutoSync { /// Duration to wait before switching to SYNC by force (in seconds) #[prost(int32, tag = "4")] pub wait_sync_timeout_hint: i32, + /// Stores should only sync messages with available stores when state is ASYNC or ASYNC_WAIT. + #[prost(uint64, repeated, tag = "5")] + pub available_stores: ::prost::alloc::vec::Vec, + /// Stores should forbid region split. + #[prost(bool, tag = "6")] + pub pause_region_split: bool, } /// The replication status sync from TiKV to PD. #[allow(clippy::derive_partial_eq_without_eq)] @@ -33,6 +40,14 @@ pub struct RegionReplicationStatus { #[prost(uint64, tag = "2")] pub state_id: u64, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StoreDrAutoSyncStatus { + #[prost(enumeration = "DrAutoSyncState", tag = "1")] + pub state: i32, + #[prost(uint64, tag = "2")] + pub state_id: u64, +} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum ReplicationMode { @@ -66,10 +81,12 @@ impl ReplicationMode { pub enum DrAutoSyncState { /// Raft logs need to sync between different DCs Sync = 0, + /// Wait for switching to ASYNC. Stop sync raft logs between DCs. + AsyncWait = 1, /// Raft logs need to sync to majority peers - Async = 1, + Async = 2, /// Switching from ASYNC to SYNC mode - SyncRecover = 2, + SyncRecover = 3, } impl DrAutoSyncState { /// String value of the enum field names used in the ProtoBuf definition. @@ -79,6 +96,7 @@ impl DrAutoSyncState { pub fn as_str_name(&self) -> &'static str { match self { DrAutoSyncState::Sync => "SYNC", + DrAutoSyncState::AsyncWait => "ASYNC_WAIT", DrAutoSyncState::Async => "ASYNC", DrAutoSyncState::SyncRecover => "SYNC_RECOVER", } @@ -87,6 +105,7 @@ impl DrAutoSyncState { pub fn from_str_name(value: &str) -> ::core::option::Option { match value { "SYNC" => Some(Self::Sync), + "ASYNC_WAIT" => Some(Self::AsyncWait), "ASYNC" => Some(Self::Async), "SYNC_RECOVER" => Some(Self::SyncRecover), _ => None, diff --git a/src/generated/resource_manager.rs b/src/generated/resource_manager.rs new file mode 100644 index 0000000..caf4b11 --- /dev/null +++ b/src/generated/resource_manager.rs @@ -0,0 +1,707 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListResourceGroupsRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListResourceGroupsResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub groups: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetResourceGroupRequest { + #[prost(string, tag = "1")] + pub resource_group_name: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetResourceGroupResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub group: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DeleteResourceGroupRequest { + #[prost(string, tag = "1")] + pub resource_group_name: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DeleteResourceGroupResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(string, tag = "2")] + pub body: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PutResourceGroupRequest { + #[prost(message, optional, tag = "1")] + pub group: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PutResourceGroupResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(string, tag = "2")] + pub body: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TokenBucketsRequest { + #[prost(message, repeated, tag = "1")] + pub requests: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "2")] + pub target_request_period_ms: u64, + #[prost(uint64, tag = "3")] + pub client_unique_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TokenBucketRequest { + #[prost(string, tag = "1")] + pub resource_group_name: ::prost::alloc::string::String, + /// Aggregate statistics in group level. + #[prost(message, optional, tag = "4")] + pub consumption_since_last_request: ::core::option::Option, + /// label background request. + #[prost(bool, tag = "5")] + pub is_background: bool, + #[prost(bool, tag = "6")] + pub is_tiflash: bool, + #[prost(oneof = "token_bucket_request::Request", tags = "2, 3")] + pub request: ::core::option::Option, +} +/// Nested message and enum types in `TokenBucketRequest`. +pub mod token_bucket_request { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct RequestRu { + #[prost(message, repeated, tag = "1")] + pub request_r_u: ::prost::alloc::vec::Vec, + } + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct RequestRawResource { + #[prost(message, repeated, tag = "1")] + pub request_raw_resource: ::prost::alloc::vec::Vec, + } + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Request { + /// RU mode, group settings with WRU/RRU etc resource abstract unit. + #[prost(message, tag = "2")] + RuItems(RequestRu), + /// Raw mode, group settings with CPU/IO etc resource unit. + #[prost(message, tag = "3")] + RawResourceItems(RequestRawResource), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TokenBucketsResponse { + #[prost(message, optional, tag = "1")] + pub error: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub responses: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TokenBucketResponse { + #[prost(string, tag = "1")] + pub resource_group_name: ::prost::alloc::string::String, + /// RU mode + #[prost(message, repeated, tag = "2")] + pub granted_r_u_tokens: ::prost::alloc::vec::Vec, + /// Raw mode + #[prost(message, repeated, tag = "3")] + pub granted_resource_tokens: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GrantedRuTokenBucket { + #[prost(enumeration = "RequestUnitType", tag = "1")] + pub r#type: i32, + #[prost(message, optional, tag = "2")] + pub granted_tokens: ::core::option::Option, + #[prost(int64, tag = "3")] + pub trickle_time_ms: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GrantedRawResourceTokenBucket { + #[prost(enumeration = "RawResourceType", tag = "1")] + pub r#type: i32, + #[prost(message, optional, tag = "2")] + pub granted_tokens: ::core::option::Option, + #[prost(int64, tag = "3")] + pub trickle_time_ms: i64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Consumption { + #[prost(double, tag = "1")] + pub r_r_u: f64, + #[prost(double, tag = "2")] + pub w_r_u: f64, + #[prost(double, tag = "3")] + pub read_bytes: f64, + #[prost(double, tag = "4")] + pub write_bytes: f64, + #[prost(double, tag = "5")] + pub total_cpu_time_ms: f64, + #[prost(double, tag = "6")] + pub sql_layer_cpu_time_ms: f64, + #[prost(double, tag = "7")] + pub kv_read_rpc_count: f64, + #[prost(double, tag = "8")] + pub kv_write_rpc_count: f64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RequestUnitItem { + #[prost(enumeration = "RequestUnitType", tag = "1")] + pub r#type: i32, + #[prost(double, tag = "2")] + pub value: f64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RawResourceItem { + #[prost(enumeration = "RawResourceType", tag = "1")] + pub r#type: i32, + #[prost(double, tag = "2")] + pub value: f64, +} +/// ResourceGroup the settings definitions. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResourceGroup { + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + #[prost(enumeration = "GroupMode", tag = "2")] + pub mode: i32, + /// Used in RU mode, group settings with WRU/RRU etc resource abstract unit. + #[prost(message, optional, tag = "3")] + pub r_u_settings: ::core::option::Option, + /// Used in Raw mode, group settings with CPU/IO etc resource unit. + #[prost(message, optional, tag = "4")] + pub raw_resource_settings: ::core::option::Option, + /// The task scheduling priority + #[prost(uint32, tag = "5")] + pub priority: u32, + /// Runaway queries settings + #[prost(message, optional, tag = "6")] + pub runaway_settings: ::core::option::Option, + #[prost(message, optional, tag = "7")] + pub background_settings: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GroupRequestUnitSettings { + #[prost(message, optional, tag = "1")] + pub r_u: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GroupRawResourceSettings { + #[prost(message, optional, tag = "1")] + pub cpu: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub io_read: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub io_write: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TokenBucket { + #[prost(message, optional, tag = "1")] + pub settings: ::core::option::Option, + /// Once used to reconfigure, the tokens is delta tokens. + #[prost(double, tag = "2")] + pub tokens: f64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TokenLimitSettings { + #[prost(uint64, tag = "1")] + pub fill_rate: u64, + #[prost(int64, tag = "2")] + pub burst_limit: i64, + #[prost(double, tag = "3")] + pub max_tokens: f64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Error { + #[prost(string, tag = "1")] + pub message: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RunawayRule { + #[prost(uint64, tag = "1")] + pub exec_elapsed_time_ms: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RunawayWatch { + /// how long would the watch last + #[prost(int64, tag = "1")] + pub lasting_duration_ms: i64, + #[prost(enumeration = "RunawayWatchType", tag = "2")] + pub r#type: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RunawaySettings { + #[prost(message, optional, tag = "1")] + pub rule: ::core::option::Option, + #[prost(enumeration = "RunawayAction", tag = "2")] + pub action: i32, + #[prost(message, optional, tag = "3")] + pub watch: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct BackgroundSettings { + #[prost(string, repeated, tag = "1")] + pub job_types: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Participant { + /// name is the unique name of the resource manager participant. + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + /// id is the unique id of the resource manager participant. + #[prost(uint64, tag = "2")] + pub id: u64, + /// listen_urls is the serivce endpoint list in the url format. + /// listen_urls\[0\] is primary service endpoint. + #[prost(string, repeated, tag = "3")] + pub listen_urls: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum RequestUnitType { + Ru = 0, +} +impl RequestUnitType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + RequestUnitType::Ru => "RU", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "RU" => Some(Self::Ru), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum RawResourceType { + Cpu = 0, + IoReadFlow = 1, + IoWriteFlow = 2, +} +impl RawResourceType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + RawResourceType::Cpu => "CPU", + RawResourceType::IoReadFlow => "IOReadFlow", + RawResourceType::IoWriteFlow => "IOWriteFlow", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "CPU" => Some(Self::Cpu), + "IOReadFlow" => Some(Self::IoReadFlow), + "IOWriteFlow" => Some(Self::IoWriteFlow), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum GroupMode { + Unknown = 0, + RuMode = 1, + RawMode = 2, +} +impl GroupMode { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + GroupMode::Unknown => "Unknown", + GroupMode::RuMode => "RUMode", + GroupMode::RawMode => "RawMode", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "Unknown" => Some(Self::Unknown), + "RUMode" => Some(Self::RuMode), + "RawMode" => Some(Self::RawMode), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum RunawayAction { + NoneAction = 0, + /// do nothing + DryRun = 1, + /// deprioritize the task + CoolDown = 2, + /// kill the task + Kill = 3, +} +impl RunawayAction { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + RunawayAction::NoneAction => "NoneAction", + RunawayAction::DryRun => "DryRun", + RunawayAction::CoolDown => "CoolDown", + RunawayAction::Kill => "Kill", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "NoneAction" => Some(Self::NoneAction), + "DryRun" => Some(Self::DryRun), + "CoolDown" => Some(Self::CoolDown), + "Kill" => Some(Self::Kill), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum RunawayWatchType { + NoneWatch = 0, + Exact = 1, + Similar = 2, + Plan = 3, +} +impl RunawayWatchType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + RunawayWatchType::NoneWatch => "NoneWatch", + RunawayWatchType::Exact => "Exact", + RunawayWatchType::Similar => "Similar", + RunawayWatchType::Plan => "Plan", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "NoneWatch" => Some(Self::NoneWatch), + "Exact" => Some(Self::Exact), + "Similar" => Some(Self::Similar), + "Plan" => Some(Self::Plan), + _ => None, + } + } +} +/// Generated client implementations. +pub mod resource_manager_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct ResourceManagerClient { + inner: tonic::client::Grpc, + } + impl ResourceManagerClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl ResourceManagerClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> ResourceManagerClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + ResourceManagerClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn list_resource_groups( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/resource_manager.ResourceManager/ListResourceGroups", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "resource_manager.ResourceManager", + "ListResourceGroups", + ), + ); + self.inner.unary(req, path, codec).await + } + pub async fn get_resource_group( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/resource_manager.ResourceManager/GetResourceGroup", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "resource_manager.ResourceManager", + "GetResourceGroup", + ), + ); + self.inner.unary(req, path, codec).await + } + pub async fn add_resource_group( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/resource_manager.ResourceManager/AddResourceGroup", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "resource_manager.ResourceManager", + "AddResourceGroup", + ), + ); + self.inner.unary(req, path, codec).await + } + pub async fn modify_resource_group( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/resource_manager.ResourceManager/ModifyResourceGroup", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "resource_manager.ResourceManager", + "ModifyResourceGroup", + ), + ); + self.inner.unary(req, path, codec).await + } + pub async fn delete_resource_group( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/resource_manager.ResourceManager/DeleteResourceGroup", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "resource_manager.ResourceManager", + "DeleteResourceGroup", + ), + ); + self.inner.unary(req, path, codec).await + } + pub async fn acquire_token_buckets( + &mut self, + request: impl tonic::IntoStreamingRequest< + Message = super::TokenBucketsRequest, + >, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/resource_manager.ResourceManager/AcquireTokenBuckets", + ); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "resource_manager.ResourceManager", + "AcquireTokenBuckets", + ), + ); + self.inner.streaming(req, path, codec).await + } + } +} diff --git a/src/generated/resource_usage_agent.rs b/src/generated/resource_usage_agent.rs index 325e8f2..a8f7f5f 100644 --- a/src/generated/resource_usage_agent.rs +++ b/src/generated/resource_usage_agent.rs @@ -1,39 +1,46 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct CpuTimeRecord { - #[prost(bytes = "vec", tag = "1")] - pub resource_group_tag: ::prost::alloc::vec::Vec, - /// UNIX timestamp in second. - #[prost(uint64, repeated, tag = "2")] - pub record_list_timestamp_sec: ::prost::alloc::vec::Vec, - /// The value can be greater than 1000ms if the requests are running parallelly. - #[prost(uint32, repeated, tag = "3")] - pub record_list_cpu_time_ms: ::prost::alloc::vec::Vec, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ResourceUsageRecord { - #[prost(bytes = "vec", tag = "1")] - pub resource_group_tag: ::prost::alloc::vec::Vec, - /// UNIX timestamp in second. - #[prost(uint64, repeated, tag = "2")] - pub record_list_timestamp_sec: ::prost::alloc::vec::Vec, - /// The value can be greater than 1000ms if the requests are running parallelly. - #[prost(uint32, repeated, tag = "3")] - pub record_list_cpu_time_ms: ::prost::alloc::vec::Vec, - /// The number of reads of keys associated with resource_group_tag. - #[prost(uint32, repeated, tag = "4")] - pub record_list_read_keys: ::prost::alloc::vec::Vec, - /// The number of writes of keys associated with resource_group_tag. - #[prost(uint32, repeated, tag = "5")] - pub record_list_write_keys: ::prost::alloc::vec::Vec, -} +pub struct ResourceMeteringRequest {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct EmptyResponse {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct ResourceMeteringRequest {} +pub struct ResourceUsageRecord { + #[prost(oneof = "resource_usage_record::RecordOneof", tags = "1")] + pub record_oneof: ::core::option::Option, +} +/// Nested message and enum types in `ResourceUsageRecord`. +pub mod resource_usage_record { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum RecordOneof { + #[prost(message, tag = "1")] + Record(super::GroupTagRecord), + } +} +/// GroupTagRecord is a set of resource usage data grouped by resource_group_tag. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GroupTagRecord { + #[prost(bytes = "vec", tag = "1")] + pub resource_group_tag: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "2")] + pub items: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GroupTagRecordItem { + #[prost(uint64, tag = "1")] + pub timestamp_sec: u64, + #[prost(uint32, tag = "2")] + pub cpu_time_ms: u32, + #[prost(uint32, tag = "3")] + pub read_keys: u32, + #[prost(uint32, tag = "4")] + pub write_keys: u32, +} /// Generated client implementations. pub mod resource_usage_agent_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] @@ -120,37 +127,6 @@ pub mod resource_usage_agent_client { self.inner = self.inner.max_encoding_message_size(limit); self } - /// DEPRECATED: We now use `Report` to report not only CPU time. - /// - /// Report the CPU time records. By default, the records with the same - /// resource group tag will be batched by minute. - pub async fn report_cpu_time( - &mut self, - request: impl tonic::IntoStreamingRequest, - ) -> std::result::Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; - let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/resource_usage_agent.ResourceUsageAgent/ReportCPUTime", - ); - let mut req = request.into_streaming_request(); - req.extensions_mut() - .insert( - GrpcMethod::new( - "resource_usage_agent.ResourceUsageAgent", - "ReportCPUTime", - ), - ); - self.inner.client_streaming(req, path, codec).await - } /// Report the resource usage records. By default, the records with the same /// resource group tag will be batched by minute. pub async fn report( diff --git a/src/generated/schedulingpb.rs b/src/generated/schedulingpb.rs new file mode 100644 index 0000000..5c6d5de --- /dev/null +++ b/src/generated/schedulingpb.rs @@ -0,0 +1,512 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RequestHeader { + /// cluster_id is the ID of the cluster which be sent to. + #[prost(uint64, tag = "1")] + pub cluster_id: u64, + /// sender_id is the ID of the sender server. + #[prost(uint64, tag = "2")] + pub sender_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResponseHeader { + /// cluster_id is the ID of the cluster which sent the response. + #[prost(uint64, tag = "1")] + pub cluster_id: u64, + #[prost(message, optional, tag = "2")] + pub error: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Error { + #[prost(enumeration = "ErrorType", tag = "1")] + pub r#type: i32, + #[prost(string, tag = "2")] + pub message: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Participant { + /// name is the unique name of the scheduling participant. + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + /// id is the unique id of the scheduling participant. + #[prost(uint64, tag = "2")] + pub id: u64, + /// listen_urls is the serivce endpoint list in the url format. + /// listen_urls\[0\] is primary service endpoint. + #[prost(string, repeated, tag = "3")] + pub listen_urls: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StoreHeartbeatRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub stats: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StoreHeartbeatResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(string, tag = "2")] + pub cluster_version: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RegionHeartbeatRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub region: ::core::option::Option, + /// Leader Peer sending the heartbeat. + #[prost(message, optional, tag = "3")] + pub leader: ::core::option::Option, + /// Term is the term of raft group. + #[prost(uint64, tag = "4")] + pub term: u64, + /// Leader considers that these peers are down. + #[prost(message, repeated, tag = "5")] + pub down_peers: ::prost::alloc::vec::Vec, + /// Pending peers are the peers that the leader can't consider as + /// working followers. + #[prost(message, repeated, tag = "6")] + pub pending_peers: ::prost::alloc::vec::Vec, + /// Bytes read/written during this period. + #[prost(uint64, tag = "7")] + pub bytes_written: u64, + #[prost(uint64, tag = "8")] + pub bytes_read: u64, + /// Keys read/written during this period. + #[prost(uint64, tag = "9")] + pub keys_written: u64, + #[prost(uint64, tag = "10")] + pub keys_read: u64, + /// Approximate region size. + #[prost(uint64, tag = "11")] + pub approximate_size: u64, + /// Approximate number of keys. + #[prost(uint64, tag = "12")] + pub approximate_keys: u64, + /// QueryStats reported write query stats, and there are read query stats in store heartbeat + #[prost(message, optional, tag = "13")] + pub query_stats: ::core::option::Option, + /// Actually reported time interval + #[prost(message, optional, tag = "14")] + pub interval: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RegionHeartbeatResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + /// ID of the region + #[prost(uint64, tag = "2")] + pub region_id: u64, + #[prost(message, optional, tag = "3")] + pub region_epoch: ::core::option::Option, + /// Leader of the region at the moment of the corresponding request was made. + #[prost(message, optional, tag = "4")] + pub target_peer: ::core::option::Option, + /// Notice, Pd only allows handling reported epoch >= current pd's. + /// Leader peer reports region status with RegionHeartbeatRequest + /// to pd regularly, pd will determine whether this region + /// should do ChangePeer or not. + /// E,g, max peer number is 3, region A, first only peer 1 in A. + /// + /// 1. Pd region state -> Peers (1), ConfVer (1). + /// 1. Leader peer 1 reports region state to pd, pd finds the + /// peer number is \< 3, so first changes its current region + /// state -> Peers (1, 2), ConfVer (1), and returns ChangePeer Adding 2. + /// 1. Leader does ChangePeer, then reports Peers (1, 2), ConfVer (2), + /// pd updates its state -> Peers (1, 2), ConfVer (2). + /// 1. Leader may report old Peers (1), ConfVer (1) to pd before ConfChange + /// finished, pd stills responses ChangePeer Adding 2, of course, we must + /// guarantee the second ChangePeer can't be applied in TiKV. + #[prost(message, optional, tag = "5")] + pub change_peer: ::core::option::Option, + /// Pd can return transfer_leader to let TiKV does leader transfer itself. + #[prost(message, optional, tag = "6")] + pub transfer_leader: ::core::option::Option, + #[prost(message, optional, tag = "7")] + pub merge: ::core::option::Option, + /// PD sends split_region to let TiKV split a region into two regions. + #[prost(message, optional, tag = "8")] + pub split_region: ::core::option::Option, + /// Multiple change peer operations atomically. + /// Note: PD can use both ChangePeer and ChangePeerV2 at the same time + /// (not in the same RegionHeartbeatResponse). + /// Now, PD use ChangePeerV2 in following scenarios: + /// 1. replacing peers + /// 2. demoting voter directly + #[prost(message, optional, tag = "9")] + pub change_peer_v2: ::core::option::Option, + #[prost(message, optional, tag = "10")] + pub switch_witnesses: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ScatterRegionsRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + /// If group is defined, the regions with the same group would be scattered as a whole group. + /// If not defined, the regions would be scattered in a cluster level. + #[prost(string, tag = "2")] + pub group: ::prost::alloc::string::String, + /// If regions_id is defined, the region_id would be ignored. + #[prost(uint64, repeated, tag = "3")] + pub regions_id: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "4")] + pub retry_limit: u64, + #[prost(bool, tag = "5")] + pub skip_store_limit: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ScatterRegionsResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub finished_percentage: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SplitRegionsRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(bytes = "vec", repeated, tag = "2")] + pub split_keys: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec>, + #[prost(uint64, tag = "3")] + pub retry_limit: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SplitRegionsResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub finished_percentage: u64, + #[prost(uint64, repeated, tag = "3")] + pub regions_id: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetOperatorRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub region_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetOperatorResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint64, tag = "2")] + pub region_id: u64, + #[prost(bytes = "vec", tag = "3")] + pub desc: ::prost::alloc::vec::Vec, + #[prost(enumeration = "super::pdpb::OperatorStatus", tag = "4")] + pub status: i32, + #[prost(bytes = "vec", tag = "5")] + pub kind: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AskBatchSplitRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub region: ::core::option::Option, + #[prost(uint32, tag = "3")] + pub split_count: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AskBatchSplitResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, repeated, tag = "2")] + pub ids: ::prost::alloc::vec::Vec, +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ErrorType { + Ok = 0, + Unknown = 1, + NotBootstrapped = 2, + AlreadyBootstrapped = 3, + InvalidValue = 4, + ClusterMismatched = 5, +} +impl ErrorType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ErrorType::Ok => "OK", + ErrorType::Unknown => "UNKNOWN", + ErrorType::NotBootstrapped => "NOT_BOOTSTRAPPED", + ErrorType::AlreadyBootstrapped => "ALREADY_BOOTSTRAPPED", + ErrorType::InvalidValue => "INVALID_VALUE", + ErrorType::ClusterMismatched => "CLUSTER_MISMATCHED", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "OK" => Some(Self::Ok), + "UNKNOWN" => Some(Self::Unknown), + "NOT_BOOTSTRAPPED" => Some(Self::NotBootstrapped), + "ALREADY_BOOTSTRAPPED" => Some(Self::AlreadyBootstrapped), + "INVALID_VALUE" => Some(Self::InvalidValue), + "CLUSTER_MISMATCHED" => Some(Self::ClusterMismatched), + _ => None, + } + } +} +/// Generated client implementations. +pub mod scheduling_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct SchedulingClient { + inner: tonic::client::Grpc, + } + impl SchedulingClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl SchedulingClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> SchedulingClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + SchedulingClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn store_heartbeat( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/schedulingpb.Scheduling/StoreHeartbeat", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("schedulingpb.Scheduling", "StoreHeartbeat")); + self.inner.unary(req, path, codec).await + } + pub async fn region_heartbeat( + &mut self, + request: impl tonic::IntoStreamingRequest< + Message = super::RegionHeartbeatRequest, + >, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/schedulingpb.Scheduling/RegionHeartbeat", + ); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert(GrpcMethod::new("schedulingpb.Scheduling", "RegionHeartbeat")); + self.inner.streaming(req, path, codec).await + } + pub async fn split_regions( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/schedulingpb.Scheduling/SplitRegions", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("schedulingpb.Scheduling", "SplitRegions")); + self.inner.unary(req, path, codec).await + } + pub async fn scatter_regions( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/schedulingpb.Scheduling/ScatterRegions", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("schedulingpb.Scheduling", "ScatterRegions")); + self.inner.unary(req, path, codec).await + } + pub async fn get_operator( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/schedulingpb.Scheduling/GetOperator", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("schedulingpb.Scheduling", "GetOperator")); + self.inner.unary(req, path, codec).await + } + pub async fn ask_batch_split( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/schedulingpb.Scheduling/AskBatchSplit", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("schedulingpb.Scheduling", "AskBatchSplit")); + self.inner.unary(req, path, codec).await + } + } +} diff --git a/src/generated/tikvpb.rs b/src/generated/tikvpb.rs index a0391c1..e27b1cb 100644 --- a/src/generated/tikvpb.rs +++ b/src/generated/tikvpb.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct BatchCommandsRequest { @@ -13,7 +14,7 @@ pub mod batch_commands_request { pub struct Request { #[prost( oneof = "request::Cmd", - tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 33, 34, 255" + tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 33, 34, 35, 36, 255" )] pub cmd: ::core::option::Option, } @@ -82,6 +83,12 @@ pub mod batch_commands_request { ), #[prost(message, tag = "34")] RawCoprocessor(super::super::super::kvrpcpb::RawCoprocessorRequest), + #[prost(message, tag = "35")] + FlashbackToVersion(super::super::super::kvrpcpb::FlashbackToVersionRequest), + #[prost(message, tag = "36")] + PrepareFlashbackToVersion( + super::super::super::kvrpcpb::PrepareFlashbackToVersionRequest, + ), /// For some test cases. #[prost(message, tag = "255")] Empty(super::super::BatchCommandsEmptyRequest), @@ -106,7 +113,7 @@ pub mod batch_commands_response { pub struct Response { #[prost( oneof = "response::Cmd", - tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 33, 34, 255" + tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 33, 34, 35, 36, 255" )] pub cmd: ::core::option::Option, } @@ -175,6 +182,12 @@ pub mod batch_commands_response { ), #[prost(message, tag = "34")] RawCoprocessor(super::super::super::kvrpcpb::RawCoprocessorResponse), + #[prost(message, tag = "35")] + FlashbackToVersion(super::super::super::kvrpcpb::FlashbackToVersionResponse), + #[prost(message, tag = "36")] + PrepareFlashbackToVersion( + super::super::super::kvrpcpb::PrepareFlashbackToVersionResponse, + ), /// For some test cases. #[prost(message, tag = "255")] Empty(super::super::BatchCommandsEmptyResponse), @@ -695,6 +708,60 @@ pub mod tikv_client { req.extensions_mut().insert(GrpcMethod::new("tikvpb.Tikv", "KvDeleteRange")); self.inner.unary(req, path, codec).await } + pub async fn kv_prepare_flashback_to_version( + &mut self, + request: impl tonic::IntoRequest< + super::super::kvrpcpb::PrepareFlashbackToVersionRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/KvPrepareFlashbackToVersion", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "KvPrepareFlashbackToVersion")); + self.inner.unary(req, path, codec).await + } + pub async fn kv_flashback_to_version( + &mut self, + request: impl tonic::IntoRequest< + super::super::kvrpcpb::FlashbackToVersionRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/KvFlashbackToVersion", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "KvFlashbackToVersion")); + self.inner.unary(req, path, codec).await + } /// Raw commands; no transaction support. pub async fn raw_get( &mut self, @@ -1288,6 +1355,37 @@ pub mod tikv_client { req.extensions_mut().insert(GrpcMethod::new("tikvpb.Tikv", "Snapshot")); self.inner.client_streaming(req, path, codec).await } + pub async fn tablet_snapshot( + &mut self, + request: impl tonic::IntoStreamingRequest< + Message = super::super::raft_serverpb::TabletSnapshotRequest, + >, + ) -> std::result::Result< + tonic::Response< + tonic::codec::Streaming< + super::super::raft_serverpb::TabletSnapshotResponse, + >, + >, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/TabletSnapshot", + ); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "TabletSnapshot")); + self.inner.streaming(req, path, codec).await + } /// Sent from PD or TiDB to a TiKV node. pub async fn split_region( &mut self, @@ -1510,6 +1608,31 @@ pub mod tikv_client { req.extensions_mut().insert(GrpcMethod::new("tikvpb.Tikv", "IsAlive")); self.inner.unary(req, path, codec).await } + pub async fn report_mpp_task_status( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/ReportMPPTaskStatus", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "ReportMPPTaskStatus")); + self.inner.unary(req, path, codec).await + } /// / CheckLeader sends all information (includes region term and epoch) to other stores. /// / Once a store receives a request, it checks term and epoch for each region, and sends the regions whose /// / term and epoch match with local information in the store. @@ -1591,5 +1714,254 @@ pub mod tikv_client { .insert(GrpcMethod::new("tikvpb.Tikv", "GetLockWaitInfo")); self.inner.unary(req, path, codec).await } + /// / Compact a specified key range. This request is not restricted to raft leaders and will not be replicated. + /// / It only compacts data on this node. + /// / TODO: Currently this RPC is designed to be only compatible with TiFlash. + /// / Shall be move out in https://github.com/pingcap/kvproto/issues/912 + pub async fn compact( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/tikvpb.Tikv/Compact"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("tikvpb.Tikv", "Compact")); + self.inner.unary(req, path, codec).await + } + /// / Get the information about history lock waiting from TiKV. + pub async fn get_lock_wait_history( + &mut self, + request: impl tonic::IntoRequest< + super::super::kvrpcpb::GetLockWaitHistoryRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/GetLockWaitHistory", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "GetLockWaitHistory")); + self.inner.unary(req, path, codec).await + } + /// / Get system table from TiFlash + pub async fn get_ti_flash_system_table( + &mut self, + request: impl tonic::IntoRequest< + super::super::kvrpcpb::TiFlashSystemTableRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/GetTiFlashSystemTable", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "GetTiFlashSystemTable")); + self.inner.unary(req, path, codec).await + } + /// These are for TiFlash disaggregated architecture + /// / Try to lock a S3 object, atomically + pub async fn try_add_lock( + &mut self, + request: impl tonic::IntoRequest< + super::super::disaggregated::TryAddLockRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/tikvpb.Tikv/tryAddLock"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("tikvpb.Tikv", "tryAddLock")); + self.inner.unary(req, path, codec).await + } + /// / Try to delete a S3 object, atomically + pub async fn try_mark_delete( + &mut self, + request: impl tonic::IntoRequest< + super::super::disaggregated::TryMarkDeleteRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/tryMarkDelete", + ); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("tikvpb.Tikv", "tryMarkDelete")); + self.inner.unary(req, path, codec).await + } + /// / Build the disaggregated task on TiFlash write node + pub async fn establish_disagg_task( + &mut self, + request: impl tonic::IntoRequest< + super::super::disaggregated::EstablishDisaggTaskRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/EstablishDisaggTask", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "EstablishDisaggTask")); + self.inner.unary(req, path, codec).await + } + /// / Cancel the disaggregated task on TiFlash write node + pub async fn cancel_disagg_task( + &mut self, + request: impl tonic::IntoRequest< + super::super::disaggregated::CancelDisaggTaskRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/CancelDisaggTask", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "CancelDisaggTask")); + self.inner.unary(req, path, codec).await + } + /// / Exchange page data between TiFlash write node and compute node + pub async fn fetch_disagg_pages( + &mut self, + request: impl tonic::IntoRequest< + super::super::disaggregated::FetchDisaggPagesRequest, + >, + ) -> std::result::Result< + tonic::Response< + tonic::codec::Streaming, + >, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/FetchDisaggPages", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "FetchDisaggPages")); + self.inner.server_streaming(req, path, codec).await + } + /// / Compute node get configuration from Write node + pub async fn get_disagg_config( + &mut self, + request: impl tonic::IntoRequest< + super::super::disaggregated::GetDisaggConfigRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tikvpb.Tikv/GetDisaggConfig", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tikvpb.Tikv", "GetDisaggConfig")); + self.inner.unary(req, path, codec).await + } } } diff --git a/src/generated/tracepb.rs b/src/generated/tracepb.rs new file mode 100644 index 0000000..51a0154 --- /dev/null +++ b/src/generated/tracepb.rs @@ -0,0 +1,197 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TraceRecordRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TraceRecord { + #[prost(oneof = "trace_record::RecordOneof", tags = "1, 2")] + pub record_oneof: ::core::option::Option, +} +/// Nested message and enum types in `TraceRecord`. +pub mod trace_record { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum RecordOneof { + #[prost(message, tag = "1")] + Report(super::Report), + #[prost(message, tag = "2")] + NotifyCollect(super::NotifyCollect), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RemoteParentSpan { + /// A unique id to identify the request. It's usually a UUID. + #[prost(uint64, tag = "1")] + pub trace_id: u64, + /// The span of remote caller that is awaiting the request. + #[prost(uint64, tag = "2")] + pub span_id: u64, +} +/// The context of the request to be traced. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TraceContext { + #[prost(message, repeated, tag = "1")] + pub remote_parent_spans: ::prost::alloc::vec::Vec, + /// Report the trace records only if the duration of handling the request exceeds the threshold. + #[prost(uint32, tag = "2")] + pub duration_threshold_ms: u32, +} +/// Report the spans collected when handling a request on a service. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Report { + #[prost(message, repeated, tag = "1")] + pub remote_parent_spans: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "2")] + pub spans: ::prost::alloc::vec::Vec, +} +/// Notify the subscriber to persis the spans of the trace. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct NotifyCollect { + #[prost(uint64, tag = "1")] + pub trace_id: u64, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Span { + /// The unique span id within the spans with the same `trace_id`. + /// The most significant 32 bits should be random number generated by each service instance. + #[prost(uint64, tag = "1")] + pub span_id: u64, + #[prost(uint64, tag = "2")] + pub parent_id: u64, + #[prost(uint64, tag = "3")] + pub begin_unix_ns: u64, + #[prost(uint64, tag = "4")] + pub duration_ns: u64, + #[prost(string, tag = "5")] + pub event: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "6")] + pub properties: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Property { + #[prost(string, tag = "1")] + pub key: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub value: ::prost::alloc::string::String, +} +/// Generated client implementations. +pub mod trace_record_pub_sub_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct TraceRecordPubSubClient { + inner: tonic::client::Grpc, + } + impl TraceRecordPubSubClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl TraceRecordPubSubClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> TraceRecordPubSubClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + TraceRecordPubSubClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + /// Subscribe the Trace records generated on this service. The service will periodically (e.g. per minute) + /// publishes Trace records to clients via gRPC stream. + pub async fn subscribe( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tracepb.TraceRecordPubSub/Subscribe", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tracepb.TraceRecordPubSub", "Subscribe")); + self.inner.server_streaming(req, path, codec).await + } + } +} diff --git a/src/generated/tsopb.rs b/src/generated/tsopb.rs new file mode 100644 index 0000000..61a3c6f --- /dev/null +++ b/src/generated/tsopb.rs @@ -0,0 +1,337 @@ +// This file is @generated by prost-build. +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RequestHeader { + /// cluster_id is the ID of the cluster which be sent to. + #[prost(uint64, tag = "1")] + pub cluster_id: u64, + /// sender_id is the ID of the sender server. + #[prost(uint64, tag = "2")] + pub sender_id: u64, + /// keyspace_id is the unique id of the tenant/keyspace. + #[prost(uint32, tag = "3")] + pub keyspace_id: u32, + /// keyspace_group_id is the unique id of the keyspace group to which the tenant/keyspace belongs. + #[prost(uint32, tag = "4")] + pub keyspace_group_id: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ResponseHeader { + /// cluster_id is the ID of the cluster which sent the response. + #[prost(uint64, tag = "1")] + pub cluster_id: u64, + #[prost(message, optional, tag = "2")] + pub error: ::core::option::Option, + /// keyspace_id is the unique id of the tenant/keyspace as the response receiver. + #[prost(uint32, tag = "3")] + pub keyspace_id: u32, + /// keyspace_group_id is the unique id of the keyspace group to which the tenant/keyspace belongs. + #[prost(uint32, tag = "4")] + pub keyspace_group_id: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Error { + #[prost(enumeration = "ErrorType", tag = "1")] + pub r#type: i32, + #[prost(string, tag = "2")] + pub message: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TsoRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint32, tag = "2")] + pub count: u32, + #[prost(string, tag = "3")] + pub dc_location: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TsoResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint32, tag = "2")] + pub count: u32, + #[prost(message, optional, tag = "3")] + pub timestamp: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Participant { + /// name is the unique name of the TSO participant. + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + /// id is the unique id of the TSO participant. + #[prost(uint64, tag = "2")] + pub id: u64, + /// listen_urls is the serivce endpoint list in the url format. + /// listen_urls\[0\] is primary service endpoint. + #[prost(string, repeated, tag = "3")] + pub listen_urls: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct KeyspaceGroupMember { + #[prost(string, tag = "1")] + pub address: ::prost::alloc::string::String, + #[prost(bool, tag = "2")] + pub is_primary: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SplitState { + #[prost(uint32, tag = "1")] + pub split_source: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct KeyspaceGroup { + #[prost(uint32, tag = "1")] + pub id: u32, + #[prost(string, tag = "2")] + pub user_kind: ::prost::alloc::string::String, + #[prost(message, optional, tag = "3")] + pub split_state: ::core::option::Option, + #[prost(message, repeated, tag = "4")] + pub members: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FindGroupByKeyspaceIdRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(uint32, tag = "2")] + pub keyspace_id: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FindGroupByKeyspaceIdResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub keyspace_group: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetMinTsRequest { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(string, tag = "2")] + pub dc_location: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetMinTsResponse { + #[prost(message, optional, tag = "1")] + pub header: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub timestamp: ::core::option::Option, + /// the count of keyspace group primaries that the TSO server/pod is serving + #[prost(uint32, tag = "3")] + pub keyspace_groups_serving: u32, + /// the total count of keyspace groups + #[prost(uint32, tag = "4")] + pub keyspace_groups_total: u32, +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ErrorType { + Ok = 0, + Unknown = 1, + NotBootstrapped = 2, + AlreadyBootstrapped = 3, + InvalidValue = 4, + ClusterMismatched = 5, +} +impl ErrorType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ErrorType::Ok => "OK", + ErrorType::Unknown => "UNKNOWN", + ErrorType::NotBootstrapped => "NOT_BOOTSTRAPPED", + ErrorType::AlreadyBootstrapped => "ALREADY_BOOTSTRAPPED", + ErrorType::InvalidValue => "INVALID_VALUE", + ErrorType::ClusterMismatched => "CLUSTER_MISMATCHED", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "OK" => Some(Self::Ok), + "UNKNOWN" => Some(Self::Unknown), + "NOT_BOOTSTRAPPED" => Some(Self::NotBootstrapped), + "ALREADY_BOOTSTRAPPED" => Some(Self::AlreadyBootstrapped), + "INVALID_VALUE" => Some(Self::InvalidValue), + "CLUSTER_MISMATCHED" => Some(Self::ClusterMismatched), + _ => None, + } + } +} +/// Generated client implementations. +pub mod tso_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct TsoClient { + inner: tonic::client::Grpc, + } + impl TsoClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl TsoClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> TsoClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + TsoClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn tso( + &mut self, + request: impl tonic::IntoStreamingRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/tsopb.TSO/Tso"); + let mut req = request.into_streaming_request(); + req.extensions_mut().insert(GrpcMethod::new("tsopb.TSO", "Tso")); + self.inner.streaming(req, path, codec).await + } + /// Find the keyspace group that the keyspace belongs to by keyspace id. + pub async fn find_group_by_keyspace_id( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/tsopb.TSO/FindGroupByKeyspaceID", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("tsopb.TSO", "FindGroupByKeyspaceID")); + self.inner.unary(req, path, codec).await + } + /// Get the minimum timestamp across all keyspace groups served by the TSO server who receives + /// and handle the request. If the TSO server/pod is not serving any keyspace group, return + /// an empty timestamp, and the client needs to skip the empty timestamps when collecting + /// the min timestamp from all TSO servers/pods. + pub async fn get_min_ts( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/tsopb.TSO/GetMinTS"); + let mut req = request.into_request(); + req.extensions_mut().insert(GrpcMethod::new("tsopb.TSO", "GetMinTS")); + self.inner.unary(req, path, codec).await + } + } +} diff --git a/src/kv/bound_range.rs b/src/kv/bound_range.rs index 1cf5ac8..b99cba3 100644 --- a/src/kv/bound_range.rs +++ b/src/kv/bound_range.rs @@ -136,17 +136,11 @@ impl BoundRange { pub fn into_keys(self) -> (Key, Option) { let start = match self.from { Bound::Included(v) => v, - Bound::Excluded(mut v) => { - v.push_zero(); - v - } + Bound::Excluded(v) => v.next_key(), Bound::Unbounded => Key::EMPTY, }; let end = match self.to { - Bound::Included(mut v) => { - v.push_zero(); - Some(v) - } + Bound::Included(v) => Some(v.next_key()), Bound::Excluded(v) => Some(v), Bound::Unbounded => None, }; diff --git a/src/kv/key.rs b/src/kv/key.rs index 7ee1659..fa19d3f 100644 --- a/src/kv/key.rs +++ b/src/kv/key.rs @@ -2,7 +2,6 @@ use std::fmt; use std::ops::Bound; -use std::u8; #[allow(unused_imports)] #[cfg(test)] @@ -17,6 +16,7 @@ use super::HexRepr; use crate::kv::codec::BytesEncoder; use crate::kv::codec::{self}; use crate::proto::kvrpcpb; +use crate::proto::kvrpcpb::KvPair; const _PROPTEST_KEY_MAX: usize = 1024 * 2; // 2 KB @@ -71,7 +71,7 @@ pub struct Key( test, proptest(strategy = "any_with::>((size_range(_PROPTEST_KEY_MAX), ()))") )] - pub(super) Vec, + pub(crate) Vec, ); impl AsRef for kvrpcpb::Mutation { @@ -80,6 +80,20 @@ impl AsRef for kvrpcpb::Mutation { } } +pub struct KvPairTTL(pub KvPair, pub u64); + +impl AsRef for KvPairTTL { + fn as_ref(&self) -> &Key { + self.0.key.as_ref() + } +} + +impl From for (KvPair, u64) { + fn from(value: KvPairTTL) -> Self { + (value.0, value.1) + } +} + impl Key { /// The empty key. pub const EMPTY: Self = Key(Vec::new()); @@ -98,10 +112,11 @@ impl Key { /// Push a zero to the end of the key. /// - /// Extending a zero makes the new key the smallest key that is greater than than the original one, i.e. the succeeder. + /// Extending a zero makes the new key the smallest key that is greater than than the original one. #[inline] - pub(super) fn push_zero(&mut self) { - self.0.push(0) + pub(crate) fn next_key(mut self) -> Self { + self.0.push(0); + self } /// Convert the key to a lower bound. The key is treated as inclusive. diff --git a/src/kv/kvpair.rs b/src/kv/kvpair.rs index cfc6ee1..f609f23 100644 --- a/src/kv/kvpair.rs +++ b/src/kv/kvpair.rs @@ -25,7 +25,7 @@ use crate::proto::kvrpcpb; /// /// Many functions which accept a `KvPair` accept an `Into`, which means all of the above /// types (Like a `(Key, Value)`) can be passed directly to those functions. -#[derive(Default, Clone, Eq, PartialEq)] +#[derive(Default, Clone, Eq, PartialEq, Hash)] #[cfg_attr(test, derive(Arbitrary))] pub struct KvPair(pub Key, pub Value); diff --git a/src/kv/mod.rs b/src/kv/mod.rs index 489110e..41da842 100644 --- a/src/kv/mod.rs +++ b/src/kv/mod.rs @@ -1,6 +1,5 @@ // Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. use std::fmt; -use std::u8; mod bound_range; pub mod codec; @@ -11,6 +10,7 @@ mod value; pub use bound_range::BoundRange; pub use bound_range::IntoOwnedRange; pub use key::Key; +pub use key::KvPairTTL; pub use kvpair::KvPair; pub use value::Value; diff --git a/src/lib.rs b/src/lib.rs index bb454dd..7bb3cff 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -94,8 +94,6 @@ pub mod backoff; #[doc(hidden)] -pub mod proto; // export `proto` to enable user customized codec -#[doc(hidden)] pub mod raw; pub mod request; #[doc(hidden)] @@ -106,6 +104,7 @@ mod compat; mod config; mod kv; mod pd; +mod proto; mod region; mod region_cache; mod stats; @@ -146,8 +145,6 @@ pub use crate::raw::Client as RawClient; #[doc(inline)] pub use crate::raw::ColumnFamily; #[doc(inline)] -pub use crate::request::codec; -#[doc(inline)] pub use crate::request::RetryOptions; #[doc(inline)] pub use crate::timestamp::Timestamp; diff --git a/src/mock.rs b/src/mock.rs index f9c94ae..0a3fb83 100644 --- a/src/mock.rs +++ b/src/mock.rs @@ -14,11 +14,11 @@ use derive_new::new; use crate::pd::PdClient; use crate::pd::PdRpcClient; use crate::pd::RetryClient; +use crate::proto::keyspacepb; use crate::proto::metapb::RegionEpoch; use crate::proto::metapb::{self}; use crate::region::RegionId; use crate::region::RegionWithLeader; -use crate::request::codec::ApiV1TxnCodec; use crate::store::KvConnect; use crate::store::RegionStore; use crate::store::Request; @@ -31,7 +31,7 @@ use crate::Timestamp; /// Create a `PdRpcClient` with it's internals replaced with mocks so that the /// client can be tested without doing any RPC calls. -pub async fn pd_rpc_client() -> PdRpcClient { +pub async fn pd_rpc_client() -> PdRpcClient { let config = Config::default(); PdRpcClient::new( config.clone(), @@ -44,7 +44,6 @@ pub async fn pd_rpc_client() -> PdRpcClient MockPdClient { - MockPdClient { - client, - codec: ApiV1TxnCodec::default(), - } - } } #[async_trait] @@ -113,7 +103,6 @@ impl MockPdClient { pub fn default() -> MockPdClient { MockPdClient { client: MockKvClient::default(), - codec: ApiV1TxnCodec::default(), } } @@ -177,7 +166,6 @@ impl MockPdClient { #[async_trait] impl PdClient for MockPdClient { - type Codec = ApiV1TxnCodec; type KvClient = MockKvClient; async fn map_region_to_store(self: Arc, region: RegionWithLeader) -> Result { @@ -228,7 +216,7 @@ impl PdClient for MockPdClient { async fn invalidate_region_cache(&self, _ver_id: crate::region::RegionVerId) {} - fn get_codec(&self) -> &Self::Codec { - &self.codec + async fn load_keyspace(&self, _keyspace: &str) -> Result { + unimplemented!() } } diff --git a/src/pd/client.rs b/src/pd/client.rs index 5461cb5..ba36c0e 100644 --- a/src/pd/client.rs +++ b/src/pd/client.rs @@ -14,13 +14,13 @@ use crate::kv::codec; use crate::pd::retry::RetryClientTrait; use crate::pd::Cluster; use crate::pd::RetryClient; +use crate::proto::keyspacepb; use crate::proto::kvrpcpb; use crate::proto::metapb; use crate::region::RegionId; use crate::region::RegionVerId; use crate::region::RegionWithLeader; use crate::region_cache::RegionCache; -use crate::request::codec::{ApiV1TxnCodec, Codec}; use crate::store::KvConnect; use crate::store::RegionStore; use crate::store::TikvConnect; @@ -51,7 +51,6 @@ use crate::Timestamp; /// So if we use transactional APIs, keys in PD are encoded and PD does not know about the encoding stuff. #[async_trait] pub trait PdClient: Send + Sync + 'static { - type Codec: Codec; type KvClient: KvClient + Send + Sync + 'static; /// In transactional API, `region` is decoded (keys in raw format). @@ -67,6 +66,8 @@ pub trait PdClient: Send + Sync + 'static { async fn update_safepoint(self: Arc, safepoint: u64) -> Result; + async fn load_keyspace(&self, keyspace: &str) -> Result; + /// In transactional API, `key` is in raw format async fn store_for_key(self: Arc, key: &Key) -> Result { let region = self.region_for_key(key).await?; @@ -193,11 +194,8 @@ pub trait PdClient: Send + Sync + 'static { .boxed() } - fn decode_region( - mut region: RegionWithLeader, - enable_mvcc_codec: bool, - ) -> Result { - if enable_mvcc_codec { + fn decode_region(mut region: RegionWithLeader, enable_codec: bool) -> Result { + if enable_codec { codec::decode_bytes_in_place(&mut region.region.start_key, false)?; codec::decode_bytes_in_place(&mut region.region.end_key, false)?; } @@ -207,30 +205,20 @@ pub trait PdClient: Send + Sync + 'static { async fn update_leader(&self, ver_id: RegionVerId, leader: metapb::Peer) -> Result<()>; async fn invalidate_region_cache(&self, ver_id: RegionVerId); - - /// Get the codec carried by `PdClient`. - /// The purpose of carrying the codec is to avoid passing it on so many calling paths. - fn get_codec(&self) -> &Self::Codec; } /// This client converts requests for the logical TiKV cluster into requests /// for a single TiKV store using PD and internal logic. -pub struct PdRpcClient< - Cod: Codec = ApiV1TxnCodec, - KvC: KvConnect + Send + Sync + 'static = TikvConnect, - Cl = Cluster, -> { +pub struct PdRpcClient { pd: Arc>, kv_connect: KvC, kv_client_cache: Arc>>, - enable_mvcc_codec: bool, + enable_codec: bool, region_cache: RegionCache>, - codec: Option, } #[async_trait] -impl PdClient for PdRpcClient { - type Codec = Cod; +impl PdClient for PdRpcClient { type KvClient = KvC::KvClient; async fn map_region_to_store(self: Arc, region: RegionWithLeader) -> Result { @@ -241,20 +229,20 @@ impl PdClient for PdRpcClien } async fn region_for_key(&self, key: &Key) -> Result { - let enable_mvcc_codec = self.enable_mvcc_codec; - let key = if enable_mvcc_codec { + let enable_codec = self.enable_codec; + let key = if enable_codec { key.to_encoded() } else { key.clone() }; let region = self.region_cache.get_region_by_key(&key).await?; - Self::decode_region(region, enable_mvcc_codec) + Self::decode_region(region, enable_codec) } async fn region_for_id(&self, id: RegionId) -> Result { let region = self.region_cache.get_region_by_id(id).await?; - Self::decode_region(region, self.enable_mvcc_codec) + Self::decode_region(region, self.enable_codec) } async fn all_stores(&self) -> Result> { @@ -283,39 +271,34 @@ impl PdClient for PdRpcClien self.region_cache.invalidate_region_cache(ver_id).await } - fn get_codec(&self) -> &Self::Codec { - self.codec - .as_ref() - .unwrap_or_else(|| panic!("codec not set")) + async fn load_keyspace(&self, keyspace: &str) -> Result { + self.pd.load_keyspace(keyspace).await } } -impl PdRpcClient { +impl PdRpcClient { pub async fn connect( pd_endpoints: &[String], config: Config, - enable_mvcc_codec: bool, // TODO: infer from `codec`. - codec: Option, - ) -> Result> { + enable_codec: bool, + ) -> Result { PdRpcClient::new( config.clone(), |security_mgr| TikvConnect::new(security_mgr, config.timeout), |security_mgr| RetryClient::connect(pd_endpoints, security_mgr, config.timeout), - enable_mvcc_codec, - codec, + enable_codec, ) .await } } -impl PdRpcClient { +impl PdRpcClient { pub async fn new( config: Config, kv_connect: MakeKvC, pd: MakePd, - enable_mvcc_codec: bool, - codec: Option, - ) -> Result> + enable_codec: bool, + ) -> Result> where PdFut: Future>>, MakeKvC: FnOnce(Arc) -> KvC, @@ -337,9 +320,8 @@ impl PdRpcClient PdRpcClient Err(e), } } - - pub fn set_codec(&mut self, codec: Cod) { - self.codec = Some(codec); - } } fn make_key_range(start_key: Vec, end_key: Vec) -> kvrpcpb::KeyRange { diff --git a/src/pd/cluster.rs b/src/pd/cluster.rs index 3df4d25..8c8f0d7 100644 --- a/src/pd/cluster.rs +++ b/src/pd/cluster.rs @@ -15,7 +15,9 @@ use tonic::Request; use super::timestamp::TimestampOracle; use crate::internal_err; +use crate::proto::keyspacepb; use crate::proto::pdpb; +use crate::Error; use crate::Result; use crate::SecurityManager; use crate::Timestamp; @@ -24,6 +26,7 @@ use crate::Timestamp; pub struct Cluster { id: u64, client: pdpb::pd_client::PdClient, + keyspace_client: keyspacepb::keyspace_client::KeyspaceClient, members: pdpb::GetMembersResponse, tso: TimestampOracle, } @@ -46,7 +49,7 @@ impl Cluster { timeout: Duration, ) -> Result { let mut req = pd_request!(self.id, pdpb::GetRegionRequest); - req.region_key = key.clone(); + req.region_key = key; req.send(&mut self.client, timeout).await } @@ -91,6 +94,20 @@ impl Cluster { req.safe_point = safepoint; req.send(&mut self.client, timeout).await } + + pub async fn load_keyspace( + &mut self, + keyspace: &str, + timeout: Duration, + ) -> Result { + let mut req = pd_request!(self.id, keyspacepb::LoadKeyspaceRequest); + req.name = keyspace.to_string(); + let resp = req.send(&mut self.keyspace_client, timeout).await?; + let keyspace = resp + .keyspace + .ok_or_else(|| Error::KeyspaceNotFound(keyspace.to_owned()))?; + Ok(keyspace) + } } /// An object for connecting and reconnecting to a PD cluster. @@ -109,12 +126,13 @@ impl Connection { timeout: Duration, ) -> Result { let members = self.validate_endpoints(endpoints, timeout).await?; - let (client, members) = self.try_connect_leader(&members, timeout).await?; + let (client, keyspace_client, members) = self.try_connect_leader(&members, timeout).await?; let id = members.header.as_ref().unwrap().cluster_id; let tso = TimestampOracle::new(id, &client)?; let cluster = Cluster { id, client, + keyspace_client, members, tso, }; @@ -125,11 +143,13 @@ impl Connection { pub async fn reconnect(&self, cluster: &mut Cluster, timeout: Duration) -> Result<()> { warn!("updating pd client"); let start = Instant::now(); - let (client, members) = self.try_connect_leader(&cluster.members, timeout).await?; + let (client, keyspace_client, members) = + self.try_connect_leader(&cluster.members, timeout).await?; let tso = TimestampOracle::new(cluster.id, &client)?; *cluster = Cluster { id: cluster.id, client, + keyspace_client, members, tso, }; @@ -152,7 +172,7 @@ impl Connection { return Err(internal_err!("duplicated PD endpoint {}", ep)); } - let (_, resp) = match self.connect(ep, timeout).await { + let (_, _, resp) = match self.connect(ep, timeout).await { Ok(resp) => resp, // Ignore failed PD node. Err(e) => { @@ -194,16 +214,40 @@ impl Connection { &self, addr: &str, _timeout: Duration, - ) -> Result<(pdpb::pd_client::PdClient, pdpb::GetMembersResponse)> { + ) -> Result<( + pdpb::pd_client::PdClient, + keyspacepb::keyspace_client::KeyspaceClient, + pdpb::GetMembersResponse, + )> { let mut client = self .security_mgr .connect(addr, pdpb::pd_client::PdClient::::new) .await?; + let keyspace_client = self + .security_mgr + .connect( + addr, + keyspacepb::keyspace_client::KeyspaceClient::::new, + ) + .await?; let resp: pdpb::GetMembersResponse = client .get_members(pdpb::GetMembersRequest::default()) .await? .into_inner(); - Ok((client, resp)) + if let Some(err) = resp + .header + .as_ref() + .and_then(|header| header.error.as_ref()) + { + return Err(internal_err!("failed to get PD members, err {:?}", err)); + } + if resp.leader.is_none() { + return Err(internal_err!( + "unexpected no PD leader in get member resp: {:?}", + resp + )); + } + Ok((client, keyspace_client, resp)) } async fn try_connect( @@ -211,10 +255,14 @@ impl Connection { addr: &str, cluster_id: u64, timeout: Duration, - ) -> Result<(pdpb::pd_client::PdClient, pdpb::GetMembersResponse)> { - let (client, r) = self.connect(addr, timeout).await?; + ) -> Result<( + pdpb::pd_client::PdClient, + keyspacepb::keyspace_client::KeyspaceClient, + pdpb::GetMembersResponse, + )> { + let (client, keyspace_client, r) = self.connect(addr, timeout).await?; Connection::validate_cluster_id(addr, &r, cluster_id)?; - Ok((client, r)) + Ok((client, keyspace_client, r)) } fn validate_cluster_id( @@ -239,7 +287,11 @@ impl Connection { &self, previous: &pdpb::GetMembersResponse, timeout: Duration, - ) -> Result<(pdpb::pd_client::PdClient, pdpb::GetMembersResponse)> { + ) -> Result<( + pdpb::pd_client::PdClient, + keyspacepb::keyspace_client::KeyspaceClient, + pdpb::GetMembersResponse, + )> { let previous_leader = previous.leader.as_ref().unwrap(); let members = &previous.members; let cluster_id = previous.header.as_ref().unwrap().cluster_id; @@ -253,7 +305,7 @@ impl Connection { { for ep in &m.client_urls { match self.try_connect(ep.as_str(), cluster_id, timeout).await { - Ok((_, r)) => { + Ok((_, _, r)) => { resp = Some(r); break 'outer; } @@ -269,9 +321,10 @@ impl Connection { if let Some(resp) = resp { let leader = resp.leader.as_ref().unwrap(); for ep in &leader.client_urls { - let r = self.try_connect(ep.as_str(), cluster_id, timeout).await; - if r.is_ok() { - return r; + if let Ok((client, keyspace_client, members)) = + self.try_connect(ep.as_str(), cluster_id, timeout).await + { + return Ok((client, keyspace_client, members)); } } } @@ -284,18 +337,12 @@ type GrpcResult = std::result::Result; #[async_trait] trait PdMessage: Sized { + type Client: Send; type Response: PdResponse; - async fn rpc( - req: Request, - client: &mut pdpb::pd_client::PdClient, - ) -> GrpcResult; + async fn rpc(req: Request, client: &mut Self::Client) -> GrpcResult; - async fn send( - self, - client: &mut pdpb::pd_client::PdClient, - timeout: Duration, - ) -> Result { + async fn send(self, client: &mut Self::Client, timeout: Duration) -> Result { let mut req = self.into_request(); req.set_timeout(timeout); let response = Self::rpc(req, client).await?; @@ -310,64 +357,64 @@ trait PdMessage: Sized { #[async_trait] impl PdMessage for pdpb::GetRegionRequest { + type Client = pdpb::pd_client::PdClient; type Response = pdpb::GetRegionResponse; - async fn rpc( - req: Request, - client: &mut pdpb::pd_client::PdClient, - ) -> GrpcResult { + async fn rpc(req: Request, client: &mut Self::Client) -> GrpcResult { Ok(client.get_region(req).await?.into_inner()) } } #[async_trait] impl PdMessage for pdpb::GetRegionByIdRequest { + type Client = pdpb::pd_client::PdClient; type Response = pdpb::GetRegionResponse; - async fn rpc( - req: Request, - client: &mut pdpb::pd_client::PdClient, - ) -> GrpcResult { + async fn rpc(req: Request, client: &mut Self::Client) -> GrpcResult { Ok(client.get_region_by_id(req).await?.into_inner()) } } #[async_trait] impl PdMessage for pdpb::GetStoreRequest { + type Client = pdpb::pd_client::PdClient; type Response = pdpb::GetStoreResponse; - async fn rpc( - req: Request, - client: &mut pdpb::pd_client::PdClient, - ) -> GrpcResult { + async fn rpc(req: Request, client: &mut Self::Client) -> GrpcResult { Ok(client.get_store(req).await?.into_inner()) } } #[async_trait] impl PdMessage for pdpb::GetAllStoresRequest { + type Client = pdpb::pd_client::PdClient; type Response = pdpb::GetAllStoresResponse; - async fn rpc( - req: Request, - client: &mut pdpb::pd_client::PdClient, - ) -> GrpcResult { + async fn rpc(req: Request, client: &mut Self::Client) -> GrpcResult { Ok(client.get_all_stores(req).await?.into_inner()) } } #[async_trait] impl PdMessage for pdpb::UpdateGcSafePointRequest { + type Client = pdpb::pd_client::PdClient; type Response = pdpb::UpdateGcSafePointResponse; - async fn rpc( - req: Request, - client: &mut pdpb::pd_client::PdClient, - ) -> GrpcResult { + async fn rpc(req: Request, client: &mut Self::Client) -> GrpcResult { Ok(client.update_gc_safe_point(req).await?.into_inner()) } } +#[async_trait] +impl PdMessage for keyspacepb::LoadKeyspaceRequest { + type Client = keyspacepb::keyspace_client::KeyspaceClient; + type Response = keyspacepb::LoadKeyspaceResponse; + + async fn rpc(req: Request, client: &mut Self::Client) -> GrpcResult { + Ok(client.load_keyspace(req).await?.into_inner()) + } +} + trait PdResponse { fn header(&self) -> &pdpb::ResponseHeader; } @@ -395,3 +442,9 @@ impl PdResponse for pdpb::UpdateGcSafePointResponse { self.header.as_ref().unwrap() } } + +impl PdResponse for keyspacepb::LoadKeyspaceResponse { + fn header(&self) -> &pdpb::ResponseHeader { + self.header.as_ref().unwrap() + } +} diff --git a/src/pd/retry.rs b/src/pd/retry.rs index 3c17a49..c9ccf1e 100644 --- a/src/pd/retry.rs +++ b/src/pd/retry.rs @@ -13,6 +13,7 @@ use tokio::time::sleep; use crate::pd::Cluster; use crate::pd::Connection; +use crate::proto::keyspacepb; use crate::proto::metapb; use crate::proto::pdpb::Timestamp; use crate::proto::pdpb::{self}; @@ -45,6 +46,8 @@ pub trait RetryClientTrait { async fn get_timestamp(self: Arc) -> Result; async fn update_safepoint(self: Arc, safepoint: u64) -> Result; + + async fn load_keyspace(&self, keyspace: &str) -> Result; } /// Client for communication with a PD cluster. Has the facility to reconnect to the cluster. pub struct RetryClient { @@ -197,6 +200,12 @@ impl RetryClientTrait for RetryClient { .map(|resp| resp.new_safe_point == safepoint) }) } + + async fn load_keyspace(&self, keyspace: &str) -> Result { + retry_mut!(self, "load_keyspace", |cluster| async { + cluster.load_keyspace(keyspace, self.timeout).await + }) + } } impl fmt::Debug for RetryClient { diff --git a/src/raw/client.rs b/src/raw/client.rs index fc73301..76d40b6 100644 --- a/src/raw/client.rs +++ b/src/raw/client.rs @@ -3,7 +3,6 @@ use core::ops::Range; use std::str::FromStr; use std::sync::Arc; -use std::u32; use futures::StreamExt; use log::debug; @@ -15,10 +14,13 @@ use crate::pd::PdClient; use crate::pd::PdRpcClient; use crate::proto::metapb; use crate::raw::lowering::*; -use crate::request::codec::{ApiV1RawCodec, Codec, EncodedRequest}; use crate::request::Collect; use crate::request::CollectSingle; +use crate::request::EncodeKeyspace; +use crate::request::KeyMode; +use crate::request::Keyspace; use crate::request::Plan; +use crate::request::TruncateKeyspace; use crate::Backoff; use crate::BoundRange; use crate::ColumnFamily; @@ -36,16 +38,13 @@ const MAX_RAW_KV_SCAN_LIMIT: u32 = 10240; /// /// The returned results of raw request methods are [`Future`](std::future::Future)s that must be /// awaited to execute. -pub struct Client> -where - Cod: Codec, - PdC: PdClient, -{ +pub struct Client { rpc: Arc, cf: Option, backoff: Backoff, /// Whether to use the [`atomic mode`](Client::with_atomic_for_cas). atomic: bool, + keyspace: Keyspace, } impl Clone for Client { @@ -55,11 +54,12 @@ impl Clone for Client { cf: self.cf.clone(), backoff: self.backoff.clone(), atomic: self.atomic, + keyspace: self.keyspace, } } } -impl Client> { +impl Client { /// Create a raw [`Client`] and connect to the TiKV cluster. /// /// Because TiKV is managed by a [PD](https://github.com/pingcap/pd/) cluster, the endpoints for @@ -104,16 +104,25 @@ impl Client> { pd_endpoints: Vec, config: Config, ) -> Result { + let enable_codec = config.keyspace.is_some(); let pd_endpoints: Vec = pd_endpoints.into_iter().map(Into::into).collect(); - let rpc = Arc::new( - PdRpcClient::connect(&pd_endpoints, config, false, Some(ApiV1RawCodec::default())) - .await?, - ); + let rpc = + Arc::new(PdRpcClient::connect(&pd_endpoints, config.clone(), enable_codec).await?); + let keyspace = match config.keyspace { + Some(keyspace) => { + let keyspace = rpc.load_keyspace(&keyspace).await?; + Keyspace::Enable { + keyspace_id: keyspace.id, + } + } + None => Keyspace::Disable, + }; Ok(Client { rpc, cf: None, backoff: DEFAULT_REGION_BACKOFF, atomic: false, + keyspace, }) } @@ -148,11 +157,10 @@ impl Client> { cf: Some(cf), backoff: self.backoff.clone(), atomic: self.atomic, + keyspace: self.keyspace, } } -} -impl Client> { /// Set the [`Backoff`] strategy for retrying requests. /// The default strategy is [`DEFAULT_REGION_BACKOFF`](crate::backoff::DEFAULT_REGION_BACKOFF). /// See [`Backoff`] for more information. @@ -178,6 +186,7 @@ impl Client> { cf: self.cf.clone(), backoff, atomic: self.atomic, + keyspace: self.keyspace, } } @@ -195,11 +204,12 @@ impl Client> { cf: self.cf.clone(), backoff: self.backoff.clone(), atomic: true, + keyspace: self.keyspace, } } } -impl> Client { +impl Client { /// Create a new 'get' request. /// /// Once resolved this request will result in the fetching of the value associated with the @@ -220,9 +230,9 @@ impl> Client { /// ``` pub async fn get(&self, key: impl Into) -> Result> { debug!("invoking raw get request"); - let request = new_raw_get_request(key.into(), self.cf.clone()); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw); + let request = new_raw_get_request(key, self.cf.clone()); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .retry_multi_region(self.backoff.clone()) .merge(CollectSingle) .post_process_default() @@ -253,15 +263,47 @@ impl> Client { keys: impl IntoIterator>, ) -> Result> { debug!("invoking raw batch_get request"); - let request = new_raw_batch_get_request(keys.into_iter().map(Into::into), self.cf.clone()); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let keys = keys + .into_iter() + .map(|k| k.into().encode_keyspace(self.keyspace, KeyMode::Raw)); + let request = new_raw_batch_get_request(keys, self.cf.clone()); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .retry_multi_region(self.backoff.clone()) .merge(Collect) .plan(); - plan.execute() - .await - .map(|r| r.into_iter().map(Into::into).collect()) + plan.execute().await.map(|r| { + r.into_iter() + .map(|pair| pair.truncate_keyspace(self.keyspace)) + .collect() + }) + } + + /// Create a new 'get key ttl' request. + /// + /// Once resolved this request will result in the fetching of the alive time left for the + /// given key. + /// + /// Retuning `Ok(None)` indicates the key does not exist in TiKV. + /// + /// # Examples + /// # use tikv_client::{Value, Config, RawClient}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let client = RawClient::new(vec!["192.168.0.100"]).await.unwrap(); + /// let key = "TiKV".to_owned(); + /// let req = client.get_key_ttl_secs(key); + /// let result: Option = req.await.unwrap(); + /// # }); + pub async fn get_key_ttl_secs(&self, key: impl Into) -> Result> { + debug!("invoking raw get_key_ttl_secs request"); + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw); + let request = new_raw_get_key_ttl_request(key, self.cf.clone()); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) + .retry_multi_region(self.backoff.clone()) + .merge(CollectSingle) + .post_process_default() + .plan(); + plan.execute().await } /// Create a new 'put' request. @@ -281,10 +323,20 @@ impl> Client { /// # }); /// ``` pub async fn put(&self, key: impl Into, value: impl Into) -> Result<()> { + self.put_with_ttl(key, value, 0).await + } + + pub async fn put_with_ttl( + &self, + key: impl Into, + value: impl Into, + ttl_secs: u64, + ) -> Result<()> { debug!("invoking raw put request"); - let request = new_raw_put_request(key.into(), value.into(), self.cf.clone(), self.atomic); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw); + let request = + new_raw_put_request(key, value.into(), self.cf.clone(), ttl_secs, self.atomic); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .retry_multi_region(self.backoff.clone()) .merge(CollectSingle) .extract_error() @@ -313,15 +365,22 @@ impl> Client { pub async fn batch_put( &self, pairs: impl IntoIterator>, + ) -> Result<()> { + self.batch_put_with_ttl(pairs, std::iter::repeat(0)).await + } + + pub async fn batch_put_with_ttl( + &self, + pairs: impl IntoIterator>, + ttls: impl IntoIterator, ) -> Result<()> { debug!("invoking raw batch_put request"); - let request = new_raw_batch_put_request( - pairs.into_iter().map(Into::into), - self.cf.clone(), - self.atomic, - ); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let pairs = pairs + .into_iter() + .map(|pair| pair.into().encode_keyspace(self.keyspace, KeyMode::Raw)); + let request = + new_raw_batch_put_request(pairs, ttls.into_iter(), self.cf.clone(), self.atomic); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .retry_multi_region(self.backoff.clone()) .extract_error() .plan(); @@ -348,9 +407,9 @@ impl> Client { /// ``` pub async fn delete(&self, key: impl Into) -> Result<()> { debug!("invoking raw delete request"); - let request = new_raw_delete_request(key.into(), self.cf.clone(), self.atomic); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw); + let request = new_raw_delete_request(key, self.cf.clone(), self.atomic); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .retry_multi_region(self.backoff.clone()) .merge(CollectSingle) .extract_error() @@ -379,10 +438,11 @@ impl> Client { pub async fn batch_delete(&self, keys: impl IntoIterator>) -> Result<()> { debug!("invoking raw batch_delete request"); self.assert_non_atomic()?; - let request = - new_raw_batch_delete_request(keys.into_iter().map(Into::into), self.cf.clone()); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let keys = keys + .into_iter() + .map(|k| k.into().encode_keyspace(self.keyspace, KeyMode::Raw)); + let request = new_raw_batch_delete_request(keys, self.cf.clone()); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .retry_multi_region(self.backoff.clone()) .extract_error() .plan(); @@ -408,9 +468,9 @@ impl> Client { pub async fn delete_range(&self, range: impl Into) -> Result<()> { debug!("invoking raw delete_range request"); self.assert_non_atomic()?; - let request = new_raw_delete_range_request(range.into(), self.cf.clone()); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let range = range.into().encode_keyspace(self.keyspace, KeyMode::Raw); + let request = new_raw_delete_range_request(range, self.cf.clone()); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .retry_multi_region(self.backoff.clone()) .extract_error() .plan(); @@ -439,7 +499,40 @@ impl> Client { /// ``` pub async fn scan(&self, range: impl Into, limit: u32) -> Result> { debug!("invoking raw scan request"); - self.scan_inner(range.into(), limit, false).await + self.scan_inner(range.into(), limit, false, false).await + } + + /// Create a new 'scan' request but scans in "reverse" direction. + /// + /// Once resolved this request will result in a `Vec` of key-value pairs that lies in the specified range. + /// + /// If the number of eligible key-value pairs are greater than `limit`, + /// only the first `limit` pairs are returned, ordered by the key. + /// + /// + /// Reverse Scan queries continuous kv pairs in range [startKey, endKey), + /// from startKey(lowerBound) to endKey(upperBound) in reverse order, up to limit pairs. + /// The returned keys are in reversed lexicographical order. + /// If you want to include the endKey or exclude the startKey, push a '\0' to the key. + /// It doesn't support Scanning from "", because locating the last Region is not yet implemented. + /// # Examples + /// ```rust,no_run + /// # use tikv_client::{KvPair, Config, RawClient, IntoOwnedRange}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let client = RawClient::new(vec!["192.168.0.100"]).await.unwrap(); + /// let inclusive_range = "TiKV"..="TiDB"; + /// let req = client.scan_reverse(inclusive_range.into_owned(), 2); + /// let result: Vec = req.await.unwrap(); + /// # }); + /// ``` + pub async fn scan_reverse( + &self, + range: impl Into, + limit: u32, + ) -> Result> { + debug!("invoking raw reverse scan request"); + self.scan_inner(range.into(), limit, false, true).await } /// Create a new 'scan' request that only returns the keys. @@ -464,7 +557,40 @@ impl> Client { pub async fn scan_keys(&self, range: impl Into, limit: u32) -> Result> { debug!("invoking raw scan_keys request"); Ok(self - .scan_inner(range, limit, true) + .scan_inner(range, limit, true, false) + .await? + .into_iter() + .map(KvPair::into_key) + .collect()) + } + + /// Create a new 'scan' request that only returns the keys in reverse order. + /// + /// Once resolved this request will result in a `Vec` of keys that lies in the specified range. + /// + /// If the number of eligible keys are greater than `limit`, + /// only the first `limit` pairs are returned, ordered by the key. + /// + /// + /// # Examples + /// ```rust,no_run + /// # use tikv_client::{Key, Config, RawClient, IntoOwnedRange}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let client = RawClient::new(vec!["192.168.0.100"]).await.unwrap(); + /// let inclusive_range = "TiKV"..="TiDB"; + /// let req = client.scan_keys(inclusive_range.into_owned(), 2); + /// let result: Vec = req.await.unwrap(); + /// # }); + /// ``` + pub async fn scan_keys_reverse( + &self, + range: impl Into, + limit: u32, + ) -> Result> { + debug!("invoking raw scan_keys request"); + Ok(self + .scan_inner(range, limit, true, true) .await? .into_iter() .map(KvPair::into_key) @@ -560,14 +686,14 @@ impl> Client { ) -> Result<(Option, bool)> { debug!("invoking raw compare_and_swap request"); self.assert_atomic()?; + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw); let req = new_cas_request( - key.into(), + key, new_value.into(), previous_value.into(), self.cf.clone(), ); - let encoded_req = EncodedRequest::new(req, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, req) .retry_multi_region(self.backoff.clone()) .merge(CollectSingle) .post_process_default() @@ -581,22 +707,39 @@ impl> Client { copr_version_req: impl Into, ranges: impl IntoIterator>, request_builder: impl Fn(metapb::Region, Vec>) -> Vec + Send + Sync + 'static, - ) -> Result, Vec>)>> { + ) -> Result>, Vec)>> { let copr_version_req = copr_version_req.into(); semver::VersionReq::from_str(&copr_version_req)?; + let ranges = ranges + .into_iter() + .map(|range| range.into().encode_keyspace(self.keyspace, KeyMode::Raw)); + let keyspace = self.keyspace; + let request_builder = move |region, ranges: Vec>| { + request_builder( + region, + ranges + .into_iter() + .map(|range| range.truncate_keyspace(keyspace)) + .collect(), + ) + }; let req = new_raw_coprocessor_request( copr_name.into(), copr_version_req, - ranges.into_iter().map(Into::into), + ranges, request_builder, ); - let encoded_req = EncodedRequest::new(req, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, req) .preserve_shard() .retry_multi_region(self.backoff.clone()) .post_process_default() .plan(); - plan.execute().await + Ok(plan + .execute() + .await? + .into_iter() + .map(|(ranges, data)| (ranges.truncate_keyspace(keyspace), data)) + .collect()) } async fn scan_inner( @@ -604,6 +747,7 @@ impl> Client { range: impl Into, limit: u32, key_only: bool, + reverse: bool, ) -> Result> { if limit > MAX_RAW_KV_SCAN_LIMIT { return Err(Error::MaxScanLimitExceeded { @@ -611,8 +755,9 @@ impl> Client { max_limit: MAX_RAW_KV_SCAN_LIMIT, }); } + + let mut cur_range = range.into().encode_keyspace(self.keyspace, KeyMode::Raw); let mut result = Vec::new(); - let mut cur_range = range.into(); let mut scan_regions = self.rpc.clone().stores_for_range(cur_range.clone()).boxed(); let mut region_store = scan_regions @@ -622,11 +767,16 @@ impl> Client { range: (cur_range.clone()), })??; let mut cur_limit = limit; + while cur_limit > 0 { - let request = - new_raw_scan_request(cur_range.clone(), cur_limit, key_only, self.cf.clone()); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let resp = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let request = new_raw_scan_request( + cur_range.clone(), + cur_limit, + key_only, + reverse, + self.cf.clone(), + ); + let resp = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .single_region_with_store(region_store.clone()) .await? .plan() @@ -639,6 +789,7 @@ impl> Client { .collect::>(); let res_len = region_scan_res.len(); result.append(&mut region_scan_res); + // if the number of results is less than cur_limit, it means this scan range contains more than one region, so we need to scan next region if res_len < cur_limit as usize { region_store = match scan_regions.next().await { @@ -650,15 +801,20 @@ impl> Client { rs } Some(Err(e)) => return Err(e), - None => return Ok(result), + None => break, }; cur_limit -= res_len as u32; } else { break; } } + // limit is a soft limit, so we need check the number of results result.truncate(limit as usize); + + // truncate the prefix of keys + let result = result.truncate_keyspace(self.keyspace); + Ok(result) } @@ -675,18 +831,20 @@ impl> Client { }); } - let request = new_raw_batch_scan_request( - ranges.into_iter().map(Into::into), - each_limit, - key_only, - self.cf.clone(), - ); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req) + let ranges = ranges + .into_iter() + .map(|range| range.into().encode_keyspace(self.keyspace, KeyMode::Raw)); + + let request = new_raw_batch_scan_request(ranges, each_limit, key_only, self.cf.clone()); + let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request) .retry_multi_region(self.backoff.clone()) .merge(Collect) .plan(); - plan.execute().await + plan.execute().await.map(|r| { + r.into_iter() + .map(|pair| pair.truncate_keyspace(self.keyspace)) + .collect() + }) } fn assert_non_atomic(&self) -> Result<()> { @@ -717,6 +875,36 @@ mod tests { use crate::proto::kvrpcpb; use crate::Result; + #[tokio::test] + async fn test_batch_put_with_ttl() -> Result<()> { + let pd_client = Arc::new(MockPdClient::new(MockKvClient::with_dispatch_hook( + move |req: &dyn Any| { + if req.downcast_ref::().is_some() { + let resp = kvrpcpb::RawBatchPutResponse { + ..Default::default() + }; + Ok(Box::new(resp) as Box) + } else { + unreachable!() + } + }, + ))); + let client = Client { + rpc: pd_client, + cf: Some(ColumnFamily::Default), + backoff: DEFAULT_REGION_BACKOFF, + atomic: false, + keyspace: Keyspace::Enable { keyspace_id: 0 }, + }; + let pairs = vec![ + KvPair(vec![11].into(), vec![12]), + KvPair(vec![11].into(), vec![12]), + ]; + let ttls = vec![0, 0]; + assert!(client.batch_put_with_ttl(pairs, ttls).await.is_ok()); + Ok(()) + } + #[tokio::test] async fn test_raw_coprocessor() -> Result<()> { let pd_client = Arc::new(MockPdClient::new(MockKvClient::with_dispatch_hook( @@ -739,6 +927,7 @@ mod tests { cf: Some(ColumnFamily::Default), backoff: DEFAULT_REGION_BACKOFF, atomic: false, + keyspace: Keyspace::Enable { keyspace_id: 0 }, }; let resps = client .coprocessor( @@ -750,27 +939,17 @@ mod tests { .await?; let resps: Vec<_> = resps .into_iter() - .map(|(data, ranges)| (String::from_utf8(data).unwrap(), ranges)) + .map(|(ranges, data)| (ranges, String::from_utf8(data).unwrap())) .collect(); assert_eq!( resps, - vec![ - ( - "1:[Key(05)..Key(0A)]".to_string(), - vec![Key::from(vec![5])..Key::from(vec![10])] - ), - ( - "2:[Key(0A)..Key(0F), Key(14)..Key(FAFA)]".to_string(), - vec![ - Key::from(vec![10])..Key::from(vec![15]), - Key::from(vec![20])..Key::from(vec![250, 250]) - ] - ), - ( - "3:[Key(FAFA)..Key()]".to_string(), - vec![Key::from(vec![250, 250])..Key::from(vec![])] - ) - ] + vec![( + vec![ + Key::from(vec![5])..Key::from(vec![15]), + Key::from(vec![20])..Key::from(vec![]) + ], + "2:[Key(05)..Key(0F), Key(14)..Key()]".to_string(), + ),] ); Ok(()) } diff --git a/src/raw/lowering.rs b/src/raw/lowering.rs index 30db593..3065401 100644 --- a/src/raw/lowering.rs +++ b/src/raw/lowering.rs @@ -28,21 +28,33 @@ pub fn new_raw_batch_get_request( requests::new_raw_batch_get_request(keys.map(Into::into).collect(), cf) } +pub fn new_raw_get_key_ttl_request( + key: Key, + cf: Option, +) -> kvrpcpb::RawGetKeyTtlRequest { + requests::new_raw_get_key_ttl_request(key.into(), cf) +} + pub fn new_raw_put_request( key: Key, value: Value, cf: Option, + ttl: u64, atomic: bool, ) -> kvrpcpb::RawPutRequest { - requests::new_raw_put_request(key.into(), value, cf, atomic) + requests::new_raw_put_request(key.into(), value, ttl, cf, atomic) } pub fn new_raw_batch_put_request( pairs: impl Iterator, + ttls: impl Iterator, cf: Option, atomic: bool, ) -> kvrpcpb::RawBatchPutRequest { - requests::new_raw_batch_put_request(pairs.map(Into::into).collect(), cf, atomic) + let pairs = pairs.map(Into::into).collect::>(); + let ttls = ttls.take(pairs.len()).collect::>(); + assert_eq!(pairs.len(), ttls.len()); + requests::new_raw_batch_put_request(pairs, ttls, cf, atomic) } pub fn new_raw_delete_request( @@ -72,6 +84,7 @@ pub fn new_raw_scan_request( range: BoundRange, limit: u32, key_only: bool, + reverse: bool, cf: Option, ) -> kvrpcpb::RawScanRequest { let (start_key, end_key) = range.into_keys(); @@ -80,6 +93,7 @@ pub fn new_raw_scan_request( end_key.unwrap_or_default().into(), limit, key_only, + reverse, cf, ) } diff --git a/src/raw/requests.rs b/src/raw/requests.rs index 0be733c..4422c88 100644 --- a/src/raw/requests.rs +++ b/src/raw/requests.rs @@ -1,22 +1,14 @@ // Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. -use std::any::Any; -use std::ops::Range; -use std::sync::Arc; -use std::time::Duration; - -use async_trait::async_trait; -use futures::stream::BoxStream; -use tonic::transport::Channel; - use super::RawRpcRequest; -use crate::collect_first; +use crate::collect_single; +use crate::kv::KvPairTTL; use crate::pd::PdClient; use crate::proto::kvrpcpb; -use crate::proto::kvrpcpb::ApiVersion; use crate::proto::metapb; use crate::proto::tikvpb::tikv_client::TikvClient; use crate::range_request; +use crate::region::RegionWithLeader; use crate::request::plan::ResponseWithShard; use crate::request::Collect; use crate::request::CollectSingle; @@ -41,6 +33,13 @@ use crate::Key; use crate::KvPair; use crate::Result; use crate::Value; +use async_trait::async_trait; +use futures::stream::BoxStream; +use std::any::Any; +use std::ops::Range; +use std::sync::Arc; +use std::time::Duration; +use tonic::transport::Channel; pub fn new_raw_get_request(key: Vec, cf: Option) -> kvrpcpb::RawGetRequest { let mut req = kvrpcpb::RawGetRequest::default(); @@ -55,7 +54,7 @@ impl KvRequest for kvrpcpb::RawGetRequest { } shardable_key!(kvrpcpb::RawGetRequest); -collect_first!(kvrpcpb::RawGetResponse); +collect_single!(kvrpcpb::RawGetResponse); impl SingleKey for kvrpcpb::RawGetRequest { fn key(&self) -> &Vec { @@ -104,15 +103,54 @@ impl Merge for Collect { } } +pub fn new_raw_get_key_ttl_request( + key: Vec, + cf: Option, +) -> kvrpcpb::RawGetKeyTtlRequest { + let mut req = kvrpcpb::RawGetKeyTtlRequest::default(); + req.key = key; + req.maybe_set_cf(cf); + + req +} + +impl KvRequest for kvrpcpb::RawGetKeyTtlRequest { + type Response = kvrpcpb::RawGetKeyTtlResponse; +} + +shardable_key!(kvrpcpb::RawGetKeyTtlRequest); +collect_single!(kvrpcpb::RawGetKeyTtlResponse); + +impl SingleKey for kvrpcpb::RawGetKeyTtlRequest { + fn key(&self) -> &Vec { + &self.key + } +} + +impl Process for DefaultProcessor { + type Out = Option; + + fn process(&self, input: Result) -> Result { + let input = input?; + Ok(if input.not_found { + None + } else { + Some(input.ttl) + }) + } +} + pub fn new_raw_put_request( key: Vec, value: Vec, + ttl: u64, cf: Option, atomic: bool, ) -> kvrpcpb::RawPutRequest { let mut req = kvrpcpb::RawPutRequest::default(); req.key = key; req.value = value; + req.ttl = ttl; req.maybe_set_cf(cf); req.for_cas = atomic; @@ -124,7 +162,7 @@ impl KvRequest for kvrpcpb::RawPutRequest { } shardable_key!(kvrpcpb::RawPutRequest); -collect_first!(kvrpcpb::RawPutResponse); +collect_single!(kvrpcpb::RawPutResponse); impl SingleKey for kvrpcpb::RawPutRequest { fn key(&self) -> &Vec { &self.key @@ -133,11 +171,13 @@ impl SingleKey for kvrpcpb::RawPutRequest { pub fn new_raw_batch_put_request( pairs: Vec, + ttls: Vec, cf: Option, atomic: bool, ) -> kvrpcpb::RawBatchPutRequest { let mut req = kvrpcpb::RawBatchPutRequest::default(); req.pairs = pairs; + req.ttls = ttls; req.maybe_set_cf(cf); req.for_cas = atomic; @@ -149,23 +189,28 @@ impl KvRequest for kvrpcpb::RawBatchPutRequest { } impl Shardable for kvrpcpb::RawBatchPutRequest { - type Shard = Vec; + type Shard = Vec<(kvrpcpb::KvPair, u64)>; fn shards( &self, pd_client: &Arc, ) -> BoxStream<'static, Result<(Self::Shard, RegionStore)>> { - let mut pairs = self.pairs.clone(); - pairs.sort_by(|a, b| a.key.cmp(&b.key)); - store_stream_for_keys( - pairs.into_iter().map(Into::::into), - pd_client.clone(), - ) + let kvs = self.pairs.clone(); + let ttls = self.ttls.clone(); + let mut kv_ttl: Vec = kvs + .into_iter() + .zip(ttls) + .map(|(kv, ttl)| KvPairTTL(kv, ttl)) + .collect(); + kv_ttl.sort_by(|a, b| a.0.key.cmp(&b.0.key)); + store_stream_for_keys(kv_ttl.into_iter(), pd_client.clone()) } fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.set_context(store.region_with_leader.context()?); - self.pairs = shard; + let (pairs, ttls) = shard.into_iter().unzip(); + self.set_leader(&store.region_with_leader)?; + self.pairs = pairs; + self.ttls = ttls; Ok(()) } } @@ -188,7 +233,7 @@ impl KvRequest for kvrpcpb::RawDeleteRequest { } shardable_key!(kvrpcpb::RawDeleteRequest); -collect_first!(kvrpcpb::RawDeleteResponse); +collect_single!(kvrpcpb::RawDeleteResponse); impl SingleKey for kvrpcpb::RawDeleteRequest { fn key(&self) -> &Vec { &self.key @@ -237,13 +282,20 @@ pub fn new_raw_scan_request( end_key: Vec, limit: u32, key_only: bool, + reverse: bool, cf: Option, ) -> kvrpcpb::RawScanRequest { let mut req = kvrpcpb::RawScanRequest::default(); - req.start_key = start_key; - req.end_key = end_key; + if !reverse { + req.start_key = start_key; + req.end_key = end_key; + } else { + req.start_key = end_key; + req.end_key = start_key; + } req.limit = limit; req.key_only = key_only; + req.reverse = reverse; req.maybe_set_cf(cf); req @@ -253,7 +305,7 @@ impl KvRequest for kvrpcpb::RawScanRequest { type Response = kvrpcpb::RawScanResponse; } -range_request!(kvrpcpb::RawScanRequest); // TODO: support reverse raw scan. +range_request!(kvrpcpb::RawScanRequest); shardable_range!(kvrpcpb::RawScanRequest); impl Merge for Collect { @@ -297,7 +349,7 @@ impl Shardable for kvrpcpb::RawBatchScanRequest { } fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; self.ranges = shard; Ok(()) } @@ -336,7 +388,7 @@ impl KvRequest for kvrpcpb::RawCasRequest { } shardable_key!(kvrpcpb::RawCasRequest); -collect_first!(kvrpcpb::RawCasResponse); +collect_single!(kvrpcpb::RawCasResponse); impl SingleKey for kvrpcpb::RawCasRequest { fn key(&self) -> &Vec { &self.key @@ -399,11 +451,11 @@ impl Request for RawCoprocessorRequest { self.inner.as_any() } - fn set_context(&mut self, context: kvrpcpb::Context) { - self.inner.set_context(context); + fn set_leader(&mut self, leader: &RegionWithLeader) -> Result<()> { + self.inner.set_leader(leader) } - fn set_api_version(&mut self, api_version: ApiVersion) { + fn set_api_version(&mut self, api_version: kvrpcpb::ApiVersion) { self.inner.set_api_version(api_version); } } @@ -423,8 +475,8 @@ impl Shardable for RawCoprocessorRequest { } fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.inner.context = Some(store.region_with_leader.context()?); - self.inner.ranges = shard.clone(); + self.set_leader(&store.region_with_leader)?; + self.inner.ranges.clone_from(&shard); self.inner.data = (self.data_builder)(store.region_with_leader.region.clone(), shard); Ok(()) } @@ -435,7 +487,7 @@ impl Process>>>> for DefaultProcessor { - type Out = Vec<(Vec, Vec>)>; + type Out = Vec<(Vec>, Vec)>; fn process( &self, @@ -448,11 +500,11 @@ impl .map(|shard_resp| { shard_resp.map(|ResponseWithShard(resp, ranges)| { ( - resp.data, ranges .into_iter() .map(|range| range.start_key.into()..range.end_key.into()) .collect(), + resp.data, ) }) }) @@ -472,6 +524,7 @@ macro_rules! impl_raw_rpc_request { impl_raw_rpc_request!(RawGetRequest); impl_raw_rpc_request!(RawBatchGetRequest); +impl_raw_rpc_request!(RawGetKeyTtlRequest); impl_raw_rpc_request!(RawPutRequest); impl_raw_rpc_request!(RawBatchPutRequest); impl_raw_rpc_request!(RawDeleteRequest); @@ -482,22 +535,35 @@ impl_raw_rpc_request!(RawDeleteRangeRequest); impl_raw_rpc_request!(RawCasRequest); impl HasLocks for kvrpcpb::RawGetResponse {} + impl HasLocks for kvrpcpb::RawBatchGetResponse {} + +impl HasLocks for kvrpcpb::RawGetKeyTtlResponse {} + impl HasLocks for kvrpcpb::RawPutResponse {} + impl HasLocks for kvrpcpb::RawBatchPutResponse {} + impl HasLocks for kvrpcpb::RawDeleteResponse {} + impl HasLocks for kvrpcpb::RawBatchDeleteResponse {} + impl HasLocks for kvrpcpb::RawScanResponse {} + impl HasLocks for kvrpcpb::RawBatchScanResponse {} + impl HasLocks for kvrpcpb::RawDeleteRangeResponse {} + impl HasLocks for kvrpcpb::RawCasResponse {} + impl HasLocks for kvrpcpb::RawCoprocessorResponse {} #[cfg(test)] mod test { use std::any::Any; - - use futures::executor; + use std::collections::HashMap; + use std::ops::Deref; + use std::sync::Mutex; use super::*; use crate::backoff::DEFAULT_REGION_BACKOFF; @@ -505,13 +571,14 @@ mod test { use crate::mock::MockKvClient; use crate::mock::MockPdClient; use crate::proto::kvrpcpb; - use crate::request::codec::EncodedRequest; + use crate::request::Keyspace; use crate::request::Plan; - use crate::Key; - #[test] - #[ignore] - fn test_raw_scan() { + #[rstest::rstest] + #[case(Keyspace::Disable)] + #[case(Keyspace::Enable { keyspace_id: 0 })] + #[tokio::test] + async fn test_raw_scan(#[case] keyspace: Keyspace) { let client = Arc::new(MockPdClient::new(MockKvClient::with_dispatch_hook( |req: &dyn Any| { let req: &kvrpcpb::RawScanRequest = req.downcast_ref().unwrap(); @@ -540,15 +607,68 @@ mod test { key_only: true, ..Default::default() }; - let encoded_scan = EncodedRequest::new(scan, client.get_codec()); - let plan = crate::request::PlanBuilder::new(client, encoded_scan) - .resolve_lock(OPTIMISTIC_BACKOFF) + let plan = crate::request::PlanBuilder::new(client, keyspace, scan) + .resolve_lock(OPTIMISTIC_BACKOFF, keyspace) .retry_multi_region(DEFAULT_REGION_BACKOFF) .merge(Collect) .plan(); - let scan = executor::block_on(async { plan.execute().await }).unwrap(); + let scan = plan.execute().await.unwrap(); - assert_eq!(scan.len(), 10); + assert_eq!(scan.len(), 49); // FIXME test the keys returned. } + + #[tokio::test] + async fn test_raw_batch_put() -> Result<()> { + let region1_kvs = vec![KvPair(vec![9].into(), vec![12])]; + let region1_ttls = vec![0]; + let region2_kvs = vec![ + KvPair(vec![11].into(), vec![12]), + KvPair("FFF".to_string().as_bytes().to_vec().into(), vec![12]), + ]; + let region2_ttls = vec![0, 1]; + + let expected_map = HashMap::from([ + (region1_kvs.clone(), region1_ttls.clone()), + (region2_kvs.clone(), region2_ttls.clone()), + ]); + + let pairs: Vec = [region1_kvs, region2_kvs] + .concat() + .into_iter() + .map(|kv| kv.into()) + .collect(); + let ttls = [region1_ttls, region2_ttls].concat(); + let cf = ColumnFamily::Default; + + let actual_map: Arc, Vec>>> = + Arc::new(Mutex::new(HashMap::new())); + let fut_actual_map = actual_map.clone(); + let client = Arc::new(MockPdClient::new(MockKvClient::with_dispatch_hook( + move |req: &dyn Any| { + let req: &kvrpcpb::RawBatchPutRequest = req.downcast_ref().unwrap(); + let kv_pair = req + .pairs + .clone() + .into_iter() + .map(|p| p.into()) + .collect::>(); + let ttls = req.ttls.clone(); + fut_actual_map.lock().unwrap().insert(kv_pair, ttls); + let resp = kvrpcpb::RawBatchPutResponse::default(); + Ok(Box::new(resp) as Box) + }, + ))); + + let batch_put_request = + new_raw_batch_put_request(pairs.clone(), ttls.clone(), Some(cf), false); + let keyspace = Keyspace::Enable { keyspace_id: 0 }; + let plan = crate::request::PlanBuilder::new(client, keyspace, batch_put_request) + .resolve_lock(OPTIMISTIC_BACKOFF, keyspace) + .retry_multi_region(DEFAULT_REGION_BACKOFF) + .plan(); + let _ = plan.execute().await; + assert_eq!(actual_map.lock().unwrap().deref(), &expected_map); + Ok(()) + } } diff --git a/src/region.rs b/src/region.rs index 8e58522..6fb2032 100644 --- a/src/region.rs +++ b/src/region.rs @@ -2,7 +2,6 @@ use derive_new::new; -use crate::proto::kvrpcpb; use crate::proto::metapb; use crate::Error; use crate::Key; @@ -43,21 +42,6 @@ impl RegionWithLeader { key >= start_key.as_slice() && (key < end_key.as_slice() || end_key.is_empty()) } - pub fn context(&self) -> Result { - self.leader - .as_ref() - .ok_or(Error::LeaderNotFound { - region_id: self.region.id, - }) - .map(|l| { - let mut ctx = kvrpcpb::Context::default(); - ctx.region_id = self.region.id; - ctx.region_epoch = self.region.region_epoch.clone(); - ctx.peer = Some(l.clone()); - ctx - }) - } - pub fn start_key(&self) -> Key { self.region.start_key.to_vec().into() } diff --git a/src/region_cache.rs b/src/region_cache.rs index a557a96..e56068c 100644 --- a/src/region_cache.rs +++ b/src/region_cache.rs @@ -282,6 +282,7 @@ mod test { use super::RegionCache; use crate::common::Error; use crate::pd::RetryClientTrait; + use crate::proto::keyspacepb; use crate::proto::metapb::RegionEpoch; use crate::proto::metapb::{self}; use crate::region::RegionId; @@ -346,6 +347,10 @@ mod test { async fn update_safepoint(self: Arc, _safepoint: u64) -> Result { todo!() } + + async fn load_keyspace(&self, _keyspace: &str) -> Result { + unimplemented!() + } } #[tokio::test] @@ -570,10 +575,10 @@ mod test { }); assert!(!is_valid_tikv_store(&store)); - store.labels[1].value = "tiflash_compute".to_owned(); + store.labels[1].value = "tiflash_compute".to_string(); assert!(!is_valid_tikv_store(&store)); - store.labels[1].value = "other".to_owned(); + store.labels[1].value = "other".to_string(); assert!(is_valid_tikv_store(&store)); } } diff --git a/src/request/codec.rs b/src/request/codec.rs deleted file mode 100644 index a409a8e..0000000 --- a/src/request/codec.rs +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright 2023 TiKV Project Authors. Licensed under Apache-2.0. - -use crate::proto::kvrpcpb; -use crate::request::KvRequest; - -pub trait Codec: Clone + Sync + Send + 'static { - fn encode_request(&self, _req: &mut R) {} - // TODO: fn decode_response() -} - -#[derive(Clone, Default)] -pub struct ApiV1TxnCodec {} - -impl Codec for ApiV1TxnCodec {} - -#[derive(Clone, Default)] -pub struct ApiV1RawCodec {} - -impl Codec for ApiV1RawCodec {} - -#[derive(Clone)] -pub struct ApiV2TxnCodec { - _keyspace_id: u32, -} - -impl ApiV2TxnCodec { - pub fn new(keyspace_id: u32) -> Self { - Self { - _keyspace_id: keyspace_id, - } - } -} - -impl Codec for ApiV2TxnCodec { - fn encode_request(&self, req: &mut R) { - req.set_api_version(kvrpcpb::ApiVersion::V2); - // TODO: req.encode_request(self); - } -} - -// TODO: pub struct ApiV2RawCodec - -// EncodeRequest is just a type wrapper to avoid passing not encoded request to `PlanBuilder` by mistake. -#[derive(Clone)] -pub struct EncodedRequest { - pub inner: Req, -} - -impl EncodedRequest { - pub fn new(mut req: Req, codec: &C) -> Self { - codec.encode_request(&mut req); - Self { inner: req } - } -} diff --git a/src/request/keyspace.rs b/src/request/keyspace.rs new file mode 100644 index 0000000..118e6fb --- /dev/null +++ b/src/request/keyspace.rs @@ -0,0 +1,280 @@ +// Copyright 2023 TiKV Project Authors. Licensed under Apache-2.0. + +use std::ops::{Bound, Range}; + +use serde_derive::{Deserialize, Serialize}; + +use crate::transaction::Mutation; +use crate::{proto::kvrpcpb, Key}; +use crate::{BoundRange, KvPair}; + +pub const RAW_KEY_PREFIX: u8 = b'r'; +pub const TXN_KEY_PREFIX: u8 = b'x'; +pub const KEYSPACE_PREFIX_LEN: usize = 4; + +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize)] +pub enum Keyspace { + Disable, + Enable { keyspace_id: u32 }, +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize)] +pub enum KeyMode { + Raw, + Txn, +} + +impl Keyspace { + pub fn api_version(&self) -> kvrpcpb::ApiVersion { + match self { + Keyspace::Disable => kvrpcpb::ApiVersion::V1, + Keyspace::Enable { .. } => kvrpcpb::ApiVersion::V2, + } + } +} + +pub trait EncodeKeyspace { + fn encode_keyspace(self, keyspace: Keyspace, key_mode: KeyMode) -> Self; +} + +pub trait TruncateKeyspace { + fn truncate_keyspace(self, keyspace: Keyspace) -> Self; +} + +impl EncodeKeyspace for Key { + fn encode_keyspace(mut self, keyspace: Keyspace, key_mode: KeyMode) -> Self { + let prefix = match keyspace { + Keyspace::Disable => { + return self; + } + Keyspace::Enable { keyspace_id } => keyspace_prefix(keyspace_id, key_mode), + }; + + prepend_bytes(&mut self.0, &prefix); + + self + } +} + +impl EncodeKeyspace for KvPair { + fn encode_keyspace(mut self, keyspace: Keyspace, key_mode: KeyMode) -> Self { + self.0 = self.0.encode_keyspace(keyspace, key_mode); + self + } +} + +impl EncodeKeyspace for BoundRange { + fn encode_keyspace(mut self, keyspace: Keyspace, key_mode: KeyMode) -> Self { + self.from = match self.from { + Bound::Included(key) => Bound::Included(key.encode_keyspace(keyspace, key_mode)), + Bound::Excluded(key) => Bound::Excluded(key.encode_keyspace(keyspace, key_mode)), + Bound::Unbounded => { + let key = Key::from(vec![]); + Bound::Included(key.encode_keyspace(keyspace, key_mode)) + } + }; + self.to = match self.to { + Bound::Included(key) if !key.is_empty() => { + Bound::Included(key.encode_keyspace(keyspace, key_mode)) + } + Bound::Excluded(key) if !key.is_empty() => { + Bound::Excluded(key.encode_keyspace(keyspace, key_mode)) + } + _ => { + let key = Key::from(vec![]); + let keyspace = match keyspace { + Keyspace::Disable => Keyspace::Disable, + Keyspace::Enable { keyspace_id } => Keyspace::Enable { + keyspace_id: keyspace_id + 1, + }, + }; + Bound::Excluded(key.encode_keyspace(keyspace, key_mode)) + } + }; + self + } +} + +impl EncodeKeyspace for Mutation { + fn encode_keyspace(self, keyspace: Keyspace, key_mode: KeyMode) -> Self { + match self { + Mutation::Put(key, val) => Mutation::Put(key.encode_keyspace(keyspace, key_mode), val), + Mutation::Delete(key) => Mutation::Delete(key.encode_keyspace(keyspace, key_mode)), + } + } +} + +impl TruncateKeyspace for Key { + fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self { + if let Keyspace::Disable = keyspace { + return self; + } + + pretruncate_bytes::(&mut self.0); + + self + } +} + +impl TruncateKeyspace for KvPair { + fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self { + self.0 = self.0.truncate_keyspace(keyspace); + self + } +} + +impl TruncateKeyspace for Range { + fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self { + self.start = self.start.truncate_keyspace(keyspace); + self.end = self.end.truncate_keyspace(keyspace); + self + } +} + +impl TruncateKeyspace for Vec> { + fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self { + for range in &mut self { + take_mut::take(range, |range| range.truncate_keyspace(keyspace)); + } + self + } +} + +impl TruncateKeyspace for Vec { + fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self { + for pair in &mut self { + take_mut::take(pair, |pair| pair.truncate_keyspace(keyspace)); + } + self + } +} + +impl TruncateKeyspace for Vec { + fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self { + for lock in &mut self { + take_mut::take(&mut lock.key, |key| { + Key::from(key).truncate_keyspace(keyspace).into() + }); + take_mut::take(&mut lock.primary_lock, |primary| { + Key::from(primary).truncate_keyspace(keyspace).into() + }); + for secondary in lock.secondaries.iter_mut() { + take_mut::take(secondary, |secondary| { + Key::from(secondary).truncate_keyspace(keyspace).into() + }); + } + } + self + } +} + +fn keyspace_prefix(keyspace_id: u32, key_mode: KeyMode) -> [u8; KEYSPACE_PREFIX_LEN] { + let mut prefix = keyspace_id.to_be_bytes(); + prefix[0] = match key_mode { + KeyMode::Raw => RAW_KEY_PREFIX, + KeyMode::Txn => TXN_KEY_PREFIX, + }; + prefix +} + +fn prepend_bytes(vec: &mut Vec, prefix: &[u8; N]) { + unsafe { + vec.reserve_exact(N); + std::ptr::copy(vec.as_ptr(), vec.as_mut_ptr().add(N), vec.len()); + std::ptr::copy_nonoverlapping(prefix.as_ptr(), vec.as_mut_ptr(), N); + vec.set_len(vec.len() + N); + } +} + +fn pretruncate_bytes(vec: &mut Vec) { + assert!(vec.len() >= N); + unsafe { + std::ptr::copy(vec.as_ptr().add(N), vec.as_mut_ptr(), vec.len() - N); + vec.set_len(vec.len() - N); + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_keyspace_prefix() { + let key_mode = KeyMode::Raw; + assert_eq!(keyspace_prefix(0, key_mode), [b'r', 0, 0, 0]); + assert_eq!(keyspace_prefix(1, key_mode), [b'r', 0, 0, 1]); + assert_eq!(keyspace_prefix(0xFFFF, key_mode), [b'r', 0, 0xFF, 0xFF]); + + let key_mode = KeyMode::Txn; + assert_eq!(keyspace_prefix(0, key_mode), [b'x', 0, 0, 0]); + assert_eq!(keyspace_prefix(1, key_mode), [b'x', 0, 0, 1]); + assert_eq!(keyspace_prefix(0xFFFF, key_mode), [b'x', 0, 0xFF, 0xFF]); + } + + #[test] + fn test_encode_version() { + let keyspace = Keyspace::Enable { + keyspace_id: 0xDEAD, + }; + let key_mode = KeyMode::Raw; + + let key = Key::from(vec![0xBE, 0xEF]); + let expected_key = Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF]); + assert_eq!(key.encode_keyspace(keyspace, key_mode), expected_key); + + let bound: BoundRange = (Key::from(vec![0xDE, 0xAD])..Key::from(vec![0xBE, 0xEF])).into(); + let expected_bound: BoundRange = (Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xDE, 0xAD]) + ..Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF])) + .into(); + assert_eq!(bound.encode_keyspace(keyspace, key_mode), expected_bound); + + let bound: BoundRange = (..).into(); + let expected_bound: BoundRange = + (Key::from(vec![b'r', 0, 0xDE, 0xAD])..Key::from(vec![b'r', 0, 0xDE, 0xAE])).into(); + assert_eq!(bound.encode_keyspace(keyspace, key_mode), expected_bound); + + let bound: BoundRange = (Key::from(vec![])..Key::from(vec![])).into(); + let expected_bound: BoundRange = + (Key::from(vec![b'r', 0, 0xDE, 0xAD])..Key::from(vec![b'r', 0, 0xDE, 0xAE])).into(); + assert_eq!(bound.encode_keyspace(keyspace, key_mode), expected_bound); + + let bound: BoundRange = (Key::from(vec![])..=Key::from(vec![])).into(); + let expected_bound: BoundRange = + (Key::from(vec![b'r', 0, 0xDE, 0xAD])..Key::from(vec![b'r', 0, 0xDE, 0xAE])).into(); + assert_eq!(bound.encode_keyspace(keyspace, key_mode), expected_bound); + + let mutation = Mutation::Put(Key::from(vec![0xBE, 0xEF]), vec![4, 5, 6]); + let expected_mutation = Mutation::Put( + Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF]), + vec![4, 5, 6], + ); + assert_eq!( + mutation.encode_keyspace(keyspace, key_mode), + expected_mutation + ); + + let mutation = Mutation::Delete(Key::from(vec![0xBE, 0xEF])); + let expected_mutation = Mutation::Delete(Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF])); + assert_eq!( + mutation.encode_keyspace(keyspace, key_mode), + expected_mutation + ); + } + + #[test] + fn test_truncate_version() { + let key = Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF]); + let keyspace = Keyspace::Enable { + keyspace_id: 0xDEAD, + }; + let expected_key = Key::from(vec![0xBE, 0xEF]); + assert_eq!(key.truncate_keyspace(keyspace), expected_key); + + let key = Key::from(vec![b'x', 0, 0xDE, 0xAD, 0xBE, 0xEF]); + let keyspace = Keyspace::Enable { + keyspace_id: 0xDEAD, + }; + let expected_key = Key::from(vec![0xBE, 0xEF]); + assert_eq!(key.truncate_keyspace(keyspace), expected_key); + } +} diff --git a/src/request/mod.rs b/src/request/mod.rs index 8c3a45c..14de8e9 100644 --- a/src/request/mod.rs +++ b/src/request/mod.rs @@ -3,6 +3,10 @@ use async_trait::async_trait; use derive_new::new; +pub use self::keyspace::EncodeKeyspace; +pub use self::keyspace::KeyMode; +pub use self::keyspace::Keyspace; +pub use self::keyspace::TruncateKeyspace; pub use self::plan::Collect; pub use self::plan::CollectError; pub use self::plan::CollectSingle; @@ -33,7 +37,7 @@ use crate::store::Request; use crate::store::{HasKeyErrors, Store}; use crate::transaction::HasLocks; -pub mod codec; +mod keyspace; pub mod plan; mod plan_builder; mod shard; @@ -43,9 +47,6 @@ mod shard; pub trait KvRequest: Request + Sized + Clone + Sync + Send + 'static { /// The expected response to the request. type Response: HasKeyErrors + HasLocks + Clone + Send + 'static; - - // TODO: fn encode_request() - // TODO: fn decode_response() } /// For requests or plans which are handled at TiKV store (other than region) level. @@ -98,12 +99,10 @@ mod test { use super::*; use crate::mock::MockKvClient; use crate::mock::MockPdClient; - use crate::pd::PdClient; use crate::proto::kvrpcpb; - use crate::proto::kvrpcpb::ApiVersion; use crate::proto::pdpb::Timestamp; use crate::proto::tikvpb::tikv_client::TikvClient; - use crate::request::codec::EncodedRequest; + use crate::region::RegionWithLeader; use crate::store::store_stream_for_keys; use crate::store::HasRegionError; use crate::transaction::lowering::new_commit_request; @@ -113,7 +112,7 @@ mod test { #[tokio::test] async fn test_region_retry() { - #[derive(Clone)] + #[derive(Debug, Clone)] struct MockRpcResponse; impl HasKeyErrors for MockRpcResponse { @@ -149,11 +148,11 @@ mod test { self } - fn set_context(&mut self, _: kvrpcpb::Context) { - unreachable!(); + fn set_leader(&mut self, _: &RegionWithLeader) -> Result<()> { + Ok(()) } - fn set_api_version(&mut self, _api_version: ApiVersion) {} + fn set_api_version(&mut self, _: kvrpcpb::ApiVersion) {} } #[async_trait] @@ -199,9 +198,8 @@ mod test { |_: &dyn Any| Ok(Box::new(MockRpcResponse) as Box), ))); - let encoded_req = EncodedRequest::new(request, pd_client.get_codec()); - let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req) - .resolve_lock(Backoff::no_jitter_backoff(1, 1, 3)) + let plan = crate::request::PlanBuilder::new(pd_client.clone(), Keyspace::Disable, request) + .resolve_lock(Backoff::no_jitter_backoff(1, 1, 3), Keyspace::Disable) .retry_multi_region(Backoff::no_jitter_backoff(1, 1, 3)) .extract_error() .plan(); @@ -224,18 +222,18 @@ mod test { let key: Key = "key".to_owned().into(); let req = new_commit_request(iter::once(key), Timestamp::default(), Timestamp::default()); - let encoded_req = EncodedRequest::new(req, pd_client.get_codec()); // does not extract error - let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req.clone()) - .resolve_lock(OPTIMISTIC_BACKOFF) - .retry_multi_region(OPTIMISTIC_BACKOFF) - .plan(); + let plan = + crate::request::PlanBuilder::new(pd_client.clone(), Keyspace::Disable, req.clone()) + .resolve_lock(OPTIMISTIC_BACKOFF, Keyspace::Disable) + .retry_multi_region(OPTIMISTIC_BACKOFF) + .plan(); assert!(plan.execute().await.is_ok()); // extract error - let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req) - .resolve_lock(OPTIMISTIC_BACKOFF) + let plan = crate::request::PlanBuilder::new(pd_client.clone(), Keyspace::Disable, req) + .resolve_lock(OPTIMISTIC_BACKOFF, Keyspace::Disable) .retry_multi_region(OPTIMISTIC_BACKOFF) .extract_error() .plan(); diff --git a/src/request/plan.rs b/src/request/plan.rs index ab72e8a..369a2ff 100644 --- a/src/request/plan.rs +++ b/src/request/plan.rs @@ -35,6 +35,8 @@ use crate::util::iter::FlatMapOkIterExt; use crate::Error; use crate::Result; +use super::keyspace::Keyspace; + /// A plan for how to execute a request. A user builds up a plan with various /// options, then exectutes it. #[async_trait] @@ -485,7 +487,7 @@ pub struct CollectSingle; #[doc(hidden)] #[macro_export] -macro_rules! collect_first { +macro_rules! collect_single { ($type_: ty) => { impl Merge<$type_> for CollectSingle { type Out = $type_; @@ -546,6 +548,7 @@ pub struct ResolveLock { pub inner: P, pub pd_client: Arc, pub backoff: Backoff, + pub keyspace: Keyspace, } impl Clone for ResolveLock { @@ -554,6 +557,7 @@ impl Clone for ResolveLock { inner: self.inner.clone(), pd_client: self.pd_client.clone(), backoff: self.backoff.clone(), + keyspace: self.keyspace, } } } @@ -579,7 +583,7 @@ where } let pd_client = self.pd_client.clone(); - let live_locks = resolve_locks(locks, pd_client.clone()).await?; + let live_locks = resolve_locks(locks, pd_client.clone(), self.keyspace).await?; if live_locks.is_empty() { result = self.inner.execute().await?; } else { @@ -595,7 +599,7 @@ where } } -#[derive(Default)] +#[derive(Debug, Default)] pub struct CleanupLocksResult { pub region_error: Option, pub key_error: Option>, @@ -644,6 +648,7 @@ pub struct CleanupLocks { pub options: ResolveLocksOptions, pub store: Option, pub pd_client: Arc, + pub keyspace: Keyspace, pub backoff: Backoff, } @@ -655,6 +660,7 @@ impl Clone for CleanupLocks { options: self.options, store: None, pd_client: self.pd_client.clone(), + keyspace: self.keyspace, backoff: self.backoff.clone(), } } @@ -715,7 +721,12 @@ where let lock_size = locks.len(); match lock_resolver - .cleanup_locks(self.store.clone().unwrap(), locks, self.pd_client.clone()) + .cleanup_locks( + self.store.clone().unwrap(), + locks, + self.pd_client.clone(), + self.keyspace, + ) .await { Ok(()) => { @@ -891,6 +902,7 @@ mod test { inner: ErrPlan, backoff: Backoff::no_backoff(), pd_client: Arc::new(MockPdClient::default()), + keyspace: Keyspace::Disable, }, pd_client: Arc::new(MockPdClient::default()), backoff: Backoff::no_backoff(), diff --git a/src/request/plan_builder.rs b/src/request/plan_builder.rs index 8e2329e..c117d14 100644 --- a/src/request/plan_builder.rs +++ b/src/request/plan_builder.rs @@ -4,9 +4,9 @@ use std::marker::PhantomData; use std::sync::Arc; use super::plan::PreserveShard; +use super::Keyspace; use crate::backoff::Backoff; use crate::pd::PdClient; -use crate::request::codec::EncodedRequest; use crate::request::plan::{CleanupLocks, RetryableAllStores}; use crate::request::shard::HasNextBatch; use crate::request::Dispatch; @@ -47,11 +47,12 @@ pub struct Targetted; impl PlanBuilderPhase for Targetted {} impl PlanBuilder, NoTarget> { - pub fn new(pd_client: Arc, encoded_request: EncodedRequest) -> Self { + pub fn new(pd_client: Arc, keyspace: Keyspace, mut request: Req) -> Self { + request.set_api_version(keyspace.api_version()); PlanBuilder { pd_client, plan: Dispatch { - request: encoded_request.inner, + request, kv_client: None, }, phantom: PhantomData, @@ -69,7 +70,11 @@ impl PlanBuilder { impl PlanBuilder { /// If there is a lock error, then resolve the lock and retry the request. - pub fn resolve_lock(self, backoff: Backoff) -> PlanBuilder, Ph> + pub fn resolve_lock( + self, + backoff: Backoff, + keyspace: Keyspace, + ) -> PlanBuilder, Ph> where P::Result: HasLocks, { @@ -79,6 +84,7 @@ impl PlanBuilder { inner: self.plan, backoff, pd_client: self.pd_client, + keyspace, }, phantom: PhantomData, } @@ -89,6 +95,7 @@ impl PlanBuilder { ctx: ResolveLocksContext, options: ResolveLocksOptions, backoff: Backoff, + keyspace: Keyspace, ) -> PlanBuilder, Ph> where P: Shardable + NextBatch, @@ -103,6 +110,7 @@ impl PlanBuilder { store: None, backoff, pd_client: self.pd_client, + keyspace, }, phantom: PhantomData, } @@ -248,8 +256,7 @@ fn set_single_region_store( store: RegionStore, pd_client: Arc, ) -> Result, Targetted>> { - plan.request - .set_context(store.region_with_leader.context()?); + plan.request.set_leader(&store.region_with_leader)?; plan.kv_client = Some(store.client); Ok(PlanBuilder { plan, diff --git a/src/request/shard.rs b/src/request/shard.rs index ec23423..1f116f7 100644 --- a/src/request/shard.rs +++ b/src/request/shard.rs @@ -14,6 +14,7 @@ use crate::request::ResolveLock; use crate::store::RegionStore; use crate::store::Request; use crate::Result; +use std::fmt::Debug; macro_rules! impl_inner_shardable { () => { @@ -33,7 +34,7 @@ macro_rules! impl_inner_shardable { } pub trait Shardable { - type Shard: Clone + Send + Sync; + type Shard: Debug + Clone + Send + Sync; fn shards( &self, @@ -164,7 +165,7 @@ macro_rules! shardable_key { mut shard: Self::Shard, store: &$crate::store::RegionStore, ) -> $crate::Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; assert!(shard.len() == 1); self.key = shard.pop().unwrap(); Ok(()) @@ -197,7 +198,7 @@ macro_rules! shardable_keys { shard: Self::Shard, store: &$crate::store::RegionStore, ) -> $crate::Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; self.keys = shard.into_iter().map(Into::into).collect(); Ok(()) } @@ -257,12 +258,12 @@ macro_rules! shardable_range { shard: Self::Shard, store: &$crate::store::RegionStore, ) -> $crate::Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; // In a reverse range request, the range is in the meaning of [end_key, start_key), i.e. end_key <= x < start_key. // As a result, after obtaining start_key and end_key from PD, we need to swap their values when assigning them to the request. - self.start_key = shard.0.into(); - self.end_key = shard.1.into(); + self.start_key = shard.0; + self.end_key = shard.1; if self.is_reverse() { std::mem::swap(&mut self.start_key, &mut self.end_key); } diff --git a/src/store/errors.rs b/src/store/errors.rs index c9d6c77..47b6bdf 100644 --- a/src/store/errors.rs +++ b/src/store/errors.rs @@ -55,6 +55,7 @@ has_region_error!(kvrpcpb::GcResponse); has_region_error!(kvrpcpb::UnsafeDestroyRangeResponse); has_region_error!(kvrpcpb::RawGetResponse); has_region_error!(kvrpcpb::RawBatchGetResponse); +has_region_error!(kvrpcpb::RawGetKeyTtlResponse); has_region_error!(kvrpcpb::RawPutResponse); has_region_error!(kvrpcpb::RawBatchPutResponse); has_region_error!(kvrpcpb::RawDeleteResponse); @@ -103,6 +104,7 @@ macro_rules! has_str_error { } has_str_error!(kvrpcpb::RawGetResponse); +has_str_error!(kvrpcpb::RawGetKeyTtlResponse); has_str_error!(kvrpcpb::RawPutResponse); has_str_error!(kvrpcpb::RawBatchPutResponse); has_str_error!(kvrpcpb::RawDeleteResponse); diff --git a/src/store/mod.rs b/src/store/mod.rs index ee513cf..3781770 100644 --- a/src/store/mod.rs +++ b/src/store/mod.rs @@ -8,7 +8,6 @@ use std::cmp::max; use std::cmp::min; use std::sync::Arc; -use async_trait::async_trait; use derive_new::new; use futures::prelude::*; use futures::stream::BoxStream; @@ -38,21 +37,6 @@ pub struct Store { pub client: Arc, } -#[async_trait] -pub trait KvConnectStore: KvConnect { - async fn connect_to_store( - &self, - region: RegionWithLeader, - address: String, - ) -> Result { - log::info!("connect to tikv endpoint: {:?}", &address); - let client = self.connect(address.as_str()).await?; - Ok(RegionStore::new(region, Arc::new(client))) - } -} - -impl KvConnectStore for TikvConnect {} - /// Maps keys to a stream of stores. `key_data` must be sorted in increasing order pub fn store_stream_for_keys( key_data: impl Iterator + Send + Sync + 'static, diff --git a/src/store/request.rs b/src/store/request.rs index e11fc8f..9eaabe5 100644 --- a/src/store/request.rs +++ b/src/store/request.rs @@ -9,6 +9,7 @@ use tonic::IntoRequest; use crate::proto::kvrpcpb; use crate::proto::tikvpb::tikv_client::TikvClient; +use crate::store::RegionWithLeader; use crate::Error; use crate::Result; @@ -21,9 +22,7 @@ pub trait Request: Any + Sync + Send + 'static { ) -> Result>; fn label(&self) -> &'static str; fn as_any(&self) -> &dyn Any; - /// Set the context for the request. - /// Should always use `set_context` other than modify the `self.context` directly. - fn set_context(&mut self, context: kvrpcpb::Context); + fn set_leader(&mut self, leader: &RegionWithLeader) -> Result<()>; fn set_api_version(&mut self, api_version: kvrpcpb::ApiVersion); } @@ -54,19 +53,20 @@ macro_rules! impl_request { self } - fn set_context(&mut self, context: kvrpcpb::Context) { - let api_version = self - .context - .as_ref() - .map(|c| c.api_version) - .unwrap_or_default(); - self.context = Some(context); - self.set_api_version(kvrpcpb::ApiVersion::try_from(api_version).unwrap()); + fn set_leader(&mut self, leader: &RegionWithLeader) -> Result<()> { + let ctx = self.context.get_or_insert(kvrpcpb::Context::default()); + let leader_peer = leader.leader.as_ref().ok_or(Error::LeaderNotFound { + region_id: leader.region.id, + })?; + ctx.region_id = leader.region.id; + ctx.region_epoch = leader.region.region_epoch.clone(); + ctx.peer = Some(leader_peer.clone()); + Ok(()) } fn set_api_version(&mut self, api_version: kvrpcpb::ApiVersion) { - let context = self.context.get_or_insert(kvrpcpb::Context::default()); - context.api_version = api_version.into(); + let ctx = self.context.get_or_insert(kvrpcpb::Context::default()); + ctx.api_version = api_version.into(); } } }; @@ -74,6 +74,7 @@ macro_rules! impl_request { impl_request!(RawGetRequest, raw_get, "raw_get"); impl_request!(RawBatchGetRequest, raw_batch_get, "raw_batch_get"); +impl_request!(RawGetKeyTtlRequest, raw_get_key_ttl, "raw_get_key_ttl"); impl_request!(RawPutRequest, raw_put, "raw_put"); impl_request!(RawBatchPutRequest, raw_batch_put, "raw_batch_put"); impl_request!(RawDeleteRequest, raw_delete, "raw_delete"); diff --git a/src/transaction/buffer.rs b/src/transaction/buffer.rs index 202b366..be88ce8 100644 --- a/src/transaction/buffer.rs +++ b/src/transaction/buffer.rs @@ -12,6 +12,8 @@ use crate::KvPair; use crate::Result; use crate::Value; +use super::transaction::Mutation; + /// A caching layer which buffers reads and writes in a transaction. pub struct Buffer { primary_key: Option, @@ -244,12 +246,10 @@ impl Buffer { } } - pub(crate) fn mutate(&mut self, m: kvrpcpb::Mutation) { - let op = kvrpcpb::Op::try_from(m.op).unwrap(); - match op { - kvrpcpb::Op::Put => self.put(m.key.into(), m.value), - kvrpcpb::Op::Del => self.delete(m.key.into()), - _ => unimplemented!("only put and delete are supported in mutate"), + pub(crate) fn mutate(&mut self, m: Mutation) { + match m { + Mutation::Put(key, value) => self.put(key, value), + Mutation::Delete(key) => self.delete(key), }; } @@ -358,13 +358,13 @@ impl BufferEntry { BufferEntry::Cached(_) => return None, BufferEntry::Put(v) => { pb.op = kvrpcpb::Op::Put.into(); - pb.value = v.clone(); + pb.value.clone_from(v); } BufferEntry::Del => pb.op = kvrpcpb::Op::Del.into(), BufferEntry::Locked(_) => pb.op = kvrpcpb::Op::Lock.into(), BufferEntry::Insert(v) => { pb.op = kvrpcpb::Op::Insert.into(); - pb.value = v.clone(); + pb.value.clone_from(v); } BufferEntry::CheckNotExist => pb.op = kvrpcpb::Op::CheckNotExists.into(), }; diff --git a/src/transaction/client.rs b/src/transaction/client.rs index 4bcb16d..16d5c4f 100644 --- a/src/transaction/client.rs +++ b/src/transaction/client.rs @@ -10,8 +10,10 @@ use crate::config::Config; use crate::pd::PdClient; use crate::pd::PdRpcClient; use crate::proto::pdpb::Timestamp; -use crate::request::codec::{ApiV1TxnCodec, ApiV2TxnCodec, Codec, EncodedRequest}; use crate::request::plan::CleanupLocksResult; +use crate::request::EncodeKeyspace; +use crate::request::KeyMode; +use crate::request::Keyspace; use crate::request::Plan; use crate::timestamp::TimestampExt; use crate::transaction::lock::ResolveLocksOptions; @@ -44,19 +46,21 @@ const SCAN_LOCK_BATCH_SIZE: u32 = 1024; /// /// The returned results of transactional requests are [`Future`](std::future::Future)s that must be /// awaited to execute. -pub struct Client { - pd: Arc>, +pub struct Client { + pd: Arc, + keyspace: Keyspace, } -impl Clone for Client { +impl Clone for Client { fn clone(&self) -> Self { Self { pd: self.pd.clone(), + keyspace: self.keyspace, } } } -impl Client { +impl Client { /// Create a transactional [`Client`] and connect to the TiKV cluster. /// /// Because TiKV is managed by a [PD](https://github.com/pingcap/pd/) cluster, the endpoints for @@ -73,6 +77,7 @@ impl Client { /// # }); /// ``` pub async fn new>(pd_endpoints: Vec) -> Result { + // debug!("creating transactional client"); Self::new_with_config(pd_endpoints, Config::default()).await } @@ -101,36 +106,19 @@ impl Client { pd_endpoints: Vec, config: Config, ) -> Result { - Self::new_with_codec(pd_endpoints, config, ApiV1TxnCodec::default()).await - } -} - -impl Client { - pub async fn new_with_config_v2>( - _keyspace_name: &str, - pd_endpoints: Vec, - config: Config, - ) -> Result> { - debug!("creating new transactional client APIv2"); - let pd_endpoints: Vec = pd_endpoints.into_iter().map(Into::into).collect(); - let mut pd = PdRpcClient::connect(&pd_endpoints, config, true, None).await?; - let keyspace_id = 0; // TODO: get keyspace_id by pd.get_keyspace(keyspace_name) - pd.set_codec(ApiV2TxnCodec::new(keyspace_id)); - Ok(Client { pd: Arc::new(pd) }) - } -} - -impl Client { - pub async fn new_with_codec>( - pd_endpoints: Vec, - config: Config, - codec: Cod, - ) -> Result> { debug!("creating new transactional client"); let pd_endpoints: Vec = pd_endpoints.into_iter().map(Into::into).collect(); - let pd = - Arc::new(PdRpcClient::::connect(&pd_endpoints, config, true, Some(codec)).await?); - Ok(Client { pd }) + let pd = Arc::new(PdRpcClient::connect(&pd_endpoints, config.clone(), true).await?); + let keyspace = match config.keyspace { + Some(keyspace) => { + let keyspace = pd.load_keyspace(&keyspace).await?; + Keyspace::Enable { + keyspace_id: keyspace.id, + } + } + None => Keyspace::Disable, + }; + Ok(Client { pd, keyspace }) } /// Creates a new optimistic [`Transaction`]. @@ -153,7 +141,7 @@ impl Client { /// transaction.commit().await.unwrap(); /// # }); /// ``` - pub async fn begin_optimistic(&self) -> Result>> { + pub async fn begin_optimistic(&self) -> Result { debug!("creating new optimistic transaction"); let timestamp = self.current_timestamp().await?; Ok(self.new_transaction(timestamp, TransactionOptions::new_optimistic())) @@ -176,7 +164,7 @@ impl Client { /// transaction.commit().await.unwrap(); /// # }); /// ``` - pub async fn begin_pessimistic(&self) -> Result>> { + pub async fn begin_pessimistic(&self) -> Result { debug!("creating new pessimistic transaction"); let timestamp = self.current_timestamp().await?; Ok(self.new_transaction(timestamp, TransactionOptions::new_pessimistic())) @@ -199,21 +187,14 @@ impl Client { /// transaction.commit().await.unwrap(); /// # }); /// ``` - pub async fn begin_with_options( - &self, - options: TransactionOptions, - ) -> Result>> { + pub async fn begin_with_options(&self, options: TransactionOptions) -> Result { debug!("creating new customized transaction"); let timestamp = self.current_timestamp().await?; Ok(self.new_transaction(timestamp, options)) } /// Create a new [`Snapshot`](Snapshot) at the given [`Timestamp`](Timestamp). - pub fn snapshot( - &self, - timestamp: Timestamp, - options: TransactionOptions, - ) -> Snapshot> { + pub fn snapshot(&self, timestamp: Timestamp, options: TransactionOptions) -> Snapshot { debug!("creating new snapshot"); Snapshot::new(self.new_transaction(timestamp, options.read_only())) } @@ -279,10 +260,10 @@ impl Client { // scan all locks with ts <= safepoint let ctx = ResolveLocksContext::default(); let backoff = Backoff::equal_jitter_backoff(100, 10000, 50); - let req = new_scan_lock_request(range.into(), safepoint, options.batch_size); - let encoded_req = EncodedRequest::new(req, self.pd.get_codec()); - let plan = crate::request::PlanBuilder::new(self.pd.clone(), encoded_req) - .cleanup_locks(ctx.clone(), options, backoff) + let range = range.into().encode_keyspace(self.keyspace, KeyMode::Txn); + let req = new_scan_lock_request(range, safepoint, options.batch_size); + let plan = crate::request::PlanBuilder::new(self.pd.clone(), self.keyspace, req) + .cleanup_locks(ctx.clone(), options, backoff, self.keyspace) .retry_multi_region(DEFAULT_REGION_BACKOFF) .extract_error() .merge(crate::request::Collect) @@ -299,13 +280,15 @@ impl Client { range: impl Into, batch_size: u32, ) -> Result> { - let req = new_scan_lock_request(range.into(), safepoint, batch_size); - let encoded_req = EncodedRequest::new(req, self.pd.get_codec()); - let plan = crate::request::PlanBuilder::new(self.pd.clone(), encoded_req) + use crate::request::TruncateKeyspace; + + let range = range.into().encode_keyspace(self.keyspace, KeyMode::Txn); + let req = new_scan_lock_request(range, safepoint, batch_size); + let plan = crate::request::PlanBuilder::new(self.pd.clone(), self.keyspace, req) .retry_multi_region(DEFAULT_REGION_BACKOFF) .merge(crate::request::Collect) .plan(); - plan.execute().await + Ok(plan.execute().await?.truncate_keyspace(self.keyspace)) } /// Cleans up all keys in a range and quickly reclaim disk space. @@ -316,20 +299,16 @@ impl Client { /// /// This interface is intended for special scenarios that resemble operations like "drop table" or "drop database" in TiDB. pub async fn unsafe_destroy_range(&self, range: impl Into) -> Result<()> { - let req = new_unsafe_destroy_range_request(range.into()); - let encoded_req = EncodedRequest::new(req, self.pd.get_codec()); - let plan = crate::request::PlanBuilder::new(self.pd.clone(), encoded_req) + let range = range.into().encode_keyspace(self.keyspace, KeyMode::Txn); + let req = new_unsafe_destroy_range_request(range); + let plan = crate::request::PlanBuilder::new(self.pd.clone(), self.keyspace, req) .all_stores(DEFAULT_STORE_BACKOFF) .merge(crate::request::Collect) .plan(); plan.execute().await } - fn new_transaction( - &self, - timestamp: Timestamp, - options: TransactionOptions, - ) -> Transaction> { - Transaction::new(timestamp, self.pd.clone(), options) + fn new_transaction(&self, timestamp: Timestamp, options: TransactionOptions) -> Transaction { + Transaction::new(timestamp, self.pd.clone(), options, self.keyspace) } } diff --git a/src/transaction/lock.rs b/src/transaction/lock.rs index afb1d6c..efa835d 100644 --- a/src/transaction/lock.rs +++ b/src/transaction/lock.rs @@ -13,13 +13,14 @@ use crate::backoff::Backoff; use crate::backoff::DEFAULT_REGION_BACKOFF; use crate::backoff::OPTIMISTIC_BACKOFF; use crate::pd::PdClient; + use crate::proto::kvrpcpb; use crate::proto::kvrpcpb::TxnInfo; use crate::proto::pdpb::Timestamp; use crate::region::RegionVerId; -use crate::request::codec::EncodedRequest; use crate::request::Collect; use crate::request::CollectSingle; +use crate::request::Keyspace; use crate::request::Plan; use crate::store::RegionStore; use crate::timestamp::TimestampExt; @@ -44,6 +45,7 @@ const RESOLVE_LOCK_RETRY_LIMIT: usize = 10; pub async fn resolve_locks( locks: Vec, pd_client: Arc, + keyspace: Keyspace, ) -> Result /* live_locks */> { debug!("resolving locks"); let ts = pd_client.clone().get_timestamp().await?; @@ -76,9 +78,8 @@ pub async fn resolve_locks( Some(&commit_version) => commit_version, None => { let request = requests::new_cleanup_request(lock.primary_lock, lock.lock_version); - let encoded_req = EncodedRequest::new(request, pd_client.get_codec()); - let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req) - .resolve_lock(OPTIMISTIC_BACKOFF) + let plan = crate::request::PlanBuilder::new(pd_client.clone(), keyspace, request) + .resolve_lock(OPTIMISTIC_BACKOFF, keyspace) .retry_multi_region(DEFAULT_REGION_BACKOFF) .merge(CollectSingle) .post_process_default() @@ -94,6 +95,7 @@ pub async fn resolve_locks( lock.lock_version, commit_version, pd_client.clone(), + keyspace, ) .await?; clean_regions @@ -109,6 +111,7 @@ async fn resolve_lock_with_retry( start_version: u64, commit_version: u64, pd_client: Arc, + keyspace: Keyspace, ) -> Result { debug!("resolving locks with retry"); // FIXME: Add backoff @@ -118,11 +121,10 @@ async fn resolve_lock_with_retry( let store = pd_client.clone().store_for_key(key.into()).await?; let ver_id = store.region_with_leader.ver_id(); let request = requests::new_resolve_lock_request(start_version, commit_version); - let encoded_req = EncodedRequest::new(request, pd_client.get_codec()); - let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req) + let plan = crate::request::PlanBuilder::new(pd_client.clone(), keyspace, request) .single_region_with_store(store) .await? - .resolve_lock(Backoff::no_backoff()) + .resolve_lock(Backoff::no_backoff(), keyspace) .extract_error() .plan(); match plan.execute().await { @@ -214,6 +216,7 @@ impl LockResolver { store: RegionStore, locks: Vec, pd_client: Arc, // TODO: make pd_client a member of LockResolver + keyspace: Keyspace, ) -> Result<()> { if locks.is_empty() { return Ok(()); @@ -235,6 +238,7 @@ impl LockResolver { let mut status = self .check_txn_status( pd_client.clone(), + keyspace, txn_id, l.primary_lock.clone(), 0, @@ -249,7 +253,12 @@ impl LockResolver { // Then we need to check the secondary locks to determine the final status of the transaction. if let TransactionStatusKind::Locked(_, lock_info) = &status.kind { let secondary_status = self - .check_all_secondaries(pd_client.clone(), lock_info.secondaries.clone(), txn_id) + .check_all_secondaries( + pd_client.clone(), + keyspace, + lock_info.secondaries.clone(), + txn_id, + ) .await?; debug!( "secondary status, txn_id:{}, commit_ts:{:?}, min_commit_version:{}, fallback_2pc:{}", @@ -267,6 +276,7 @@ impl LockResolver { status = self .check_txn_status( pd_client.clone(), + keyspace, txn_id, l.primary_lock, 0, @@ -315,7 +325,7 @@ impl LockResolver { txn_info_vec.push(txn_info); } let cleaned_region = self - .batch_resolve_locks(pd_client.clone(), store.clone(), txn_info_vec) + .batch_resolve_locks(pd_client.clone(), keyspace, store.clone(), txn_info_vec) .await?; for txn_id in txn_ids { self.ctx @@ -330,6 +340,7 @@ impl LockResolver { pub async fn check_txn_status( &mut self, pd_client: Arc, + keyspace: Keyspace, txn_id: u64, primary: Vec, caller_start_ts: u64, @@ -359,8 +370,7 @@ impl LockResolver { force_sync_commit, resolving_pessimistic_lock, ); - let encoded_req = EncodedRequest::new(req, pd_client.get_codec()); - let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req) + let plan = crate::request::PlanBuilder::new(pd_client.clone(), keyspace, req) .retry_multi_region(DEFAULT_REGION_BACKOFF) .merge(CollectSingle) .extract_error() @@ -380,12 +390,12 @@ impl LockResolver { async fn check_all_secondaries( &mut self, pd_client: Arc, + keyspace: Keyspace, keys: Vec>, txn_id: u64, ) -> Result { let req = new_check_secondary_locks_request(keys, txn_id); - let encoded_req = EncodedRequest::new(req, pd_client.get_codec()); - let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req) + let plan = crate::request::PlanBuilder::new(pd_client.clone(), keyspace, req) .retry_multi_region(DEFAULT_REGION_BACKOFF) .extract_error() .merge(Collect) @@ -396,13 +406,13 @@ impl LockResolver { async fn batch_resolve_locks( &mut self, pd_client: Arc, + keyspace: Keyspace, store: RegionStore, txn_infos: Vec, ) -> Result { let ver_id = store.region_with_leader.ver_id(); let request = requests::new_batch_resolve_lock_request(txn_infos.clone()); - let encoded_req = EncodedRequest::new(request, pd_client.get_codec()); - let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req) + let plan = crate::request::PlanBuilder::new(pd_client.clone(), keyspace, request) .single_region_with_store(store.clone()) .await? .extract_error() @@ -422,13 +432,19 @@ pub trait HasLocks { mod tests { use std::any::Any; + use serial_test::serial; + use super::*; use crate::mock::MockKvClient; use crate::mock::MockPdClient; use crate::proto::errorpb; + #[rstest::rstest] + #[case(Keyspace::Disable)] + #[case(Keyspace::Enable { keyspace_id: 0 })] #[tokio::test] - async fn test_resolve_lock_with_retry() { + #[serial] + async fn test_resolve_lock_with_retry(#[case] keyspace: Keyspace) { // Test resolve lock within retry limit fail::cfg("region-error", "9*return").unwrap(); @@ -447,7 +463,7 @@ mod tests { let key = vec![1]; let region1 = MockPdClient::region1(); - let resolved_region = resolve_lock_with_retry(&key, 1, 2, client.clone()) + let resolved_region = resolve_lock_with_retry(&key, 1, 2, client.clone(), keyspace) .await .unwrap(); assert_eq!(region1.ver_id(), resolved_region); @@ -455,7 +471,7 @@ mod tests { // Test resolve lock over retry limit fail::cfg("region-error", "10*return").unwrap(); let key = vec![100]; - resolve_lock_with_retry(&key, 3, 4, client) + resolve_lock_with_retry(&key, 3, 4, client, keyspace) .await .expect_err("should return error"); } diff --git a/src/transaction/mod.rs b/src/transaction/mod.rs index 4b1f314..047ec61 100644 --- a/src/transaction/mod.rs +++ b/src/transaction/mod.rs @@ -16,6 +16,7 @@ pub use transaction::CheckLevel; pub use transaction::CommitTTLParameters; #[doc(hidden)] pub use transaction::HeartbeatOption; +pub use transaction::Mutation; pub use transaction::Transaction; pub use transaction::TransactionOptions; diff --git a/src/transaction/requests.rs b/src/transaction/requests.rs index 4f3e1b9..231c9e5 100644 --- a/src/transaction/requests.rs +++ b/src/transaction/requests.rs @@ -10,7 +10,7 @@ use futures::stream::{self}; use futures::StreamExt; use super::transaction::TXN_COMMIT_BATCH_SIZE; -use crate::collect_first; +use crate::collect_single; use crate::common::Error::PessimisticLockError; use crate::pd::PdClient; use crate::proto::kvrpcpb::Action; @@ -42,6 +42,7 @@ use crate::store::RegionStore; use crate::store::Request; use crate::store::{store_stream_for_keys, Store}; use crate::timestamp::TimestampExt; +use crate::transaction::requests::kvrpcpb::prewrite_request::PessimisticAction; use crate::transaction::HasLocks; use crate::util::iter::FlatMapOkIterExt; use crate::KvPair; @@ -101,7 +102,7 @@ impl KvRequest for kvrpcpb::GetRequest { } shardable_key!(kvrpcpb::GetRequest); -collect_first!(kvrpcpb::GetResponse); +collect_single!(kvrpcpb::GetResponse); impl SingleKey for kvrpcpb::GetRequest { fn key(&self) -> &Vec { &self.key @@ -224,7 +225,7 @@ impl KvRequest for kvrpcpb::CleanupRequest { } shardable_key!(kvrpcpb::CleanupRequest); -collect_first!(kvrpcpb::CleanupResponse); +collect_single!(kvrpcpb::CleanupResponse); impl SingleKey for kvrpcpb::CleanupRequest { fn key(&self) -> &Vec { &self.key @@ -251,7 +252,7 @@ pub fn new_prewrite_request( req.start_version = start_version; req.lock_ttl = lock_ttl; // FIXME: Lite resolve lock is currently disabled - req.txn_size = std::u64::MAX; + req.txn_size = u64::MAX; req } @@ -266,7 +267,9 @@ pub fn new_pessimistic_prewrite_request( let len = mutations.len(); let mut req = new_prewrite_request(mutations, primary_lock, start_version, lock_ttl); req.for_update_ts = for_update_ts; - req.is_pessimistic_lock = iter::repeat(true).take(len).collect(); + req.pessimistic_actions = iter::repeat(PessimisticAction::DoPessimisticCheck.into()) + .take(len) + .collect(); req } @@ -298,7 +301,7 @@ impl Shardable for kvrpcpb::PrewriteRequest { } fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; // Only need to set secondary keys if we're sending the primary key. if self.use_async_commit && !self.mutations.iter().any(|m| m.key == self.primary_lock) { @@ -365,7 +368,7 @@ impl Shardable for kvrpcpb::CommitRequest { } fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; self.keys = shard.into_iter().map(Into::into).collect(); Ok(()) } @@ -456,7 +459,7 @@ impl Shardable for kvrpcpb::PessimisticLockRequest { } fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; self.mutations = shard; Ok(()) } @@ -557,7 +560,7 @@ impl Shardable for kvrpcpb::ScanLockRequest { } fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; self.start_key = shard.0; Ok(()) } @@ -618,14 +621,14 @@ impl Shardable for kvrpcpb::TxnHeartBeatRequest { } fn apply_shard(&mut self, mut shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; assert!(shard.len() == 1); self.primary_lock = shard.pop().unwrap(); Ok(()) } } -collect_first!(TxnHeartBeatResponse); +collect_single!(TxnHeartBeatResponse); impl SingleKey for kvrpcpb::TxnHeartBeatRequest { fn key(&self) -> &Vec { @@ -676,7 +679,7 @@ impl Shardable for kvrpcpb::CheckTxnStatusRequest { } fn apply_shard(&mut self, mut shard: Self::Shard, store: &RegionStore) -> Result<()> { - self.set_context(store.region_with_leader.context()?); + self.set_leader(&store.region_with_leader)?; assert!(shard.len() == 1); self.primary_key = shard.pop().unwrap(); Ok(()) @@ -689,7 +692,7 @@ impl SingleKey for kvrpcpb::CheckTxnStatusRequest { } } -collect_first!(kvrpcpb::CheckTxnStatusResponse); +collect_single!(kvrpcpb::CheckTxnStatusResponse); impl Process for DefaultProcessor { type Out = TransactionStatus; diff --git a/src/transaction/snapshot.rs b/src/transaction/snapshot.rs index 0d1e480..5694614 100644 --- a/src/transaction/snapshot.rs +++ b/src/transaction/snapshot.rs @@ -2,11 +2,7 @@ use derive_new::new; use log::debug; -use std::marker::PhantomData; -use crate::codec::ApiV1TxnCodec; -use crate::pd::{PdClient, PdRpcClient}; -use crate::request::codec::Codec; use crate::BoundRange; use crate::Key; use crate::KvPair; @@ -22,12 +18,11 @@ use crate::Value; /// /// See the [Transaction](struct@crate::Transaction) docs for more information on the methods. #[derive(new)] -pub struct Snapshot = PdRpcClient> { - transaction: Transaction, - phantom: PhantomData, +pub struct Snapshot { + transaction: Transaction, } -impl> Snapshot { +impl Snapshot { /// Get the value associated with the given key. pub async fn get(&mut self, key: impl Into) -> Result> { debug!("invoking get request on snapshot"); diff --git a/src/transaction/transaction.rs b/src/transaction/transaction.rs index 764a1bf..7dbf9a3 100644 --- a/src/transaction/transaction.rs +++ b/src/transaction/transaction.rs @@ -1,7 +1,6 @@ // Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. use std::iter; -use std::marker::PhantomData; use std::sync::atomic; use std::sync::atomic::AtomicU8; use std::sync::Arc; @@ -16,19 +15,21 @@ use tokio::time::Duration; use crate::backoff::Backoff; use crate::backoff::DEFAULT_REGION_BACKOFF; -use crate::codec::ApiV1TxnCodec; use crate::pd::PdClient; use crate::pd::PdRpcClient; use crate::proto::kvrpcpb; use crate::proto::pdpb::Timestamp; -use crate::request::codec::{Codec, EncodedRequest}; use crate::request::Collect; use crate::request::CollectError; use crate::request::CollectSingle; use crate::request::CollectWithShard; +use crate::request::EncodeKeyspace; +use crate::request::KeyMode; +use crate::request::Keyspace; use crate::request::Plan; use crate::request::PlanBuilder; use crate::request::RetryOptions; +use crate::request::TruncateKeyspace; use crate::timestamp::TimestampExt; use crate::transaction::buffer::Buffer; use crate::transaction::lowering::*; @@ -77,23 +78,24 @@ use crate::Value; /// txn.commit().await.unwrap(); /// # }); /// ``` -pub struct Transaction = PdRpcClient> { +pub struct Transaction { status: Arc, timestamp: Timestamp, buffer: Buffer, rpc: Arc, options: TransactionOptions, + keyspace: Keyspace, is_heartbeat_started: bool, start_instant: Instant, - phantom: PhantomData, } -impl> Transaction { +impl Transaction { pub(crate) fn new( timestamp: Timestamp, rpc: Arc, options: TransactionOptions, - ) -> Transaction { + keyspace: Keyspace, + ) -> Transaction { let status = if options.read_only { TransactionStatus::ReadOnly } else { @@ -105,9 +107,9 @@ impl> Transaction { buffer: Buffer::new(options.is_pessimistic()), rpc, options, + keyspace, is_heartbeat_started: false, start_instant: std::time::Instant::now(), - phantom: PhantomData, } } @@ -134,15 +136,15 @@ impl> Transaction { self.check_allow_operation().await?; let timestamp = self.timestamp.clone(); let rpc = self.rpc.clone(); - let key = key.into(); + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn); let retry_options = self.options.retry_options.clone(); + let keyspace = self.keyspace; self.buffer .get_or_else(key, |key| async move { let request = new_get_request(key, timestamp); - let encoded_req = EncodedRequest::new(request, rpc.get_codec()); - let plan = PlanBuilder::new(rpc, encoded_req) - .resolve_lock(retry_options.lock_backoff) + let plan = PlanBuilder::new(rpc, keyspace, request) + .resolve_lock(retry_options.lock_backoff, keyspace) .retry_multi_region(DEFAULT_REGION_BACKOFF) .merge(CollectSingle) .post_process_default() @@ -202,7 +204,8 @@ impl> Transaction { self.lock_keys(iter::once(key.clone())).await?; self.get(key).await } else { - let mut pairs = self.pessimistic_lock(iter::once(key.into()), true).await?; + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn); + let mut pairs = self.pessimistic_lock(iter::once(key), true).await?; debug_assert!(pairs.len() <= 1); match pairs.pop() { Some(pair) => Ok(Some(pair.1)), @@ -266,14 +269,17 @@ impl> Transaction { self.check_allow_operation().await?; let timestamp = self.timestamp.clone(); let rpc = self.rpc.clone(); + let keyspace = self.keyspace; + let keys = keys + .into_iter() + .map(move |k| k.into().encode_keyspace(keyspace, KeyMode::Txn)); let retry_options = self.options.retry_options.clone(); self.buffer - .batch_get_or_else(keys.into_iter().map(|k| k.into()), move |keys| async move { + .batch_get_or_else(keys, move |keys| async move { let request = new_batch_get_request(keys, timestamp); - let encoded_req = EncodedRequest::new(request, rpc.get_codec()); - let plan = PlanBuilder::new(rpc, encoded_req) - .resolve_lock(retry_options.lock_backoff) + let plan = PlanBuilder::new(rpc, keyspace, request) + .resolve_lock(retry_options.lock_backoff, keyspace) .retry_multi_region(retry_options.region_backoff) .merge(Collect) .plan(); @@ -282,6 +288,7 @@ impl> Transaction { .map(|r| r.into_iter().map(Into::into).collect()) }) .await + .map(move |pairs| pairs.map(move |pair| pair.truncate_keyspace(keyspace))) } /// Create a new 'batch get for update' request. @@ -317,12 +324,20 @@ impl> Transaction { ) -> Result> { debug!("invoking transactional batch_get_for_update request"); self.check_allow_operation().await?; - let keys: Vec = keys.into_iter().map(|k| k.into()).collect(); if !self.is_pessimistic() { + let keys: Vec = keys.into_iter().map(|k| k.into()).collect(); self.lock_keys(keys.clone()).await?; Ok(self.batch_get(keys).await?.collect()) } else { - self.pessimistic_lock(keys, true).await + let keyspace = self.keyspace; + let keys = keys + .into_iter() + .map(move |k| k.into().encode_keyspace(keyspace, KeyMode::Txn)); + let pairs = self + .pessimistic_lock(keys, true) + .await? + .truncate_keyspace(keyspace); + Ok(pairs) } } @@ -448,7 +463,7 @@ impl> Transaction { pub async fn put(&mut self, key: impl Into, value: impl Into) -> Result<()> { debug!("invoking transactional put request"); self.check_allow_operation().await?; - let key = key.into(); + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn); if self.is_pessimistic() { self.pessimistic_lock(iter::once(key.clone()), false) .await?; @@ -479,7 +494,7 @@ impl> Transaction { pub async fn insert(&mut self, key: impl Into, value: impl Into) -> Result<()> { debug!("invoking transactional insert request"); self.check_allow_operation().await?; - let key = key.into(); + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn); if self.buffer.get(&key).is_some() { return Err(Error::DuplicateKeyInsertion); } @@ -514,7 +529,7 @@ impl> Transaction { pub async fn delete(&mut self, key: impl Into) -> Result<()> { debug!("invoking transactional delete request"); self.check_allow_operation().await?; - let key = key.into(); + let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn); if self.is_pessimistic() { self.pessimistic_lock(iter::once(key.clone()), false) .await?; @@ -530,23 +545,14 @@ impl> Transaction { /// # Examples /// /// ```rust,no_run - /// # use tikv_client::{Key, Config, TransactionClient, proto::kvrpcpb}; + /// # use tikv_client::{Key, Config, TransactionClient, transaction::Mutation}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { /// # let client = TransactionClient::new(vec!["192.168.0.100", "192.168.0.101"]).await.unwrap(); /// let mut txn = client.begin_optimistic().await.unwrap(); /// let mutations = vec![ - /// kvrpcpb::Mutation { - /// op: kvrpcpb::Op::Del.into(), - /// key: b"k0".to_vec(), - /// ..Default::default() - /// }, - /// kvrpcpb::Mutation { - /// op: kvrpcpb::Op::Put.into(), - /// key: b"k1".to_vec(), - /// value: b"v1".to_vec(), - /// ..Default::default() - /// }, + /// Mutation::Delete("k0".to_owned().into()), + /// Mutation::Put("k1".to_owned().into(), b"v1".to_vec()), /// ]; /// txn.batch_mutate(mutations).await.unwrap(); /// txn.commit().await.unwrap(); @@ -554,13 +560,16 @@ impl> Transaction { /// ``` pub async fn batch_mutate( &mut self, - mutations: impl IntoIterator, + mutations: impl IntoIterator, ) -> Result<()> { debug!("invoking transactional batch mutate request"); self.check_allow_operation().await?; + let mutations: Vec = mutations + .into_iter() + .map(|mutation| mutation.encode_keyspace(self.keyspace, KeyMode::Txn)) + .collect(); if self.is_pessimistic() { - let mutations: Vec = mutations.into_iter().collect(); - self.pessimistic_lock(mutations.iter().map(|m| Key::from(m.key.clone())), false) + self.pessimistic_lock(mutations.iter().map(|m| m.key().clone()), false) .await?; for m in mutations { self.buffer.mutate(m); @@ -602,15 +611,18 @@ impl> Transaction { ) -> Result<()> { debug!("invoking transactional lock_keys request"); self.check_allow_operation().await?; + let keyspace = self.keyspace; + let keys = keys + .into_iter() + .map(move |k| k.into().encode_keyspace(keyspace, KeyMode::Txn)); match self.options.kind { TransactionKind::Optimistic => { for key in keys { - self.buffer.lock(key.into()); + self.buffer.lock(key); } } TransactionKind::Pessimistic(_) => { - self.pessimistic_lock(keys.into_iter().map(|k| k.into()), false) - .await?; + self.pessimistic_lock(keys, false).await?; } } Ok(()) @@ -660,6 +672,7 @@ impl> Transaction { self.timestamp.clone(), self.rpc.clone(), self.options.clone(), + self.keyspace, self.buffer.get_write_size() as u64, self.start_instant, ) @@ -712,6 +725,7 @@ impl> Transaction { self.timestamp.clone(), self.rpc.clone(), self.options.clone(), + self.keyspace, self.buffer.get_write_size() as u64, self.start_instant, ) @@ -745,10 +759,13 @@ impl> Transaction { primary_key, self.start_instant.elapsed().as_millis() as u64 + MAX_TTL, ); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = PlanBuilder::new(self.rpc.clone(), encoded_req) - .resolve_lock(self.options.retry_options.lock_backoff.clone()) + let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, request) + .resolve_lock( + self.options.retry_options.lock_backoff.clone(), + self.keyspace, + ) .retry_multi_region(self.options.retry_options.region_backoff.clone()) + .extract_error() .merge(CollectSingle) .post_process_default() .plan(); @@ -766,19 +783,20 @@ impl> Transaction { let timestamp = self.timestamp.clone(); let rpc = self.rpc.clone(); let retry_options = self.options.retry_options.clone(); + let keyspace = self.keyspace; + let range = range.into().encode_keyspace(self.keyspace, KeyMode::Txn); self.buffer .scan_and_fetch( - range.into(), + range, limit, !key_only, reverse, move |new_range, new_limit| async move { let request = new_scan_request(new_range, timestamp, new_limit, key_only, reverse); - let encoded_req = EncodedRequest::new(request, rpc.get_codec()); - let plan = PlanBuilder::new(rpc, encoded_req) - .resolve_lock(retry_options.lock_backoff) + let plan = PlanBuilder::new(rpc, keyspace, request) + .resolve_lock(retry_options.lock_backoff, keyspace) .retry_multi_region(retry_options.region_backoff) .merge(Collect) .plan(); @@ -788,6 +806,7 @@ impl> Transaction { }, ) .await + .map(move |pairs| pairs.map(move |pair| pair.truncate_keyspace(keyspace))) } /// Pessimistically lock the keys, and optionally retrieve corresponding values. @@ -832,9 +851,11 @@ impl> Transaction { for_update_ts.clone(), need_value, ); - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = PlanBuilder::new(self.rpc.clone(), encoded_req) - .resolve_lock(self.options.retry_options.lock_backoff.clone()) + let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, request) + .resolve_lock( + self.options.retry_options.lock_backoff.clone(), + self.keyspace, + ) .preserve_shard() .retry_multi_region_preserve_results(self.options.retry_options.region_backoff.clone()) .merge(CollectWithShard) @@ -887,9 +908,11 @@ impl> Transaction { start_version, for_update_ts, ); - let encoded_req = EncodedRequest::new(req, self.rpc.get_codec()); - let plan = PlanBuilder::new(self.rpc.clone(), encoded_req) - .resolve_lock(self.options.retry_options.lock_backoff.clone()) + let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, req) + .resolve_lock( + self.options.retry_options.lock_backoff.clone(), + self.keyspace, + ) .retry_multi_region(self.options.retry_options.region_backoff.clone()) .extract_error() .plan(); @@ -937,6 +960,7 @@ impl> Transaction { HeartbeatOption::FixedTime(heartbeat_interval) => heartbeat_interval, }; let start_instant = self.start_instant; + let keyspace = self.keyspace; let heartbeat_task = async move { loop { @@ -957,8 +981,7 @@ impl> Transaction { primary_key.clone(), start_instant.elapsed().as_millis() as u64 + MAX_TTL, ); - let encoded_req = EncodedRequest::new(request, rpc.get_codec()); - let plan = PlanBuilder::new(rpc.clone(), encoded_req) + let plan = PlanBuilder::new(rpc.clone(), keyspace, request) .retry_multi_region(region_backoff.clone()) .merge(CollectSingle) .plan(); @@ -1005,7 +1028,7 @@ impl> Transaction { } } -impl> Drop for Transaction { +impl Drop for Transaction { fn drop(&mut self) { debug!("dropping transaction"); if std::thread::panicking() { @@ -1256,6 +1279,21 @@ impl HeartbeatOption { } } +#[derive(Clone, Eq, PartialEq, Debug)] +pub enum Mutation { + Put(Key, Value), + Delete(Key), +} + +impl Mutation { + pub fn key(&self) -> &Key { + match self { + Mutation::Put(key, _) => key, + Mutation::Delete(key) => key, + } + } +} + /// A struct wrapping the details of two-phase commit protocol (2PC). /// /// The two phases are `prewrite` and `commit`. @@ -1271,6 +1309,7 @@ struct Committer { start_version: Timestamp, rpc: Arc, options: TransactionOptions, + keyspace: Keyspace, #[new(default)] undetermined: bool, write_size: u64, @@ -1348,9 +1387,11 @@ impl Committer { .collect(); // FIXME set max_commit_ts and min_commit_ts - let encoded_req = EncodedRequest::new(request, self.rpc.get_codec()); - let plan = PlanBuilder::new(self.rpc.clone(), encoded_req) - .resolve_lock(self.options.retry_options.lock_backoff.clone()) + let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, request) + .resolve_lock( + self.options.retry_options.lock_backoff.clone(), + self.keyspace, + ) .retry_multi_region(self.options.retry_options.region_backoff.clone()) .merge(CollectError) .extract_error() @@ -1389,9 +1430,11 @@ impl Committer { self.start_version.clone(), commit_version.clone(), ); - let encoded_req = EncodedRequest::new(req, self.rpc.get_codec()); - let plan = PlanBuilder::new(self.rpc.clone(), encoded_req) - .resolve_lock(self.options.retry_options.lock_backoff.clone()) + let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, req) + .resolve_lock( + self.options.retry_options.lock_backoff.clone(), + self.keyspace, + ) .retry_multi_region(self.options.retry_options.region_backoff.clone()) .extract_error() .plan(); @@ -1454,9 +1497,8 @@ impl Committer { .filter(|key| &primary_key != key); new_commit_request(keys, self.start_version, commit_version) }; - let encoded_req = EncodedRequest::new(req, self.rpc.get_codec()); - let plan = PlanBuilder::new(self.rpc, encoded_req) - .resolve_lock(self.options.retry_options.lock_backoff) + let plan = PlanBuilder::new(self.rpc, self.keyspace, req) + .resolve_lock(self.options.retry_options.lock_backoff, self.keyspace) .retry_multi_region(self.options.retry_options.region_backoff) .extract_error() .plan(); @@ -1476,9 +1518,8 @@ impl Committer { match self.options.kind { TransactionKind::Optimistic => { let req = new_batch_rollback_request(keys, self.start_version); - let encoded_req = EncodedRequest::new(req, self.rpc.get_codec()); - let plan = PlanBuilder::new(self.rpc, encoded_req) - .resolve_lock(self.options.retry_options.lock_backoff) + let plan = PlanBuilder::new(self.rpc, self.keyspace, req) + .resolve_lock(self.options.retry_options.lock_backoff, self.keyspace) .retry_multi_region(self.options.retry_options.region_backoff) .extract_error() .plan(); @@ -1486,9 +1527,8 @@ impl Committer { } TransactionKind::Pessimistic(for_update_ts) => { let req = new_pessimistic_rollback_request(keys, self.start_version, for_update_ts); - let encoded_req = EncodedRequest::new(req, self.rpc.get_codec()); - let plan = PlanBuilder::new(self.rpc, encoded_req) - .resolve_lock(self.options.retry_options.lock_backoff) + let plan = PlanBuilder::new(self.rpc, self.keyspace, req) + .resolve_lock(self.options.retry_options.lock_backoff, self.keyspace) .retry_multi_region(self.options.retry_options.region_backoff) .extract_error() .plan(); @@ -1565,12 +1605,16 @@ mod tests { use crate::mock::MockPdClient; use crate::proto::kvrpcpb; use crate::proto::pdpb::Timestamp; + use crate::request::Keyspace; use crate::transaction::HeartbeatOption; use crate::Transaction; use crate::TransactionOptions; + #[rstest::rstest] + #[case(Keyspace::Disable)] + #[case(Keyspace::Enable { keyspace_id: 0 })] #[tokio::test] - async fn test_optimistic_heartbeat() -> Result<(), io::Error> { + async fn test_optimistic_heartbeat(#[case] keyspace: Keyspace) -> Result<(), io::Error> { let scenario = FailScenario::setup(); fail::cfg("after-prewrite", "sleep(1500)").unwrap(); let heartbeats = Arc::new(AtomicUsize::new(0)); @@ -1593,6 +1637,7 @@ mod tests { pd_client, TransactionOptions::new_optimistic() .heartbeat_option(HeartbeatOption::FixedTime(Duration::from_secs(1))), + keyspace, ); heartbeat_txn.put(key1.clone(), "foo").await.unwrap(); let heartbeat_txn_handle = tokio::task::spawn_blocking(move || { @@ -1605,8 +1650,11 @@ mod tests { Ok(()) } + #[rstest::rstest] + #[case(Keyspace::Disable)] + #[case(Keyspace::Enable { keyspace_id: 0 })] #[tokio::test] - async fn test_pessimistic_heartbeat() -> Result<(), io::Error> { + async fn test_pessimistic_heartbeat(#[case] keyspace: Keyspace) -> Result<(), io::Error> { let heartbeats = Arc::new(AtomicUsize::new(0)); let heartbeats_cloned = heartbeats.clone(); let pd_client = Arc::new(MockPdClient::new(MockKvClient::with_dispatch_hook( @@ -1632,6 +1680,7 @@ mod tests { pd_client, TransactionOptions::new_pessimistic() .heartbeat_option(HeartbeatOption::FixedTime(Duration::from_secs(1))), + keyspace, ); heartbeat_txn.put(key1.clone(), "foo").await.unwrap(); assert_eq!(heartbeats.load(Ordering::SeqCst), 0); diff --git a/tests/common/mod.rs b/tests/common/mod.rs index 4d63dd5..9a32619 100644 --- a/tests/common/mod.rs +++ b/tests/common/mod.rs @@ -2,20 +2,20 @@ mod ctl; +use log::info; +use log::warn; +use rand::Rng; use std::collections::HashSet; use std::convert::TryInto; use std::env; use std::time::Duration; - -use log::info; -use log::warn; -use rand::Rng; +use tikv_client::Config; use tikv_client::Key; use tikv_client::RawClient; use tikv_client::Result; use tikv_client::Transaction; use tikv_client::TransactionClient; -use tikv_client::{ColumnFamily, Snapshot, TransactionOptions}; +use tikv_client::{Snapshot, TransactionOptions}; use tokio::time::sleep; const ENV_PD_ADDRS: &str = "PD_ADDRS"; @@ -24,21 +24,23 @@ const REGION_SPLIT_TIME_LIMIT: Duration = Duration::from_secs(15); // Delete all entries in TiKV to leave a clean space for following tests. pub async fn clear_tikv() { - let cfs = vec![ - ColumnFamily::Default, - ColumnFamily::Lock, - ColumnFamily::Write, - ]; // DEFAULT_REGION_BACKOFF is not long enough for CI environment. So set a longer backoff. let backoff = tikv_client::Backoff::no_jitter_backoff(100, 30000, 20); - for cf in cfs { - let raw_client = RawClient::new(pd_addrs()).await.unwrap().with_cf(cf); - raw_client - .with_backoff(backoff.clone()) - .delete_range(vec![]..) + let raw_client = + RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) .await .unwrap(); - } + raw_client + .with_backoff(backoff) + .delete_range(..) + .await + .unwrap(); + + let txn_client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await + .unwrap(); + txn_client.unsafe_destroy_range(..).await.unwrap(); } // To test with multiple regions, prewrite some data. Tests that hope to test @@ -78,14 +80,16 @@ async fn ensure_region_split( // 1. write plenty transactional keys // 2. wait until regions split - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut txn = client.begin_optimistic().await?; for key in keys.into_iter() { txn.put(key.into(), vec![0, 0, 0, 0]).await?; } txn.commit().await?; let mut txn = client.begin_optimistic().await?; - let _ = txn.scan(vec![].., 2048).await?; + let _ = txn.scan(.., 2048).await?; txn.commit().await?; info!("splitting regions..."); diff --git a/tests/failpoint_tests.rs b/tests/failpoint_tests.rs index f34dff4..d58b401 100644 --- a/tests/failpoint_tests.rs +++ b/tests/failpoint_tests.rs @@ -17,6 +17,7 @@ use tikv_client::transaction::HeartbeatOption; use tikv_client::transaction::ResolveLocksOptions; use tikv_client::Backoff; use tikv_client::CheckLevel; +use tikv_client::Config; use tikv_client::Result; use tikv_client::RetryOptions; use tikv_client::TransactionClient; @@ -34,7 +35,9 @@ async fn txn_optimistic_heartbeat() -> Result<()> { let key1 = "key1".to_owned(); let key2 = "key2".to_owned(); - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; // CheckLevel::Panic makes the case unstable, change to Warn level for now. // See https://github.com/tikv/client-rust/issues/389 @@ -110,7 +113,9 @@ async fn txn_cleanup_locks_batch_size() -> Result<()> { fail::cfg("before-cleanup-locks", "off").unwrap(); }} - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let keys = write_data(&client, true, true).await?; assert_eq!(count_locks(&client).await?, keys.len()); @@ -145,7 +150,11 @@ async fn txn_cleanup_async_commit_locks() -> Result<()> { fail::cfg("after-prewrite", "off").unwrap() } - let client = TransactionClient::new(pd_addrs()).await?; + let client = TransactionClient::new_with_config( + pd_addrs(), + Config::default().with_default_keyspace(), + ) + .await?; let keys = write_data(&client, true, true).await?; assert_eq!(count_locks(&client).await?, keys.len()); @@ -171,7 +180,11 @@ async fn txn_cleanup_async_commit_locks() -> Result<()> { fail::cfg("before-commit-secondary", "off").unwrap() } - let client = TransactionClient::new(pd_addrs()).await?; + let client = TransactionClient::new_with_config( + pd_addrs(), + Config::default().with_default_keyspace(), + ) + .await?; let keys = write_data(&client, true, false).await?; thread::sleep(Duration::from_secs(1)); // Wait for async commit to complete. assert_eq!(count_locks(&client).await?, keys.len() * percent / 100); @@ -192,7 +205,11 @@ async fn txn_cleanup_async_commit_locks() -> Result<()> { // all committed { info!("test all committed"); - let client = TransactionClient::new(pd_addrs()).await?; + let client = TransactionClient::new_with_config( + pd_addrs(), + Config::default().with_default_keyspace(), + ) + .await?; let keys = write_data(&client, true, false).await?; let safepoint = client.current_timestamp().await?; @@ -227,7 +244,9 @@ async fn txn_cleanup_range_async_commit_locks() -> Result<()> { fail::cfg("after-prewrite", "off").unwrap() } - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let keys = write_data(&client, true, true).await?; assert_eq!(count_locks(&client).await?, keys.len()); @@ -276,7 +295,11 @@ async fn txn_cleanup_2pc_locks() -> Result<()> { fail::cfg("after-prewrite", "off").unwrap() } - let client = TransactionClient::new(pd_addrs()).await?; + let client = TransactionClient::new_with_config( + pd_addrs(), + Config::default().with_default_keyspace(), + ) + .await?; let keys = write_data(&client, false, true).await?; assert_eq!(count_locks(&client).await?, keys.len()); @@ -306,7 +329,11 @@ async fn txn_cleanup_2pc_locks() -> Result<()> { // all committed { info!("test all committed"); - let client = TransactionClient::new(pd_addrs()).await?; + let client = TransactionClient::new_with_config( + pd_addrs(), + Config::default().with_default_keyspace(), + ) + .await?; let keys = write_data(&client, false, false).await?; assert_eq!(count_locks(&client).await?, 0); @@ -347,7 +374,7 @@ async fn must_rollbacked(client: &TransactionClient, keys: HashSet>) { async fn count_locks(client: &TransactionClient) -> Result { let ts = client.current_timestamp().await.unwrap(); - let locks = client.scan_locks(&ts, vec![].., 1024).await?; + let locks = client.scan_locks(&ts, .., 1024).await?; // De-duplicated as `scan_locks` will return duplicated locks due to retry on region changes. let locks_set: HashSet> = HashSet::from_iter(locks.into_iter().map(|l| l.key)); Ok(locks_set.len()) diff --git a/tests/integration_tests.rs b/tests/integration_tests.rs index 82442c4..514c4aa 100644 --- a/tests/integration_tests.rs +++ b/tests/integration_tests.rs @@ -12,18 +12,18 @@ //! requirements on the region boundaries. mod common; -use std::collections::HashMap; -use std::iter; - use common::*; use futures::prelude::*; use rand::seq::IteratorRandom; use rand::thread_rng; use rand::Rng; use serial_test::serial; +use std::collections::HashMap; +use std::iter; use tikv_client::backoff::DEFAULT_REGION_BACKOFF; -use tikv_client::proto::kvrpcpb; use tikv_client::transaction::HeartbeatOption; +use tikv_client::transaction::Mutation; +use tikv_client::Config; use tikv_client::Error; use tikv_client::Key; use tikv_client::KvPair; @@ -42,7 +42,9 @@ const NUM_TRNASFER: u32 = 100; #[serial] async fn txn_get_timestamp() -> Result<()> { const COUNT: usize = 1 << 8; // use a small number to make test fast - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut versions = future::join_all((0..COUNT).map(|_| client.current_timestamp())) .await @@ -63,7 +65,9 @@ async fn txn_get_timestamp() -> Result<()> { async fn txn_crud() -> Result<()> { init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut txn = client.begin_optimistic().await?; // Get non-existent keys @@ -147,7 +151,9 @@ async fn txn_crud() -> Result<()> { async fn txn_insert_duplicate_keys() -> Result<()> { init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; // Initialize TiKV store with {foo => bar} let mut txn = client.begin_optimistic().await?; txn.put("foo".to_owned(), "bar".to_owned()).await?; @@ -171,7 +177,9 @@ async fn txn_insert_duplicate_keys() -> Result<()> { async fn txn_pessimistic() -> Result<()> { init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut txn = client.begin_pessimistic().await?; txn.put("foo".to_owned(), "foo".to_owned()).await.unwrap(); @@ -188,7 +196,9 @@ async fn txn_pessimistic() -> Result<()> { async fn txn_split_batch() -> Result<()> { init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut txn = client.begin_optimistic().await?; let mut rng = thread_rng(); @@ -226,7 +236,8 @@ async fn txn_split_batch() -> Result<()> { #[serial] async fn raw_bank_transfer() -> Result<()> { init().await?; - let client = RawClient::new(pd_addrs()).await?; + let client = + RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?; let mut rng = thread_rng(); let people = gen_u32_keys(NUM_PEOPLE, &mut rng); @@ -278,7 +289,9 @@ async fn txn_read() -> Result<()> { let value = "large_value".repeat(10); init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; for i in 0..2u32.pow(NUM_BITS_TXN) { let mut cur = i * 2u32.pow(32 - NUM_BITS_TXN); @@ -370,7 +383,9 @@ async fn txn_read() -> Result<()> { #[serial] async fn txn_bank_transfer() -> Result<()> { init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut rng = thread_rng(); let options = TransactionOptions::new_optimistic() .use_async_commit() @@ -423,7 +438,8 @@ async fn txn_bank_transfer() -> Result<()> { #[serial] async fn raw_req() -> Result<()> { init().await?; - let client = RawClient::new(pd_addrs()).await?; + let client = + RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?; // empty; get non-existent key let res = client.get("k1".to_owned()).await; @@ -545,6 +561,62 @@ async fn raw_req() -> Result<()> { assert_eq!(res[5].1, "v4".as_bytes()); assert_eq!(res[6].1, "v5".as_bytes()); + // reverse scan + + // By default end key is exclusive, so k5 is not included and start key in included + let res = client + .scan_reverse("k2".to_owned().."k5".to_owned(), 5) + .await?; + assert_eq!(res.len(), 3); + assert_eq!(res[0].1, "v4".as_bytes()); + assert_eq!(res[1].1, "v3".as_bytes()); + assert_eq!(res[2].1, "v2".as_bytes()); + + // by default end key in exclusive and start key is inclusive but now exclude start key + let res = client + .scan_reverse("k2\0".to_owned().."k5".to_owned(), 5) + .await?; + assert_eq!(res.len(), 2); + assert_eq!(res[0].1, "v4".as_bytes()); + assert_eq!(res[1].1, "v3".as_bytes()); + + // reverse scan + // by default end key is exclusive and start key is inclusive but now include end key + let res = client + .scan_reverse("k2".to_owned()..="k5".to_owned(), 5) + .await?; + assert_eq!(res.len(), 4); + assert_eq!(res[0].1, "v5".as_bytes()); + assert_eq!(res[1].1, "v4".as_bytes()); + assert_eq!(res[2].1, "v3".as_bytes()); + assert_eq!(res[3].1, "v2".as_bytes()); + + // by default end key is exclusive and start key is inclusive but now include end key and exclude start key + let res = client + .scan_reverse("k2\0".to_owned()..="k5".to_owned(), 5) + .await?; + assert_eq!(res.len(), 3); + assert_eq!(res[0].1, "v5".as_bytes()); + assert_eq!(res[1].1, "v4".as_bytes()); + assert_eq!(res[2].1, "v3".as_bytes()); + + // limit results to first 2 + let res = client + .scan_reverse("k2".to_owned().."k5".to_owned(), 2) + .await?; + assert_eq!(res.len(), 2); + assert_eq!(res[0].1, "v4".as_bytes()); + assert_eq!(res[1].1, "v3".as_bytes()); + + // if endKey is not provided then it scan everything including end key + let range = BoundRange::range_from(Key::from("k2".to_owned())); + let res = client.scan_reverse(range, 20).await?; + assert_eq!(res.len(), 4); + assert_eq!(res[0].1, "v5".as_bytes()); + assert_eq!(res[1].1, "v4".as_bytes()); + assert_eq!(res[2].1, "v3".as_bytes()); + assert_eq!(res[3].1, "v2".as_bytes()); + Ok(()) } @@ -553,7 +625,9 @@ async fn raw_req() -> Result<()> { #[serial] async fn txn_update_safepoint() -> Result<()> { init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let res = client.gc(client.current_timestamp().await?).await?; assert!(res); Ok(()) @@ -568,7 +642,8 @@ async fn raw_write_million() -> Result<()> { let interval = 2u32.pow(32 - NUM_BITS_TXN - NUM_BITS_KEY_PER_TXN); init().await?; - let client = RawClient::new(pd_addrs()).await?; + let client = + RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?; for i in 0..2u32.pow(NUM_BITS_TXN) { let mut cur = i * 2u32.pow(32 - NUM_BITS_TXN); @@ -685,6 +760,105 @@ async fn raw_write_million() -> Result<()> { r = client.scan(.., limit).await?; assert_eq!(r.len(), limit as usize); + // test scan_reverse + // test scan, key range from [0,0,0,0] to [255.0.0.0] + let mut limit = 2000; + let mut r = client.scan_reverse(.., limit).await?; + assert_eq!(r.len(), 256); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8); + } + r = client.scan_reverse(vec![100, 0, 0, 0].., limit).await?; + assert_eq!(r.len(), 156); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + 100); + } + r = client + .scan_reverse(vec![5, 0, 0, 0]..vec![200, 0, 0, 0], limit) + .await?; + assert_eq!(r.len(), 195); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + 5); + } + r = client + .scan_reverse(vec![5, 0, 0, 0]..=vec![200, 0, 0, 0], limit) + .await?; + assert_eq!(r.len(), 196); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + 5); + } + r = client + .scan_reverse(vec![5, 0, 0, 0]..=vec![255, 10, 0, 0], limit) + .await?; + assert_eq!(r.len(), 251); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + 5); + } + r = client + .scan_reverse(vec![255, 1, 0, 0]..=vec![255, 10, 0, 0], limit) + .await?; + assert_eq!(r.len(), 0); + r = client.scan_reverse(..vec![0, 0, 0, 0], limit).await?; + assert_eq!(r.len(), 0); + + limit = 3; + let mut r = client.scan_reverse(.., limit).await?; + let mut expected_start: u8 = 255 - limit as u8 + 1; // including endKey + assert_eq!(r.len(), limit as usize); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + expected_start); + } + r = client.scan_reverse(vec![100, 0, 0, 0].., limit).await?; + expected_start = 255 - limit as u8 + 1; // including endKey + assert_eq!(r.len(), limit as usize); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + expected_start); + } + r = client + .scan_reverse(vec![5, 0, 0, 0]..vec![200, 0, 0, 0], limit) + .await?; + expected_start = 200 - limit as u8; + assert_eq!(r.len(), limit as usize); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + expected_start); + } + r = client + .scan_reverse(vec![5, 0, 0, 0]..=vec![200, 0, 0, 0], limit) + .await?; + expected_start = 200 - limit as u8 + 1; // including endKey + assert_eq!(r.len(), limit as usize); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + expected_start); + } + r = client + .scan_reverse(vec![5, 0, 0, 0]..=vec![255, 10, 0, 0], limit) + .await?; + expected_start = 255 - limit as u8 + 1; // including endKey + assert_eq!(r.len(), limit as usize); + for (i, val) in r.iter().rev().enumerate() { + let k: Vec = val.0.clone().into(); + assert_eq!(k[0], i as u8 + expected_start); + } + r = client + .scan_reverse(vec![255, 1, 0, 0]..=vec![255, 10, 0, 0], limit) + .await?; + assert_eq!(r.len(), 0); + r = client.scan_reverse(..vec![0, 0, 0, 0], limit).await?; + assert_eq!(r.len(), 0); + + limit = 0; + r = client.scan_reverse(.., limit).await?; + assert_eq!(r.len(), limit as usize); + // test batch_scan for batch_num in 1..4 { let _ = client @@ -698,11 +872,42 @@ async fn raw_write_million() -> Result<()> { Ok(()) } +/// Tests raw ttl API. +#[tokio::test] +#[serial] +async fn raw_ttl() -> Result<()> { + init().await?; + let client = + RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?; + let key1 = vec![1]; + let key2 = vec![2]; + let val = vec![42]; + + assert_eq!(client.get_key_ttl_secs(key1.clone()).await?, None); + client.put_with_ttl(key1.clone(), val.clone(), 10).await?; + assert_eq!(client.get(key1.clone()).await?, Some(val.clone())); + assert_eq!(client.get_key_ttl_secs(key1.clone()).await?, Some(10)); + client + .batch_put_with_ttl( + vec![(key1.clone(), val.clone()), (key2.clone(), val.clone())], + vec![20, 20], + ) + .await?; + assert_eq!(client.get(key1.clone()).await?, Some(val.clone())); + assert_eq!(client.get(key2.clone()).await?, Some(val.clone())); + assert_eq!(client.get_key_ttl_secs(key1.clone()).await?, Some(20)); + assert_eq!(client.get_key_ttl_secs(key2.clone()).await?, Some(20)); + + Ok(()) +} + #[tokio::test] #[serial] async fn txn_pessimistic_rollback() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut preload_txn = client.begin_optimistic().await?; let key1 = vec![1]; let key2 = vec![2]; @@ -734,7 +939,9 @@ async fn txn_pessimistic_rollback() -> Result<()> { #[serial] async fn txn_pessimistic_delete() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; // The transaction will lock the keys and must release the locks on commit, // even when values are not written to the DB. @@ -785,7 +992,9 @@ async fn txn_pessimistic_delete() -> Result<()> { #[serial] async fn txn_lock_keys() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let k1 = b"key1".to_vec(); let k2 = b"key2".to_vec(); @@ -819,7 +1028,9 @@ async fn txn_lock_keys() -> Result<()> { #[serial] async fn txn_lock_keys_error_handle() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; // Keys in `k` should locate in different regions. See `init()` for boundary of regions. let k: Vec = vec![ @@ -856,7 +1067,9 @@ async fn txn_lock_keys_error_handle() -> Result<()> { #[serial] async fn txn_get_for_update() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let key1 = "key".to_owned(); let key2 = "another key".to_owned(); let value1 = b"some value".to_owned(); @@ -903,7 +1116,9 @@ async fn txn_pessimistic_heartbeat() -> Result<()> { let key1 = "key1".to_owned(); let key2 = "key2".to_owned(); - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut heartbeat_txn = client .begin_with_options(TransactionOptions::new_pessimistic()) @@ -943,7 +1158,9 @@ async fn txn_pessimistic_heartbeat() -> Result<()> { #[serial] async fn raw_cas() -> Result<()> { init().await?; - let client = RawClient::new(pd_addrs()).await?.with_atomic_for_cas(); + let client = RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await? + .with_atomic_for_cas(); let key = "key".to_owned(); let value = "value".to_owned(); let new_value = "new value".to_owned(); @@ -986,7 +1203,8 @@ async fn raw_cas() -> Result<()> { client.batch_delete(vec![key.clone()]).await.err().unwrap(), Error::UnsupportedMode )); - let client = RawClient::new(pd_addrs()).await?; + let client = + RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?; assert!(matches!( client .compare_and_swap(key.clone(), None, vec![]) @@ -1003,7 +1221,9 @@ async fn raw_cas() -> Result<()> { #[serial] async fn txn_scan() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let k1 = b"a".to_vec(); let v = b"b".to_vec(); @@ -1026,7 +1246,9 @@ async fn txn_scan() -> Result<()> { #[serial] async fn txn_scan_reverse() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let k1 = b"k1".to_vec(); let k2 = b"k2".to_vec(); @@ -1099,7 +1321,9 @@ async fn txn_scan_reverse() -> Result<()> { #[serial] async fn txn_scan_reverse_multi_regions() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; // Keys in `keys` should locate in different regions. See `init()` for boundary of regions. let keys: Vec = vec![ @@ -1143,7 +1367,9 @@ async fn txn_scan_reverse_multi_regions() -> Result<()> { #[serial] async fn txn_key_exists() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let key = "key".to_owned(); let value = "value".to_owned(); let mut t1 = client.begin_optimistic().await?; @@ -1166,7 +1392,9 @@ async fn txn_key_exists() -> Result<()> { #[serial] async fn txn_batch_mutate_optimistic() -> Result<()> { init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; // Put k0 { @@ -1177,7 +1405,7 @@ async fn txn_batch_mutate_optimistic() -> Result<()> { // Delete k0 and put k1, k2 do_mutate(false).await.unwrap(); // Read and verify - verify_mutate(false).await; + verify_mutate(false).await?; Ok(()) } @@ -1185,7 +1413,9 @@ async fn txn_batch_mutate_optimistic() -> Result<()> { #[serial] async fn txn_batch_mutate_pessimistic() -> Result<()> { init().await?; - let client = TransactionClient::new(pd_addrs()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; // Put k0 { @@ -1210,7 +1440,7 @@ async fn txn_batch_mutate_pessimistic() -> Result<()> { txn3_handle.await?.unwrap(); // Read and verify - verify_mutate(true).await; + verify_mutate(true).await?; Ok(()) } @@ -1227,27 +1457,15 @@ async fn begin_mutate(client: &TransactionClient, is_pessimistic: bool) -> Resul } async fn do_mutate(is_pessimistic: bool) -> Result<()> { - let client = TransactionClient::new(pd_addrs()).await.unwrap(); + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut txn = begin_mutate(&client, is_pessimistic).await.unwrap(); let mutations = vec![ - kvrpcpb::Mutation { - op: kvrpcpb::Op::Del.into(), - key: b"k0".to_vec(), - ..Default::default() - }, - kvrpcpb::Mutation { - op: kvrpcpb::Op::Put.into(), - key: b"k1".to_vec(), - value: b"v1".to_vec(), - ..Default::default() - }, - kvrpcpb::Mutation { - op: kvrpcpb::Op::Put.into(), - key: b"k2".to_vec(), - value: b"v2".to_vec(), - ..Default::default() - }, + Mutation::Delete(Key::from("k0".to_owned())), + Mutation::Put(Key::from("k1".to_owned()), Value::from("v1".to_owned())), + Mutation::Put(Key::from("k2".to_owned()), Value::from("v2".to_owned())), ]; match txn.batch_mutate(mutations).await { @@ -1262,8 +1480,10 @@ async fn do_mutate(is_pessimistic: bool) -> Result<()> { } } -async fn verify_mutate(is_pessimistic: bool) { - let client = TransactionClient::new(pd_addrs()).await.unwrap(); +async fn verify_mutate(is_pessimistic: bool) -> Result<()> { + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; let mut snapshot = snapshot(&client, is_pessimistic).await.unwrap(); let res: HashMap = snapshot .batch_get(vec!["k0".to_owned(), "k1".to_owned(), "k2".to_owned()]) @@ -1280,13 +1500,16 @@ async fn verify_mutate(is_pessimistic: bool) { res.get(&Key::from("k2".to_owned())), Some(Value::from("v2".to_owned())).as_ref() ); + Ok(()) } #[tokio::test] #[serial] async fn txn_unsafe_destroy_range() -> Result<()> { init().await?; - let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?; + let client = + TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()) + .await?; const DATA_COUNT: usize = 10;