This commit is contained in:
jmhrpr 2024-11-19 00:40:59 +00:00
commit 11c74b39d8
107 changed files with 12413 additions and 922 deletions

View File

@ -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"

View File

@ -15,3 +15,7 @@ max-open-files = 10000
[raftdb]
max-open-files = 10000
[storage]
api-version = 2
enable-ttl = true

View File

@ -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)

View File

@ -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(())
}

View File

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

47
proto/autoid.proto Normal file
View File

@ -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) {}
}

View File

@ -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/<task-name>/<start-key> -> <end-key>'
// for storing all the target ranges of the task, the "ranges" field will not be saved in the taskinfo.
reserved "ranges";
// The status field allow us mark the task as 'paused'.
// We use '/tidb/br-stream/pause/<task-name> -> ()' for pausing a task for now.
// If we need more complexed status transforming, maybe add the status field back would be fine.
reserved "status";
}
message StorageBackend {
@ -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;
}

View File

@ -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);
}

View File

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

View File

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

169
proto/disaggregated.proto Normal file
View File

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

View File

@ -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.

View File

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

117
proto/gcpb.proto Normal file
View File

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

View File

@ -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 <name, SSTMeta>.
// We'll generate all SSTMeta into one SST File.
map<string, SSTMeta> 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;
}

View File

@ -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 {

81
proto/keyspacepb.proto Normal file
View File

@ -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<string, string> 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;
}

View File

@ -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.
//
// 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) |
// --------------------------------------------------------------------------------
// Bit 1 is for deletion. If set, the entry is logical deleted.
// ---------------------
// | Meta flags |
// ---------------------
// | 0x02 (0b00000010) |
// ---------------------
//
// 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];
}

58
proto/logbackuppb.proto Normal file
View File

@ -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) {}
}

138
proto/meta_storagepb.proto Normal file
View File

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

View File

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

View File

@ -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,6 +18,18 @@ 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 {
@ -24,6 +37,7 @@ message IsAliveRequest {
message IsAliveResponse {
bool available = 1;
int64 mpp_version = 2;
}
// Dipsatch the task request to different tiflash servers.
@ -34,6 +48,8 @@ message DispatchTaskRequest {
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,6 +68,18 @@ 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
@ -64,9 +92,12 @@ message MPPDataPacket {
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;
}

View File

@ -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<string, Member> 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;
}

View File

@ -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 {

View File

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

82
proto/recoverdatapb.proto Normal file
View File

@ -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) {}
}

View File

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

View File

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

View File

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

197
proto/schedulingpb.proto Normal file
View File

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

View File

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

View File

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

60
proto/tracepb.proto Normal file
View File

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

126
proto/tsopb.proto Normal file
View File

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

View File

@ -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<Error>,
success_keys: Vec<Vec<u8>>,
},
#[error("Keyspace not found: {0}")]
KeyspaceNotFound(String),
}
impl From<crate::proto::errorpb::Error> for Error {

View File

@ -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,27 +77,40 @@ 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 channel = if !self.ca.is_empty() {
self.tls_channel(addr).await?
} else {
self.default_channel(addr).await?
};
let ch = channel.connect().await?;
let mut builder = Channel::from_shared(addr)?
.tcp_keepalive(Some(Duration::from_secs(10)))
.keep_alive_timeout(Duration::from_secs(3));
Ok(factory(ch))
}
if !self.ca.is_empty() {
async fn tls_channel(&self, addr: &str) -> Result<Endpoint> {
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)?,
));
builder = builder.tls_config(tls)?;
};
let builder = builder.tls_config(tls)?;
Ok(builder)
}
let ch = builder.connect().await?;
async fn default_channel(&self, addr: &str) -> Result<Endpoint> {
let addr = "http://".to_string() + &SCHEME_REG.replace(addr, "");
self.endpoint(addr)
}
Ok(factory(ch))
fn endpoint(&self, addr: String) -> Result<Endpoint> {
let endpoint = Channel::from_shared(addr)?
.tcp_keepalive(Some(Duration::from_secs(10)))
.keep_alive_timeout(Duration::from_secs(3));
Ok(endpoint)
}
}

View File

@ -19,6 +19,7 @@ pub struct Config {
pub cert_path: Option<PathBuf>,
pub key_path: Option<PathBuf>,
pub timeout: Duration,
pub keyspace: Option<String>,
}
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
}
}

179
src/generated/autoid.rs Normal file
View File

@ -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<u8>,
}
#[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<u8>,
}
/// 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<T> {
inner: tonic::client::Grpc<T>,
}
impl AutoIdAllocClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> AutoIdAllocClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> AutoIdAllocClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::AutoIdRequest>,
) -> std::result::Result<tonic::Response<super::AutoIdResponse>, 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<super::RebaseRequest>,
) -> std::result::Result<tonic::Response<super::RebaseResponse>, 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
}
}
}

View File

@ -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<PlacementPolicy>,
/// 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<PitrDbMap>,
#[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<BackupRange>,
/// 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<u8>,
#[prost(bytes = "vec", tag = "2")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
#[prost(message, repeated, tag = "3")]
pub files: ::prost::alloc::vec::Vec<File>,
}
#[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<RawRange>,
/// A set of files that contains BackupRanges.
#[prost(message, repeated, tag = "6")]
pub backup_ranges: ::prost::alloc::vec::Vec<BackupRange>,
/// A set of files that contains DDLs.
#[prost(bytes = "vec", repeated, tag = "5")]
pub ddls: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec<u8>>,
}
#[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<u8>,
}
#[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<u8>,
@ -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<IdMap>,
#[prost(message, repeated, tag = "3")]
pub partitions: ::prost::alloc::vec::Vec<IdMap>,
}
#[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<IdMap>,
#[prost(message, repeated, tag = "3")]
pub tables: ::prost::alloc::vec::Vec<PitrTableMap>,
}
#[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<u8>,
@ -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<CipherInfo>,
/// 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<super::kvrpcpb::KeyRange>,
/// 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<super::kvrpcpb::Context>,
}
#[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<StorageBackend>,
/// 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
/// 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.
#[prost(string, tag = "6")]
pub shared_key: ::prost::alloc::string::String,
/// 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>
#[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 <https://learn.microsoft.com/en-us/azure/storage/blobs/encryption-scope-manage?tabs=powershell#upload-a-blob-with-an-encryption-scope>
#[prost(string, tag = "9")]
pub encryption_scope: ::prost::alloc::string::String,
/// 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>
#[prost(message, optional, tag = "10")]
pub encryption_key: ::core::option::Option<AzureCustomerKey>,
}
#[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<u8>,
#[prost(message, repeated, tag = "4")]
pub files: ::prost::alloc::vec::Vec<File>,
/// 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<Error>,
#[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<Error>,
/// 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<Error>,
#[prost(message, optional, tag = "2")]
pub region: ::core::option::Option<super::metapb::Region>,
#[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<DataFileInfo>,
#[prost(message, repeated, tag = "6")]
pub file_groups: ::prost::alloc::vec::Vec<DataFileGroup>,
#[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<DataFileInfo>,
/// / 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<u8>,
/// 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<u8>,
#[prost(bytes = "vec", tag = "9")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
/// 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<Self> {
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<Self> {
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<Self> {
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<super::CheckAdminRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::CheckAdminResponse>>,
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<super::PrepareRequest>,
) -> std::result::Result<
tonic::Response<super::PrepareResponse>,
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<super::CleanupRequest>,
) -> std::result::Result<
tonic::Response<super::CleanupResponse>,
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.

View File

@ -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<Compatibility>,
#[prost(message, optional, tag = "6")]
pub cluster_id_mismatch: ::core::option::Option<ClusterIdMismatch>,
#[prost(message, optional, tag = "7")]
pub server_is_busy: ::core::option::Option<super::errorpb::ServerIsBusy>,
}
#[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<u8>,
#[prost(bytes = "vec", tag = "7")]
pub old_value: ::prost::alloc::vec::Vec<u8>,
/// 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<u64>,
#[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<u8>,
/// 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<change_data_request::Request>,
}
/// 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<super::TxnStatus>,
}
/// 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.>
#[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<Self> {
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<Message = super::ChangeDataRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::ChangeDataEvent>>,
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
}
}
}

View File

@ -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 {

View File

@ -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<StoreBatchTask>,
/// 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<super::span::SpanSet>,
/// 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<StoreBatchTaskResponse>,
}
#[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<RegionInfo>,
}
#[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<super::kvrpcpb::Context>,
@ -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<TableRegions>,
#[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<super::metapb::Region>,
}
#[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<super::metapb::RegionEpoch>,
#[prost(message, optional, tag = "3")]
pub peer: ::core::option::Option<super::metapb::Peer>,
#[prost(message, repeated, tag = "4")]
pub ranges: ::prost::alloc::vec::Vec<KeyRange>,
#[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<u8>,
#[prost(message, optional, tag = "2")]
pub region_error: ::core::option::Option<super::errorpb::Error>,
#[prost(message, optional, tag = "3")]
pub locked: ::core::option::Option<super::kvrpcpb::LockInfo>,
#[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<super::kvrpcpb::ExecDetailsV2>,
}

View File

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

View File

@ -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<u64>,
}
#[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<u8>,
#[prost(bytes = "vec", tag = "2")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
}
#[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<super::kvrpcpb::Context>,
#[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<u8>,
#[prost(bytes = "vec", tag = "5")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
#[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<super::ResetToVersionRequest>,
) -> std::result::Result<
tonic::Response<super::ResetToVersionResponse>,
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<super::GetRangePropertiesRequest>,
) -> std::result::Result<
tonic::Response<super::GetRangePropertiesResponse>,
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<super::FlashbackToVersionRequest>,
) -> std::result::Result<
tonic::Response<super::FlashbackToVersionResponse>,
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<super::GetRegionReadProgressRequest>,
) -> std::result::Result<
tonic::Response<super::GetRegionReadProgressResponse>,
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
}
}
}

View File

@ -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 {

View File

@ -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<s3_lock_result::Error>,
}
/// 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<u8>,
/// 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<S3LockResult>,
}
#[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<u8>,
}
#[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<S3LockResult>,
}
#[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<DisaggS3Config>,
}
#[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<establish_disagg_task_error::Errors>,
}
/// 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<u64>,
}
#[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<super::kvrpcpb::LockInfo>,
}
#[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<DisaggTaskMeta>,
/// 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<super::coprocessor::RegionInfo>,
#[prost(int64, tag = "5")]
pub schema_ver: i64,
/// Used for PartitionTableScan
#[prost(message, repeated, tag = "6")]
pub table_regions: ::prost::alloc::vec::Vec<super::coprocessor::TableRegions>,
/// The encoded TableScan/PartitionTableScan + Selection.
#[prost(bytes = "vec", tag = "7")]
pub encoded_plan: ::prost::alloc::vec::Vec<u8>,
}
#[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<EstablishDisaggTaskError>,
/// 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<DisaggTaskMeta>,
/// Serialized disaggregated tasks (per physical table)
#[prost(bytes = "vec", repeated, tag = "5")]
pub tables: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec<u8>>,
}
#[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<DisaggTaskMeta>,
}
#[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<DisaggTaskMeta>,
#[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<u64>,
}
#[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<DisaggReadError>,
/// 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<u8>>,
/// * ColumnFileInMemory alone with its serialized block
#[prost(bytes = "vec", repeated, tag = "3")]
pub chunks: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec<u8>>,
/// 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<u8>>,
}

View File

@ -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 {

View File

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

View File

@ -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 {

View File

@ -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<u8>,
#[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)]

View File

@ -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<super::metapb::Peer>,
}
/// 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<u8>>,
}
#[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<RegionNotInitialized>,
#[prost(message, optional, tag = "15")]
pub disk_full: ::core::option::Option<DiskFull>,
/// Online recovery is still in performing, reject writes to avoid potential issues
#[prost(message, optional, tag = "16")]
pub recovery_in_progress: ::core::option::Option<RecoveryInProgress>,
/// 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<FlashbackInProgress>,
/// 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<FlashbackNotPrepared>,
/// 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<IsWitness>,
#[prost(message, optional, tag = "20")]
pub mismatch_peer_id: ::core::option::Option<MismatchPeerId>,
/// 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<BucketVersionNotMatch>,
}

352
src/generated/gcpb.rs Normal file
View File

@ -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<Error>,
}
#[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<u8>,
#[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<RequestHeader>,
/// 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<ResponseHeader>,
#[prost(message, repeated, tag = "2")]
pub key_spaces: ::prost::alloc::vec::Vec<KeySpace>,
}
#[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<RequestHeader>,
#[prost(bytes = "vec", tag = "2")]
pub space_id: ::prost::alloc::vec::Vec<u8>,
}
#[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<ResponseHeader>,
#[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<RequestHeader>,
#[prost(bytes = "vec", tag = "2")]
pub space_id: ::prost::alloc::vec::Vec<u8>,
#[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<ResponseHeader>,
/// 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<RequestHeader>,
#[prost(bytes = "vec", tag = "2")]
pub space_id: ::prost::alloc::vec::Vec<u8>,
#[prost(bytes = "vec", tag = "3")]
pub service_id: ::prost::alloc::vec::Vec<u8>,
/// 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<ResponseHeader>,
/// 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<Self> {
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<T> {
inner: tonic::client::Grpc<T>,
}
impl GcClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> GcClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> GcClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::ListKeySpacesRequest>,
) -> std::result::Result<
tonic::Response<super::ListKeySpacesResponse>,
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<super::GetMinServiceSafePointRequest>,
) -> std::result::Result<
tonic::Response<super::GetMinServiceSafePointResponse>,
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<super::UpdateGcSafePointRequest>,
) -> std::result::Result<
tonic::Response<super::UpdateGcSafePointResponse>,
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<super::UpdateServiceSafePointRequest>,
) -> std::result::Result<
tonic::Response<super::UpdateServiceSafePointResponse>,
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
}
}
}

View File

@ -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.

View File

@ -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 {

View File

@ -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<Range>,
}
#[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<u8>,
@ -111,6 +145,8 @@ pub struct CompactRequest {
pub range: ::core::option::Option<Range>,
#[prost(int32, tag = "2")]
pub output_level: i32,
#[prost(message, optional, tag = "3")]
pub context: ::core::option::Option<super::kvrpcpb::Context>,
}
#[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 \<name, SSTMeta>.
/// 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<RewriteRule>,
#[prost(message, optional, tag = "14")]
pub storage_backend: ::core::option::Option<super::backup::StorageBackend>,
/// 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<super::backup::CipherInfo>,
/// 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<super::kvrpcpb::Context>,
}
/// 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<super::errorpb::Error>,
}
#[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<SstMeta>,
}
#[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<super::kvrpcpb::Context>,
#[prost(oneof = "write_request::Chunk", tags = "1, 2")]
pub chunk: ::core::option::Option<write_request::Chunk>,
}
@ -279,10 +341,19 @@ pub struct RawWriteBatch {
pub ttl: u64,
#[prost(message, repeated, tag = "2")]
pub pairs: ::prost::alloc::vec::Vec<Pair>,
/// 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<super::kvrpcpb::Context>,
#[prost(oneof = "raw_write_request::Chunk", tags = "1, 2")]
pub chunk: ::core::option::Option<raw_write_request::Chunk>,
}
@ -351,6 +422,109 @@ pub struct DuplicateDetectResponse {
#[prost(message, repeated, tag = "3")]
pub pairs: ::prost::alloc::vec::Vec<KvPair>,
}
#[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<u8>,
#[prost(bytes = "vec", tag = "7")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
/// used for checksum when download kv file.
#[prost(bytes = "vec", tag = "8")]
pub sha256: ::prost::alloc::vec::Vec<u8>,
/// 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<KvMeta>,
#[prost(message, repeated, tag = "12")]
pub metas: ::prost::alloc::vec::Vec<KvMeta>,
/// 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<RewriteRule>,
#[prost(message, repeated, tag = "13")]
pub rewrite_rules: ::prost::alloc::vec::Vec<RewriteRule>,
/// 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<super::backup::StorageBackend>,
/// context represents region info and it used to build raft commands.
#[prost(message, optional, tag = "4")]
pub context: ::core::option::Option<super::kvrpcpb::Context>,
/// cipher_info is used to decrypt kv file when download file.
#[prost(message, optional, tag = "11")]
pub cipher_info: ::core::option::Option<super::backup::CipherInfo>,
}
#[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<Range>,
#[prost(message, optional, tag = "2")]
pub error: ::core::option::Option<Error>,
}
#[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<Error>,
}
#[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<Self> {
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<super::GetModeRequest>,
) -> std::result::Result<
tonic::Response<super::GetModeResponse>,
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<super::ApplyRequest>,
) -> std::result::Result<tonic::Response<super::ApplyResponse>, 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<super::ClearRequest>,
) -> std::result::Result<tonic::Response<super::ClearResponse>, 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<super::SuspendImportRpcRequest>,
) -> std::result::Result<
tonic::Response<super::SuspendImportRpcResponse>,
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
}
}
}

308
src/generated/keyspacepb.rs Normal file
View File

@ -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<super::pdpb::RequestHeader>,
#[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<super::pdpb::ResponseHeader>,
#[prost(message, optional, tag = "2")]
pub keyspace: ::core::option::Option<KeyspaceMeta>,
}
#[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<super::pdpb::RequestHeader>,
}
#[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<super::pdpb::ResponseHeader>,
#[prost(message, repeated, tag = "2")]
pub keyspaces: ::prost::alloc::vec::Vec<KeyspaceMeta>,
}
#[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<super::pdpb::RequestHeader>,
#[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<super::pdpb::ResponseHeader>,
#[prost(message, optional, tag = "2")]
pub keyspace: ::core::option::Option<KeyspaceMeta>,
}
#[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<super::pdpb::RequestHeader>,
#[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<super::pdpb::ResponseHeader>,
#[prost(message, repeated, tag = "2")]
pub keyspaces: ::prost::alloc::vec::Vec<KeyspaceMeta>,
}
#[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<Self> {
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<T> {
inner: tonic::client::Grpc<T>,
}
impl KeyspaceClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> KeyspaceClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> KeyspaceClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::LoadKeyspaceRequest>,
) -> std::result::Result<
tonic::Response<super::LoadKeyspaceResponse>,
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<super::WatchKeyspacesRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::WatchKeyspacesResponse>>,
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<super::UpdateKeyspaceStateRequest>,
) -> std::result::Result<
tonic::Response<super::UpdateKeyspaceStateResponse>,
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<super::GetAllKeyspacesRequest>,
) -> std::result::Result<
tonic::Response<super::GetAllKeyspacesResponse>,
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
}
}
}

View File

@ -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<bool>,
/// Keys with deferred constraint checks are not locked.
#[prost(enumeration = "prewrite_request::PessimisticAction", repeated, tag = "7")]
pub pessimistic_actions: ::prost::alloc::vec::Vec<i32>,
/// 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 <https://github.com/tikv/tikv/issues/14311>
#[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<Self> {
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<ExecDetailsV2>,
}
/// 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<u8>,
#[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<u8>,
/// 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<u8>>,
/// 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<bool>,
/// Execution details about the request processing.
#[prost(message, optional, tag = "7")]
pub exec_details_v2: ::core::option::Option<ExecDetailsV2>,
/// Results of the request. Only used when `wake_up_mode` is `WakeUpModeForceLock`.
#[prost(message, repeated, tag = "8")]
pub results: ::prost::alloc::vec::Vec<PessimisticLockKeyResult>,
}
/// 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<super::errorpb::Error>,
#[prost(message, repeated, tag = "2")]
pub errors: ::prost::alloc::vec::Vec<KeyError>,
/// Execution details about the request processing.
#[prost(message, optional, tag = "3")]
pub exec_details_v2: ::core::option::Option<ExecDetailsV2>,
}
/// 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<ExecDetailsV2>,
}
/// 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 <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.
#[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<LockInfo>,
/// Execution details about the request processing.
#[prost(message, optional, tag = "7")]
pub exec_details_v2: ::core::option::Option<ExecDetailsV2>,
}
/// 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<ExecDetailsV2>,
}
/// 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<ExecDetailsV2>,
}
/// Not yet implemented.
#[allow(clippy::derive_partial_eq_without_eq)]
@ -478,6 +605,9 @@ pub struct BatchRollbackResponse {
pub region_error: ::core::option::Option<super::errorpb::Error>,
#[prost(message, optional, tag = "2")]
pub error: ::core::option::Option<KeyError>,
/// Execution details about the request processing.
#[prost(message, optional, tag = "3")]
pub exec_details_v2: ::core::option::Option<ExecDetailsV2>,
}
/// 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<LockInfo>,
/// Execution details about the request processing.
#[prost(message, optional, tag = "4")]
pub exec_details_v2: ::core::option::Option<ExecDetailsV2>,
}
/// 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<super::errorpb::Error>,
#[prost(message, optional, tag = "2")]
pub error: ::core::option::Option<KeyError>,
/// Execution details about the request processing.
#[prost(message, optional, tag = "3")]
pub exec_details_v2: ::core::option::Option<ExecDetailsV2>,
}
/// 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<KeyError>,
}
/// 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<Context>,
#[prost(bytes = "vec", tag = "2")]
pub start_key: ::prost::alloc::vec::Vec<u8>,
#[prost(bytes = "vec", tag = "3")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
/// 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<super::errorpb::Error>,
#[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<Context>,
/// 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<u8>,
#[prost(bytes = "vec", tag = "4")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
/// 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<super::errorpb::Error>,
#[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<KvPair>,
#[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<u64>,
#[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<u64>,
/// The informantion to trace a request sent to TiKV.
#[prost(message, optional, tag = "23")]
pub trace_context: ::core::option::Option<super::tracepb::TraceContext>,
/// 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<ResourceControlContext>,
/// 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<SourceStmt>,
}
#[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<super::resource_manager::Consumption>,
/// 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<u8>>,
/// 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<AssertionFailed>,
/// CheckTxnStatus is sent to a lock that's not the primary.
#[prost(message, optional, tag = "11")]
pub primary_mismatch: ::core::option::Option<PrimaryMismatch>,
}
#[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<u8>,
#[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<Self> {
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<LockInfo>,
}
#[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<TimeDetail>,
/// Available when ctx.record_scan_stat = true or meet slow query.
#[prost(message, optional, tag = "2")]
pub scan_detail_v2: ::core::option::Option<ScanDetailV2>,
/// Raftstore writing durations of the request. Only available for some write requests.
#[prost(message, optional, tag = "3")]
pub write_detail: ::core::option::Option<WriteDetail>,
/// Available when ctx.record_time_stat = true or meet slow query.
#[prost(message, optional, tag = "4")]
pub time_detail_v2: ::core::option::Option<TimeDetailV2>,
}
#[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<u8>>,
#[prost(uint64, repeated, tag = "10")]
pub rollback_ts: ::prost::alloc::vec::Vec<u64>,
#[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<Context>,
}
#[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<super::errorpb::Error>,
#[prost(string, tag = "2")]
pub error: ::prost::alloc::string::String,
#[prost(message, repeated, tag = "3")]
pub entries: ::prost::alloc::vec::Vec<super::deadlock::WaitForEntry>,
}
#[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<Context>,
@ -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<compact_error::Error>,
}
/// 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<u8>,
/// 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>
#[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 <https://github.com/pingcap/kvproto/issues/912>
#[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<CompactError>,
/// 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<u8>,
#[prost(bytes = "vec", tag = "4")]
pub compacted_end_key: ::prost::alloc::vec::Vec<u8>,
}
#[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<u8>,
}
/// 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<Self> {
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<Self> {
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 <https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md.>
#[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,
}
}

199
src/generated/logbackup.rs Normal file
View File

@ -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<super::errorpb::Error>,
#[prost(message, optional, tag = "2")]
pub region: ::core::option::Option<RegionIdentity>,
#[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<RegionIdentity>,
}
#[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<RegionCheckpoint>,
}
#[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<FlushEvent>,
}
#[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<u8>,
#[prost(bytes = "vec", tag = "2")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
#[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<T> {
inner: tonic::client::Grpc<T>,
}
impl LogBackupClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> LogBackupClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> LogBackupClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::GetLastFlushTsOfRegionRequest>,
) -> std::result::Result<
tonic::Response<super::GetLastFlushTsOfRegionResponse>,
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<super::SubscribeFlushEventRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::SubscribeFlushEventResponse>>,
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
}
}
}

View File

@ -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<Error>,
#[prost(int64, tag = "3")]
pub revision: i64,
}
/// copied part of <https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto>
#[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<RequestHeader>,
#[prost(bytes = "vec", tag = "2")]
pub key: ::prost::alloc::vec::Vec<u8>,
#[prost(bytes = "vec", tag = "3")]
pub range_end: ::prost::alloc::vec::Vec<u8>,
#[prost(int64, tag = "4")]
pub start_revision: i64,
#[prost(bool, tag = "5")]
pub prev_kv: bool,
}
/// copied part of <https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto>
#[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<ResponseHeader>,
#[prost(int64, tag = "2")]
pub compact_revision: i64,
#[prost(message, repeated, tag = "3")]
pub events: ::prost::alloc::vec::Vec<Event>,
}
/// copied part of <https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto>
#[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<RequestHeader>,
#[prost(bytes = "vec", tag = "2")]
pub key: ::prost::alloc::vec::Vec<u8>,
#[prost(bytes = "vec", tag = "3")]
pub range_end: ::prost::alloc::vec::Vec<u8>,
#[prost(int64, tag = "4")]
pub limit: i64,
#[prost(int64, tag = "5")]
pub revision: i64,
}
/// copied part of <https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto>
#[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<ResponseHeader>,
#[prost(message, repeated, tag = "2")]
pub kvs: ::prost::alloc::vec::Vec<KeyValue>,
#[prost(bool, tag = "3")]
pub more: bool,
#[prost(int64, tag = "4")]
pub count: i64,
}
/// copied part of <https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto>
#[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<RequestHeader>,
#[prost(bytes = "vec", tag = "2")]
pub key: ::prost::alloc::vec::Vec<u8>,
#[prost(bytes = "vec", tag = "3")]
pub value: ::prost::alloc::vec::Vec<u8>,
#[prost(int64, tag = "4")]
pub lease: i64,
#[prost(bool, tag = "5")]
pub prev_kv: bool,
}
/// copied part of <https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/etcdserverpb/rpc.proto>
#[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<ResponseHeader>,
#[prost(message, optional, tag = "2")]
pub prev_kv: ::core::option::Option<KeyValue>,
}
/// copied from etcd <https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/mvccpb/kv.proto>
#[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<u8>,
/// 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<u8>,
/// 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 <https://github.com/etcd-io/etcd/blob/7dfd29b0cc7ce25337276dce646ca2a65aa44b4d/api/mvccpb/kv.proto>
#[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<KeyValue>,
/// prev_kv holds the key-value pair before the event happens.
#[prost(message, optional, tag = "3")]
pub prev_kv: ::core::option::Option<KeyValue>,
}
/// 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<Self> {
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<Self> {
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<T> {
inner: tonic::client::Grpc<T>,
}
impl MetaStorageClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> MetaStorageClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> MetaStorageClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::WatchRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::WatchResponse>>,
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<super::GetRequest>,
) -> std::result::Result<tonic::Response<super::GetResponse>, 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<super::PutRequest>,
) -> std::result::Result<tonic::Response<super::PutResponse>, 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
}
}
}

View File

@ -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<u64>,
/// total write in bytes of each bucket
#[prost(uint64, repeated, tag = "2")]
pub write_bytes: ::prost::alloc::vec::Vec<u64>,
/// total read qps of each bucket
#[prost(uint64, repeated, tag = "3")]
pub read_qps: ::prost::alloc::vec::Vec<u64>,
/// total write qps of each bucket
#[prost(uint64, repeated, tag = "4")]
pub write_qps: ::prost::alloc::vec::Vec<u64>,
/// total read keys of each bucket
#[prost(uint64, repeated, tag = "5")]
pub read_keys: ::prost::alloc::vec::Vec<u64>,
/// total write keys of each bucket
#[prost(uint64, repeated, tag = "6")]
pub write_keys: ::prost::alloc::vec::Vec<u64>,
}
#[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<u8>>,
/// bucket stats
#[prost(message, optional, tag = "4")]
pub stats: ::core::option::Option<BucketStats>,
/// 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<super::encryptionpb::EncryptionMeta>,
/// 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<Self> {
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 {

View File

@ -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");
}

View File

@ -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<super::coprocessor::TableRegions>,
}
/// 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<Error>,
}
/// 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<TaskMeta>,
#[prost(bytes = "vec", tag = "2")]
pub data: ::prost::alloc::vec::Vec<u8>,
#[prost(message, optional, tag = "3")]
pub error: ::core::option::Option<Error>,
}
#[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<Error>,
}
/// 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<Error>,
#[prost(bytes = "vec", repeated, tag = "3")]
pub chunks: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec<u8>>,
#[prost(uint64, repeated, tag = "4")]
pub stream_ids: ::prost::alloc::vec::Vec<u64>,
/// 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,
}

File diff suppressed because it is too large Load Diff

View File

@ -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<super::metapb::Region>,
@ -278,6 +309,9 @@ pub struct CommitMergeRequest {
pub commit: u64,
#[prost(message, repeated, tag = "3")]
pub entries: ::prost::alloc::vec::Vec<super::eraftpb::Entry>,
/// 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<super::raft_serverpb::RegionLocalState>,
}
#[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<SwitchWitnessRequest>,
}
#[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<u64>,
}
#[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<ChangePeerV2Request>,
#[prost(message, optional, tag = "12")]
pub compute_hash: ::core::option::Option<ComputeHashRequest>,
#[prost(message, optional, tag = "13")]
pub prepare_flashback: ::core::option::Option<PrepareFlashbackRequest>,
#[prost(message, optional, tag = "14")]
pub finish_flashback: ::core::option::Option<FinishFlashbackRequest>,
#[prost(message, optional, tag = "15")]
pub switch_witnesses: ::core::option::Option<BatchSwitchWitnessRequest>,
#[prost(message, optional, tag = "16")]
pub update_gc_peers: ::core::option::Option<UpdateGcPeerRequest>,
}
#[allow(clippy::derive_partial_eq_without_eq)]
#[derive(Clone, PartialEq, ::prost::Message)]
@ -346,6 +411,13 @@ pub struct AdminResponse {
pub splits: ::core::option::Option<BatchSplitResponse>,
#[prost(message, optional, tag = "11")]
pub change_peer_v2: ::core::option::Option<ChangePeerV2Response>,
#[prost(message, optional, tag = "12")]
pub prepare_flashback: ::core::option::Option<PrepareFlashbackResponse>,
#[prost(message, optional, tag = "13")]
pub finish_flashback: ::core::option::Option<FinishFlashbackResponse>,
/// UpdateGcPeer doesn't need to be responded. Avoid consuming a tag number.
#[prost(message, optional, tag = "14")]
pub switch_witnesses: ::core::option::Option<BatchSwitchWitnessResponse>,
}
/// 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<u8>,
#[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,
}
}

View File

@ -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 <https://github.com/pingcap/tiflash/issues/7568>
#[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<u8>,
/// trailing data including checksum. Unencrypted.
#[prost(bytes = "vec", tag = "4")]
pub trailing_chunk: ::prost::alloc::vec::Vec<u8>,
}
/// 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<TabletSnapshotFileMeta>,
/// 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<u8>,
/// Initial vector if encryption is enabled.
#[prost(bytes = "vec", tag = "4")]
pub iv: ::prost::alloc::vec::Vec<u8>,
#[prost(message, optional, tag = "5")]
pub key: ::core::option::Option<super::encryptionpb::DataKey>,
}
#[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<RaftMessage>,
#[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<tablet_snapshot_request::Payload>,
}
/// 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<AcceptedSnapshotFiles>,
}
#[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<u8>,
@ -88,6 +200,10 @@ pub struct RaftSnapshotData {
pub version: u64,
#[prost(message, optional, tag = "5")]
pub meta: ::core::option::Option<SnapshotMeta>,
#[prost(message, repeated, tag = "6")]
pub removed_records: ::prost::alloc::vec::Vec<super::metapb::Peer>,
#[prost(message, repeated, tag = "7")]
pub merged_records: ::prost::alloc::vec::Vec<MergedRecord>,
}
#[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<super::eraftpb::HardState>,
@ -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<super::metapb::RegionEpoch>,
/// Peers of source region when merge is committed.
#[prost(message, repeated, tag = "3")]
pub source_peers: ::prost::alloc::vec::Vec<super::metapb::Peer>,
/// Removed peers (by confchange) of source region when merge is committed.
#[prost(message, repeated, tag = "9")]
pub source_removed_records: ::prost::alloc::vec::Vec<super::metapb::Peer>,
#[prost(uint64, tag = "4")]
pub target_region_id: u64,
#[prost(message, optional, tag = "5")]
pub target_epoch: ::core::option::Option<super::metapb::RegionEpoch>,
#[prost(message, repeated, tag = "6")]
pub target_peers: ::prost::alloc::vec::Vec<super::metapb::Peer>,
/// 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<super::metapb::Region>,
#[prost(message, optional, tag = "3")]
pub merge_state: ::core::option::Option<MergeState>,
/// 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<super::metapb::Peer>,
/// 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<MergedRecord>,
}
#[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<RaftApplyState>,
#[prost(message, optional, tag = "4")]
pub region_state: ::core::option::Option<RegionLocalState>,
}
#[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<super::metapb::RegionEpoch>,
#[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<u8>>,
#[prost(uint64, repeated, tag = "3")]
pub sizes: ::prost::alloc::vec::Vec<u64>,
}
#[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<super::metapb::RegionEpoch>,
/// The peer to be checked.
#[prost(message, optional, tag = "4")]
pub check_peer: ::core::option::Option<super::metapb::Peer>,
}
#[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<super::metapb::Peer>,
#[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<CheckGcPeer>,
#[prost(message, optional, tag = "7")]
pub flush_memtable: ::core::option::Option<FlushMemtable>,
/// Used by `MsgAvailabilityRequest` and `MsgAvailabilityResponse` in v2.
#[prost(message, optional, tag = "8")]
pub availability_context: ::core::option::Option<AvailabilityContext>,
/// notice the peer to refresh buckets version
#[prost(message, optional, tag = "9")]
pub refresh_buckets: ::core::option::Option<RefreshBuckets>,
}
#[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,
}
}

View File

@ -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<u8>,
#[prost(bytes = "vec", tag = "9")]
pub end_key: ::prost::alloc::vec::Vec<u8>,
}
/// 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<Error>,
#[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<Error>,
}
/// 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<Error>,
#[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<T> {
inner: tonic::client::Grpc<T>,
}
impl RecoverDataClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> RecoverDataClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> RecoverDataClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::ReadRegionMetaRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::RegionMeta>>,
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<super::RecoverRegionResponse>,
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<super::WaitApplyRequest>,
) -> std::result::Result<
tonic::Response<super::WaitApplyResponse>,
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<super::ResolveKvDataRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::ResolveKvDataResponse>>,
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
}
}
}

View File

@ -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<u64>,
/// 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<Self> {
match value {
"SYNC" => Some(Self::Sync),
"ASYNC_WAIT" => Some(Self::AsyncWait),
"ASYNC" => Some(Self::Async),
"SYNC_RECOVER" => Some(Self::SyncRecover),
_ => None,

View File

@ -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<Error>,
#[prost(message, repeated, tag = "2")]
pub groups: ::prost::alloc::vec::Vec<ResourceGroup>,
}
#[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<Error>,
#[prost(message, optional, tag = "2")]
pub group: ::core::option::Option<ResourceGroup>,
}
#[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<Error>,
#[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<ResourceGroup>,
}
#[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<Error>,
#[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<TokenBucketRequest>,
#[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<Consumption>,
/// 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<token_bucket_request::Request>,
}
/// 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<super::RequestUnitItem>,
}
#[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<super::RawResourceItem>,
}
#[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<Error>,
#[prost(message, repeated, tag = "2")]
pub responses: ::prost::alloc::vec::Vec<TokenBucketResponse>,
}
#[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<GrantedRuTokenBucket>,
/// Raw mode
#[prost(message, repeated, tag = "3")]
pub granted_resource_tokens: ::prost::alloc::vec::Vec<GrantedRawResourceTokenBucket>,
}
#[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<TokenBucket>,
#[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<TokenBucket>,
#[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<GroupRequestUnitSettings>,
/// Used in Raw mode, group settings with CPU/IO etc resource unit.
#[prost(message, optional, tag = "4")]
pub raw_resource_settings: ::core::option::Option<GroupRawResourceSettings>,
/// 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<RunawaySettings>,
#[prost(message, optional, tag = "7")]
pub background_settings: ::core::option::Option<BackgroundSettings>,
}
#[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<TokenBucket>,
}
#[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<TokenBucket>,
#[prost(message, optional, tag = "2")]
pub io_read: ::core::option::Option<TokenBucket>,
#[prost(message, optional, tag = "3")]
pub io_write: ::core::option::Option<TokenBucket>,
}
#[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<TokenLimitSettings>,
/// 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<RunawayRule>,
#[prost(enumeration = "RunawayAction", tag = "2")]
pub action: i32,
#[prost(message, optional, tag = "3")]
pub watch: ::core::option::Option<RunawayWatch>,
}
#[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<Self> {
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<Self> {
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<Self> {
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<Self> {
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<Self> {
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<T> {
inner: tonic::client::Grpc<T>,
}
impl ResourceManagerClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> ResourceManagerClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> ResourceManagerClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::ListResourceGroupsRequest>,
) -> std::result::Result<
tonic::Response<super::ListResourceGroupsResponse>,
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<super::GetResourceGroupRequest>,
) -> std::result::Result<
tonic::Response<super::GetResourceGroupResponse>,
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<super::PutResourceGroupRequest>,
) -> std::result::Result<
tonic::Response<super::PutResourceGroupResponse>,
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<super::PutResourceGroupRequest>,
) -> std::result::Result<
tonic::Response<super::PutResourceGroupResponse>,
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<super::DeleteResourceGroupRequest>,
) -> std::result::Result<
tonic::Response<super::DeleteResourceGroupResponse>,
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::codec::Streaming<super::TokenBucketsResponse>>,
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
}
}
}

View File

@ -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<u8>,
/// UNIX timestamp in second.
#[prost(uint64, repeated, tag = "2")]
pub record_list_timestamp_sec: ::prost::alloc::vec::Vec<u64>,
/// 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<u32>,
}
#[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<u8>,
/// UNIX timestamp in second.
#[prost(uint64, repeated, tag = "2")]
pub record_list_timestamp_sec: ::prost::alloc::vec::Vec<u64>,
/// 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<u32>,
/// 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<u32>,
/// 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<u32>,
}
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<resource_usage_record::RecordOneof>,
}
/// 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<u8>,
#[prost(message, repeated, tag = "2")]
pub items: ::prost::alloc::vec::Vec<GroupTagRecordItem>,
}
#[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<Message = super::CpuTimeRecord>,
) -> std::result::Result<tonic::Response<super::EmptyResponse>, 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(

View File

@ -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<Error>,
}
#[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<RequestHeader>,
#[prost(message, optional, tag = "2")]
pub stats: ::core::option::Option<super::pdpb::StoreStats>,
}
#[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<ResponseHeader>,
#[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<RequestHeader>,
#[prost(message, optional, tag = "2")]
pub region: ::core::option::Option<super::metapb::Region>,
/// Leader Peer sending the heartbeat.
#[prost(message, optional, tag = "3")]
pub leader: ::core::option::Option<super::metapb::Peer>,
/// 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<super::pdpb::PeerStats>,
/// 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<super::metapb::Peer>,
/// 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<super::pdpb::QueryStats>,
/// Actually reported time interval
#[prost(message, optional, tag = "14")]
pub interval: ::core::option::Option<super::pdpb::TimeInterval>,
}
#[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<ResponseHeader>,
/// ID of the region
#[prost(uint64, tag = "2")]
pub region_id: u64,
#[prost(message, optional, tag = "3")]
pub region_epoch: ::core::option::Option<super::metapb::RegionEpoch>,
/// Leader of the region at the moment of the corresponding request was made.
#[prost(message, optional, tag = "4")]
pub target_peer: ::core::option::Option<super::metapb::Peer>,
/// 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<super::pdpb::ChangePeer>,
/// Pd can return transfer_leader to let TiKV does leader transfer itself.
#[prost(message, optional, tag = "6")]
pub transfer_leader: ::core::option::Option<super::pdpb::TransferLeader>,
#[prost(message, optional, tag = "7")]
pub merge: ::core::option::Option<super::pdpb::Merge>,
/// PD sends split_region to let TiKV split a region into two regions.
#[prost(message, optional, tag = "8")]
pub split_region: ::core::option::Option<super::pdpb::SplitRegion>,
/// 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<super::pdpb::ChangePeerV2>,
#[prost(message, optional, tag = "10")]
pub switch_witnesses: ::core::option::Option<super::pdpb::BatchSwitchWitness>,
}
#[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<RequestHeader>,
/// 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<u64>,
#[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<ResponseHeader>,
#[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<RequestHeader>,
#[prost(bytes = "vec", repeated, tag = "2")]
pub split_keys: ::prost::alloc::vec::Vec<::prost::alloc::vec::Vec<u8>>,
#[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<ResponseHeader>,
#[prost(uint64, tag = "2")]
pub finished_percentage: u64,
#[prost(uint64, repeated, tag = "3")]
pub regions_id: ::prost::alloc::vec::Vec<u64>,
}
#[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<RequestHeader>,
#[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<ResponseHeader>,
#[prost(uint64, tag = "2")]
pub region_id: u64,
#[prost(bytes = "vec", tag = "3")]
pub desc: ::prost::alloc::vec::Vec<u8>,
#[prost(enumeration = "super::pdpb::OperatorStatus", tag = "4")]
pub status: i32,
#[prost(bytes = "vec", tag = "5")]
pub kind: ::prost::alloc::vec::Vec<u8>,
}
#[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<RequestHeader>,
#[prost(message, optional, tag = "2")]
pub region: ::core::option::Option<super::metapb::Region>,
#[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<ResponseHeader>,
#[prost(message, repeated, tag = "2")]
pub ids: ::prost::alloc::vec::Vec<super::pdpb::SplitId>,
}
#[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<Self> {
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<T> {
inner: tonic::client::Grpc<T>,
}
impl SchedulingClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> SchedulingClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> SchedulingClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::StoreHeartbeatRequest>,
) -> std::result::Result<
tonic::Response<super::StoreHeartbeatResponse>,
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::codec::Streaming<super::RegionHeartbeatResponse>>,
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<super::SplitRegionsRequest>,
) -> std::result::Result<
tonic::Response<super::SplitRegionsResponse>,
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<super::ScatterRegionsRequest>,
) -> std::result::Result<
tonic::Response<super::ScatterRegionsResponse>,
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<super::GetOperatorRequest>,
) -> std::result::Result<
tonic::Response<super::GetOperatorResponse>,
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<super::AskBatchSplitRequest>,
) -> std::result::Result<
tonic::Response<super::AskBatchSplitResponse>,
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
}
}
}

View File

@ -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<request::Cmd>,
}
@ -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<response::Cmd>,
}
@ -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<super::super::kvrpcpb::PrepareFlashbackToVersionResponse>,
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<super::super::kvrpcpb::FlashbackToVersionResponse>,
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<super::super::mpp::ReportTaskStatusRequest>,
) -> std::result::Result<
tonic::Response<super::super::mpp::ReportTaskStatusResponse>,
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<super::super::kvrpcpb::CompactRequest>,
) -> std::result::Result<
tonic::Response<super::super::kvrpcpb::CompactResponse>,
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<super::super::kvrpcpb::GetLockWaitHistoryResponse>,
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<super::super::kvrpcpb::TiFlashSystemTableResponse>,
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<super::super::disaggregated::TryAddLockResponse>,
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<super::super::disaggregated::TryMarkDeleteResponse>,
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<super::super::disaggregated::EstablishDisaggTaskResponse>,
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<super::super::disaggregated::CancelDisaggTaskResponse>,
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<super::super::disaggregated::PagesPacket>,
>,
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<super::super::disaggregated::GetDisaggConfigResponse>,
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
}
}
}

197
src/generated/tracepb.rs Normal file
View File

@ -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<trace_record::RecordOneof>,
}
/// 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<RemoteParentSpan>,
/// 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<RemoteParentSpan>,
#[prost(message, repeated, tag = "2")]
pub spans: ::prost::alloc::vec::Vec<Span>,
}
/// 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<Property>,
}
#[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<T> {
inner: tonic::client::Grpc<T>,
}
impl TraceRecordPubSubClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> TraceRecordPubSubClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> TraceRecordPubSubClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<super::TraceRecordRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::TraceRecord>>,
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
}
}
}

337
src/generated/tsopb.rs Normal file
View File

@ -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<Error>,
/// 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<RequestHeader>,
#[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<ResponseHeader>,
#[prost(uint32, tag = "2")]
pub count: u32,
#[prost(message, optional, tag = "3")]
pub timestamp: ::core::option::Option<super::pdpb::Timestamp>,
}
#[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<SplitState>,
#[prost(message, repeated, tag = "4")]
pub members: ::prost::alloc::vec::Vec<KeyspaceGroupMember>,
}
#[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<RequestHeader>,
#[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<ResponseHeader>,
#[prost(message, optional, tag = "2")]
pub keyspace_group: ::core::option::Option<KeyspaceGroup>,
}
#[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<RequestHeader>,
#[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<ResponseHeader>,
#[prost(message, optional, tag = "2")]
pub timestamp: ::core::option::Option<super::pdpb::Timestamp>,
/// 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<Self> {
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<T> {
inner: tonic::client::Grpc<T>,
}
impl TsoClient<tonic::transport::Channel> {
/// Attempt to create a new client by connecting to a given endpoint.
pub async fn connect<D>(dst: D) -> Result<Self, tonic::transport::Error>
where
D: TryInto<tonic::transport::Endpoint>,
D::Error: Into<StdError>,
{
let conn = tonic::transport::Endpoint::new(dst)?.connect().await?;
Ok(Self::new(conn))
}
}
impl<T> TsoClient<T>
where
T: tonic::client::GrpcService<tonic::body::BoxBody>,
T::Error: Into<StdError>,
T::ResponseBody: Body<Data = Bytes> + Send + 'static,
<T::ResponseBody as Body>::Error: Into<StdError> + 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<F>(
inner: T,
interceptor: F,
) -> TsoClient<InterceptedService<T, F>>
where
F: tonic::service::Interceptor,
T::ResponseBody: Default,
T: tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
Response = http::Response<
<T as tonic::client::GrpcService<tonic::body::BoxBody>>::ResponseBody,
>,
>,
<T as tonic::codegen::Service<
http::Request<tonic::body::BoxBody>,
>>::Error: Into<StdError> + 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<Message = super::TsoRequest>,
) -> std::result::Result<
tonic::Response<tonic::codec::Streaming<super::TsoResponse>>,
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<super::FindGroupByKeyspaceIdRequest>,
) -> std::result::Result<
tonic::Response<super::FindGroupByKeyspaceIdResponse>,
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<super::GetMinTsRequest>,
) -> std::result::Result<
tonic::Response<super::GetMinTsResponse>,
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
}
}
}

View File

@ -136,17 +136,11 @@ impl BoundRange {
pub fn into_keys(self) -> (Key, Option<Key>) {
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,
};

View File

@ -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::<Vec<u8>>((size_range(_PROPTEST_KEY_MAX), ()))")
)]
pub(super) Vec<u8>,
pub(crate) Vec<u8>,
);
impl AsRef<Key> for kvrpcpb::Mutation {
@ -80,6 +80,20 @@ impl AsRef<Key> for kvrpcpb::Mutation {
}
}
pub struct KvPairTTL(pub KvPair, pub u64);
impl AsRef<Key> for KvPairTTL {
fn as_ref(&self) -> &Key {
self.0.key.as_ref()
}
}
impl From<KvPairTTL> 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.

View File

@ -25,7 +25,7 @@ use crate::proto::kvrpcpb;
///
/// Many functions which accept a `KvPair` accept an `Into<KvPair>`, 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);

View File

@ -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;

View File

@ -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;

View File

@ -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<ApiV1TxnCodec, MockKvConnect, MockCluster> {
pub async fn pd_rpc_client() -> PdRpcClient<MockKvConnect, MockCluster> {
let config = Config::default();
PdRpcClient::new(
config.clone(),
@ -44,7 +44,6 @@ pub async fn pd_rpc_client() -> PdRpcClient<ApiV1TxnCodec, MockKvConnect, MockCl
))
},
false,
Some(ApiV1TxnCodec::default()),
)
.await
.unwrap()
@ -73,18 +72,9 @@ pub struct MockKvConnect;
pub struct MockCluster;
#[derive(new)]
pub struct MockPdClient {
client: MockKvClient,
codec: ApiV1TxnCodec,
}
impl MockPdClient {
pub fn new(client: MockKvClient) -> 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<Self>, region: RegionWithLeader) -> Result<RegionStore> {
@ -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<keyspacepb::KeyspaceMeta> {
unimplemented!()
}
}

View File

@ -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<Self>, safepoint: u64) -> Result<bool>;
async fn load_keyspace(&self, keyspace: &str) -> Result<keyspacepb::KeyspaceMeta>;
/// In transactional API, `key` is in raw format
async fn store_for_key(self: Arc<Self>, key: &Key) -> Result<RegionStore> {
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<RegionWithLeader> {
if enable_mvcc_codec {
fn decode_region(mut region: RegionWithLeader, enable_codec: bool) -> Result<RegionWithLeader> {
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<KvC: KvConnect + Send + Sync + 'static = TikvConnect, Cl = Cluster> {
pd: Arc<RetryClient<Cl>>,
kv_connect: KvC,
kv_client_cache: Arc<RwLock<HashMap<String, KvC::KvClient>>>,
enable_mvcc_codec: bool,
enable_codec: bool,
region_cache: RegionCache<RetryClient<Cl>>,
codec: Option<Cod>,
}
#[async_trait]
impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static> PdClient for PdRpcClient<Cod, KvC> {
type Codec = Cod;
impl<KvC: KvConnect + Send + Sync + 'static> PdClient for PdRpcClient<KvC> {
type KvClient = KvC::KvClient;
async fn map_region_to_store(self: Arc<Self>, region: RegionWithLeader) -> Result<RegionStore> {
@ -241,20 +229,20 @@ impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static> PdClient for PdRpcClien
}
async fn region_for_key(&self, key: &Key) -> Result<RegionWithLeader> {
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<RegionWithLeader> {
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<Vec<Store>> {
@ -283,39 +271,34 @@ impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static> 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<keyspacepb::KeyspaceMeta> {
self.pd.load_keyspace(keyspace).await
}
}
impl<Cod: Codec> PdRpcClient<Cod, TikvConnect, Cluster> {
impl PdRpcClient<TikvConnect, Cluster> {
pub async fn connect(
pd_endpoints: &[String],
config: Config,
enable_mvcc_codec: bool, // TODO: infer from `codec`.
codec: Option<Cod>,
) -> Result<PdRpcClient<Cod>> {
enable_codec: bool,
) -> Result<PdRpcClient> {
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<Cod: Codec, KvC: KvConnect + Send + Sync + 'static, Cl> PdRpcClient<Cod, KvC, Cl> {
impl<KvC: KvConnect + Send + Sync + 'static, Cl> PdRpcClient<KvC, Cl> {
pub async fn new<PdFut, MakeKvC, MakePd>(
config: Config,
kv_connect: MakeKvC,
pd: MakePd,
enable_mvcc_codec: bool,
codec: Option<Cod>,
) -> Result<PdRpcClient<Cod, KvC, Cl>>
enable_codec: bool,
) -> Result<PdRpcClient<KvC, Cl>>
where
PdFut: Future<Output = Result<RetryClient<Cl>>>,
MakeKvC: FnOnce(Arc<SecurityManager>) -> KvC,
@ -337,9 +320,8 @@ impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static, Cl> PdRpcClient<Cod, Kv
pd: pd.clone(),
kv_client_cache,
kv_connect: kv_connect(security_mgr),
enable_mvcc_codec,
enable_codec,
region_cache: RegionCache::new(pd),
codec,
})
}
@ -359,10 +341,6 @@ impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static, Cl> PdRpcClient<Cod, Kv
Err(e) => Err(e),
}
}
pub fn set_codec(&mut self, codec: Cod) {
self.codec = Some(codec);
}
}
fn make_key_range(start_key: Vec<u8>, end_key: Vec<u8>) -> kvrpcpb::KeyRange {

View File

@ -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<Channel>,
keyspace_client: keyspacepb::keyspace_client::KeyspaceClient<Channel>,
members: pdpb::GetMembersResponse,
tso: TimestampOracle,
}
@ -46,7 +49,7 @@ impl Cluster {
timeout: Duration,
) -> Result<pdpb::GetRegionResponse> {
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<keyspacepb::KeyspaceMeta> {
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<Cluster> {
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<Channel>, pdpb::GetMembersResponse)> {
) -> Result<(
pdpb::pd_client::PdClient<Channel>,
keyspacepb::keyspace_client::KeyspaceClient<Channel>,
pdpb::GetMembersResponse,
)> {
let mut client = self
.security_mgr
.connect(addr, pdpb::pd_client::PdClient::<Channel>::new)
.await?;
let keyspace_client = self
.security_mgr
.connect(
addr,
keyspacepb::keyspace_client::KeyspaceClient::<Channel>::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<Channel>, pdpb::GetMembersResponse)> {
let (client, r) = self.connect(addr, timeout).await?;
) -> Result<(
pdpb::pd_client::PdClient<Channel>,
keyspacepb::keyspace_client::KeyspaceClient<Channel>,
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<Channel>, pdpb::GetMembersResponse)> {
) -> Result<(
pdpb::pd_client::PdClient<Channel>,
keyspacepb::keyspace_client::KeyspaceClient<Channel>,
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<T> = std::result::Result<T, tonic::Status>;
#[async_trait]
trait PdMessage: Sized {
type Client: Send;
type Response: PdResponse;
async fn rpc(
req: Request<Self>,
client: &mut pdpb::pd_client::PdClient<Channel>,
) -> GrpcResult<Self::Response>;
async fn rpc(req: Request<Self>, client: &mut Self::Client) -> GrpcResult<Self::Response>;
async fn send(
self,
client: &mut pdpb::pd_client::PdClient<Channel>,
timeout: Duration,
) -> Result<Self::Response> {
async fn send(self, client: &mut Self::Client, timeout: Duration) -> Result<Self::Response> {
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<Channel>;
type Response = pdpb::GetRegionResponse;
async fn rpc(
req: Request<Self>,
client: &mut pdpb::pd_client::PdClient<Channel>,
) -> GrpcResult<Self::Response> {
async fn rpc(req: Request<Self>, client: &mut Self::Client) -> GrpcResult<Self::Response> {
Ok(client.get_region(req).await?.into_inner())
}
}
#[async_trait]
impl PdMessage for pdpb::GetRegionByIdRequest {
type Client = pdpb::pd_client::PdClient<Channel>;
type Response = pdpb::GetRegionResponse;
async fn rpc(
req: Request<Self>,
client: &mut pdpb::pd_client::PdClient<Channel>,
) -> GrpcResult<Self::Response> {
async fn rpc(req: Request<Self>, client: &mut Self::Client) -> GrpcResult<Self::Response> {
Ok(client.get_region_by_id(req).await?.into_inner())
}
}
#[async_trait]
impl PdMessage for pdpb::GetStoreRequest {
type Client = pdpb::pd_client::PdClient<Channel>;
type Response = pdpb::GetStoreResponse;
async fn rpc(
req: Request<Self>,
client: &mut pdpb::pd_client::PdClient<Channel>,
) -> GrpcResult<Self::Response> {
async fn rpc(req: Request<Self>, client: &mut Self::Client) -> GrpcResult<Self::Response> {
Ok(client.get_store(req).await?.into_inner())
}
}
#[async_trait]
impl PdMessage for pdpb::GetAllStoresRequest {
type Client = pdpb::pd_client::PdClient<Channel>;
type Response = pdpb::GetAllStoresResponse;
async fn rpc(
req: Request<Self>,
client: &mut pdpb::pd_client::PdClient<Channel>,
) -> GrpcResult<Self::Response> {
async fn rpc(req: Request<Self>, client: &mut Self::Client) -> GrpcResult<Self::Response> {
Ok(client.get_all_stores(req).await?.into_inner())
}
}
#[async_trait]
impl PdMessage for pdpb::UpdateGcSafePointRequest {
type Client = pdpb::pd_client::PdClient<Channel>;
type Response = pdpb::UpdateGcSafePointResponse;
async fn rpc(
req: Request<Self>,
client: &mut pdpb::pd_client::PdClient<Channel>,
) -> GrpcResult<Self::Response> {
async fn rpc(req: Request<Self>, client: &mut Self::Client) -> GrpcResult<Self::Response> {
Ok(client.update_gc_safe_point(req).await?.into_inner())
}
}
#[async_trait]
impl PdMessage for keyspacepb::LoadKeyspaceRequest {
type Client = keyspacepb::keyspace_client::KeyspaceClient<Channel>;
type Response = keyspacepb::LoadKeyspaceResponse;
async fn rpc(req: Request<Self>, client: &mut Self::Client) -> GrpcResult<Self::Response> {
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()
}
}

View File

@ -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<Self>) -> Result<Timestamp>;
async fn update_safepoint(self: Arc<Self>, safepoint: u64) -> Result<bool>;
async fn load_keyspace(&self, keyspace: &str) -> Result<keyspacepb::KeyspaceMeta>;
}
/// Client for communication with a PD cluster. Has the facility to reconnect to the cluster.
pub struct RetryClient<Cl = Cluster> {
@ -197,6 +200,12 @@ impl RetryClientTrait for RetryClient<Cluster> {
.map(|resp| resp.new_safe_point == safepoint)
})
}
async fn load_keyspace(&self, keyspace: &str) -> Result<keyspacepb::KeyspaceMeta> {
retry_mut!(self, "load_keyspace", |cluster| async {
cluster.load_keyspace(keyspace, self.timeout).await
})
}
}
impl fmt::Debug for RetryClient {

View File

@ -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<Cod = ApiV1RawCodec, PdC = PdRpcClient<Cod>>
where
Cod: Codec,
PdC: PdClient<Codec = Cod>,
{
pub struct Client<PdC: PdClient = PdRpcClient> {
rpc: Arc<PdC>,
cf: Option<ColumnFamily>,
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<ApiV1RawCodec, PdRpcClient<ApiV1RawCodec>> {
impl Client<PdRpcClient> {
/// 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<ApiV1RawCodec, PdRpcClient<ApiV1RawCodec>> {
pd_endpoints: Vec<S>,
config: Config,
) -> Result<Self> {
let enable_codec = config.keyspace.is_some();
let pd_endpoints: Vec<String> = 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<ApiV1RawCodec, PdRpcClient<ApiV1RawCodec>> {
cf: Some(cf),
backoff: self.backoff.clone(),
atomic: self.atomic,
keyspace: self.keyspace,
}
}
}
impl<Cod: Codec> Client<Cod, PdRpcClient<Cod>> {
/// 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<Cod: Codec> Client<Cod, PdRpcClient<Cod>> {
cf: self.cf.clone(),
backoff,
atomic: self.atomic,
keyspace: self.keyspace,
}
}
@ -195,11 +204,12 @@ impl<Cod: Codec> Client<Cod, PdRpcClient<Cod>> {
cf: self.cf.clone(),
backoff: self.backoff.clone(),
atomic: true,
keyspace: self.keyspace,
}
}
}
impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
impl<PdC: PdClient> Client<PdC> {
/// 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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
/// ```
pub async fn get(&self, key: impl Into<Key>) -> Result<Option<Value>> {
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
keys: impl IntoIterator<Item = impl Into<Key>>,
) -> Result<Vec<KvPair>> {
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<Value> = req.await.unwrap();
/// # });
pub async fn get_key_ttl_secs(&self, key: impl Into<Key>) -> Result<Option<u64>> {
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
/// # });
/// ```
pub async fn put(&self, key: impl Into<Key>, value: impl Into<Value>) -> Result<()> {
self.put_with_ttl(key, value, 0).await
}
pub async fn put_with_ttl(
&self,
key: impl Into<Key>,
value: impl Into<Value>,
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
pub async fn batch_put(
&self,
pairs: impl IntoIterator<Item = impl Into<KvPair>>,
) -> Result<()> {
self.batch_put_with_ttl(pairs, std::iter::repeat(0)).await
}
pub async fn batch_put_with_ttl(
&self,
pairs: impl IntoIterator<Item = impl Into<KvPair>>,
ttls: impl IntoIterator<Item = u64>,
) -> 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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
/// ```
pub async fn delete(&self, key: impl Into<Key>) -> 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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
pub async fn batch_delete(&self, keys: impl IntoIterator<Item = impl Into<Key>>) -> 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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
pub async fn delete_range(&self, range: impl Into<BoundRange>) -> 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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
/// ```
pub async fn scan(&self, range: impl Into<BoundRange>, limit: u32) -> Result<Vec<KvPair>> {
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<KvPair> = req.await.unwrap();
/// # });
/// ```
pub async fn scan_reverse(
&self,
range: impl Into<BoundRange>,
limit: u32,
) -> Result<Vec<KvPair>> {
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
pub async fn scan_keys(&self, range: impl Into<BoundRange>, limit: u32) -> Result<Vec<Key>> {
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<Key> = req.await.unwrap();
/// # });
/// ```
pub async fn scan_keys_reverse(
&self,
range: impl Into<BoundRange>,
limit: u32,
) -> Result<Vec<Key>> {
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
) -> Result<(Option<Value>, 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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
copr_version_req: impl Into<String>,
ranges: impl IntoIterator<Item = impl Into<BoundRange>>,
request_builder: impl Fn(metapb::Region, Vec<Range<Key>>) -> Vec<u8> + Send + Sync + 'static,
) -> Result<Vec<(Vec<u8>, Vec<Range<Key>>)>> {
) -> Result<Vec<(Vec<Range<Key>>, Vec<u8>)>> {
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<Range<Key>>| {
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
range: impl Into<BoundRange>,
limit: u32,
key_only: bool,
reverse: bool,
) -> Result<Vec<KvPair>> {
if limit > MAX_RAW_KV_SCAN_LIMIT {
return Err(Error::MaxScanLimitExceeded {
@ -611,8 +755,9 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
.collect::<Vec<KvPair>>();
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
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<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
});
}
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::<kvrpcpb::RawBatchPutRequest>().is_some() {
let resp = kvrpcpb::RawBatchPutResponse {
..Default::default()
};
Ok(Box::new(resp) as Box<dyn Any>)
} 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![(
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![])]
)
]
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(())
}

View File

@ -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<ColumnFamily>,
) -> kvrpcpb::RawGetKeyTtlRequest {
requests::new_raw_get_key_ttl_request(key.into(), cf)
}
pub fn new_raw_put_request(
key: Key,
value: Value,
cf: Option<ColumnFamily>,
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<Item = KvPair>,
ttls: impl Iterator<Item = u64>,
cf: Option<ColumnFamily>,
atomic: bool,
) -> kvrpcpb::RawBatchPutRequest {
requests::new_raw_batch_put_request(pairs.map(Into::into).collect(), cf, atomic)
let pairs = pairs.map(Into::into).collect::<Vec<_>>();
let ttls = ttls.take(pairs.len()).collect::<Vec<_>>();
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<ColumnFamily>,
) -> 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,
)
}

View File

@ -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<u8>, cf: Option<ColumnFamily>) -> 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<u8> {
@ -104,15 +103,54 @@ impl Merge<kvrpcpb::RawBatchGetResponse> for Collect {
}
}
pub fn new_raw_get_key_ttl_request(
key: Vec<u8>,
cf: Option<ColumnFamily>,
) -> 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<u8> {
&self.key
}
}
impl Process<kvrpcpb::RawGetKeyTtlResponse> for DefaultProcessor {
type Out = Option<u64>;
fn process(&self, input: Result<kvrpcpb::RawGetKeyTtlResponse>) -> Result<Self::Out> {
let input = input?;
Ok(if input.not_found {
None
} else {
Some(input.ttl)
})
}
}
pub fn new_raw_put_request(
key: Vec<u8>,
value: Vec<u8>,
ttl: u64,
cf: Option<ColumnFamily>,
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<u8> {
&self.key
@ -133,11 +171,13 @@ impl SingleKey for kvrpcpb::RawPutRequest {
pub fn new_raw_batch_put_request(
pairs: Vec<kvrpcpb::KvPair>,
ttls: Vec<u64>,
cf: Option<ColumnFamily>,
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<kvrpcpb::KvPair>;
type Shard = Vec<(kvrpcpb::KvPair, u64)>;
fn shards(
&self,
pd_client: &Arc<impl PdClient>,
) -> 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::<KvPair>::into),
pd_client.clone(),
)
let kvs = self.pairs.clone();
let ttls = self.ttls.clone();
let mut kv_ttl: Vec<KvPairTTL> = 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<u8> {
&self.key
@ -237,13 +282,20 @@ pub fn new_raw_scan_request(
end_key: Vec<u8>,
limit: u32,
key_only: bool,
reverse: bool,
cf: Option<ColumnFamily>,
) -> kvrpcpb::RawScanRequest {
let mut req = kvrpcpb::RawScanRequest::default();
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<kvrpcpb::RawScanResponse> 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<u8> {
&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<Vec<Result<ResponseWithShard<kvrpcpb::RawCoprocessorResponse, Vec<kvrpcpb::KeyRange>>>>>
for DefaultProcessor
{
type Out = Vec<(Vec<u8>, Vec<Range<Key>>)>;
type Out = Vec<(Vec<Range<Key>>, Vec<u8>)>;
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<kvrpcpb::KvPair> = [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<Mutex<HashMap<Vec<KvPair>, Vec<u64>>>> =
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::<Vec<KvPair>>();
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<dyn Any>)
},
)));
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(())
}
}

View File

@ -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<kvrpcpb::Context> {
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()
}

View File

@ -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<Self>, _safepoint: u64) -> Result<bool> {
todo!()
}
async fn load_keyspace(&self, _keyspace: &str) -> Result<keyspacepb::KeyspaceMeta> {
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));
}
}

View File

@ -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<R: KvRequest>(&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<R: KvRequest>(&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<Req: KvRequest> {
pub inner: Req,
}
impl<Req: KvRequest> EncodedRequest<Req> {
pub fn new<C: Codec>(mut req: Req, codec: &C) -> Self {
codec.encode_request(&mut req);
Self { inner: req }
}
}

280
src/request/keyspace.rs Normal file
View File

@ -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::<KEYSPACE_PREFIX_LEN>(&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<Key> {
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<Range<Key>> {
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<KvPair> {
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<crate::proto::kvrpcpb::LockInfo> {
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<const N: usize>(vec: &mut Vec<u8>, 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<const N: usize>(vec: &mut Vec<u8>) {
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);
}
}

View File

@ -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<dyn Any>),
)));
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)
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();

View File

@ -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<P: Plan, PdC: PdClient> {
pub inner: P,
pub pd_client: Arc<PdC>,
pub backoff: Backoff,
pub keyspace: Keyspace,
}
impl<P: Plan, PdC: PdClient> Clone for ResolveLock<P, PdC> {
@ -554,6 +557,7 @@ impl<P: Plan, PdC: PdClient> Clone for ResolveLock<P, PdC> {
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<errorpb::Error>,
pub key_error: Option<Vec<Error>>,
@ -644,6 +648,7 @@ pub struct CleanupLocks<P: Plan, PdC: PdClient> {
pub options: ResolveLocksOptions,
pub store: Option<RegionStore>,
pub pd_client: Arc<PdC>,
pub keyspace: Keyspace,
pub backoff: Backoff,
}
@ -655,6 +660,7 @@ impl<P: Plan, PdC: PdClient> Clone for CleanupLocks<P, PdC> {
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(),

View File

@ -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<PdC: PdClient, Req: KvRequest> PlanBuilder<PdC, Dispatch<Req>, NoTarget> {
pub fn new(pd_client: Arc<PdC>, encoded_request: EncodedRequest<Req>) -> Self {
pub fn new(pd_client: Arc<PdC>, 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<PdC: PdClient, P: Plan> PlanBuilder<PdC, P, Targetted> {
impl<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> PlanBuilder<PdC, P, Ph> {
/// If there is a lock error, then resolve the lock and retry the request.
pub fn resolve_lock(self, backoff: Backoff) -> PlanBuilder<PdC, ResolveLock<P, PdC>, Ph>
pub fn resolve_lock(
self,
backoff: Backoff,
keyspace: Keyspace,
) -> PlanBuilder<PdC, ResolveLock<P, PdC>, Ph>
where
P::Result: HasLocks,
{
@ -79,6 +84,7 @@ impl<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> PlanBuilder<PdC, P, Ph> {
inner: self.plan,
backoff,
pd_client: self.pd_client,
keyspace,
},
phantom: PhantomData,
}
@ -89,6 +95,7 @@ impl<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> PlanBuilder<PdC, P, Ph> {
ctx: ResolveLocksContext,
options: ResolveLocksOptions,
backoff: Backoff,
keyspace: Keyspace,
) -> PlanBuilder<PdC, CleanupLocks<P, PdC>, Ph>
where
P: Shardable + NextBatch,
@ -103,6 +110,7 @@ impl<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> PlanBuilder<PdC, P, Ph> {
store: None,
backoff,
pd_client: self.pd_client,
keyspace,
},
phantom: PhantomData,
}
@ -248,8 +256,7 @@ fn set_single_region_store<PdC: PdClient, R: KvRequest>(
store: RegionStore,
pd_client: Arc<PdC>,
) -> Result<PlanBuilder<PdC, Dispatch<R>, 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,

View File

@ -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);
}

View File

@ -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);

View File

@ -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<dyn KvClient + Send + Sync>,
}
#[async_trait]
pub trait KvConnectStore: KvConnect {
async fn connect_to_store(
&self,
region: RegionWithLeader,
address: String,
) -> Result<RegionStore> {
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<K, KOut, PdC>(
key_data: impl Iterator<Item = K> + Send + Sync + 'static,

View File

@ -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<Box<dyn Any>>;
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");

View File

@ -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<Key>,
@ -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(),
};

View File

@ -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<Cod: Codec = ApiV1TxnCodec> {
pd: Arc<PdRpcClient<Cod>>,
pub struct Client {
pd: Arc<PdRpcClient>,
keyspace: Keyspace,
}
impl<Cod: Codec> Clone for Client<Cod> {
impl Clone for Client {
fn clone(&self) -> Self {
Self {
pd: self.pd.clone(),
keyspace: self.keyspace,
}
}
}
impl Client<ApiV1TxnCodec> {
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<ApiV1TxnCodec> {
/// # });
/// ```
pub async fn new<S: Into<String>>(pd_endpoints: Vec<S>) -> Result<Client> {
// debug!("creating transactional client");
Self::new_with_config(pd_endpoints, Config::default()).await
}
@ -101,36 +106,19 @@ impl Client<ApiV1TxnCodec> {
pd_endpoints: Vec<S>,
config: Config,
) -> Result<Client> {
Self::new_with_codec(pd_endpoints, config, ApiV1TxnCodec::default()).await
}
}
impl Client<ApiV2TxnCodec> {
pub async fn new_with_config_v2<S: Into<String>>(
_keyspace_name: &str,
pd_endpoints: Vec<S>,
config: Config,
) -> Result<Client<ApiV2TxnCodec>> {
debug!("creating new transactional client APIv2");
let pd_endpoints: Vec<String> = 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<Cod: Codec> Client<Cod> {
pub async fn new_with_codec<S: Into<String>>(
pd_endpoints: Vec<S>,
config: Config,
codec: Cod,
) -> Result<Client<Cod>> {
debug!("creating new transactional client");
let pd_endpoints: Vec<String> = pd_endpoints.into_iter().map(Into::into).collect();
let pd =
Arc::new(PdRpcClient::<Cod>::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<Cod: Codec> Client<Cod> {
/// transaction.commit().await.unwrap();
/// # });
/// ```
pub async fn begin_optimistic(&self) -> Result<Transaction<Cod, PdRpcClient<Cod>>> {
pub async fn begin_optimistic(&self) -> Result<Transaction> {
debug!("creating new optimistic transaction");
let timestamp = self.current_timestamp().await?;
Ok(self.new_transaction(timestamp, TransactionOptions::new_optimistic()))
@ -176,7 +164,7 @@ impl<Cod: Codec> Client<Cod> {
/// transaction.commit().await.unwrap();
/// # });
/// ```
pub async fn begin_pessimistic(&self) -> Result<Transaction<Cod, PdRpcClient<Cod>>> {
pub async fn begin_pessimistic(&self) -> Result<Transaction> {
debug!("creating new pessimistic transaction");
let timestamp = self.current_timestamp().await?;
Ok(self.new_transaction(timestamp, TransactionOptions::new_pessimistic()))
@ -199,21 +187,14 @@ impl<Cod: Codec> Client<Cod> {
/// transaction.commit().await.unwrap();
/// # });
/// ```
pub async fn begin_with_options(
&self,
options: TransactionOptions,
) -> Result<Transaction<Cod, PdRpcClient<Cod>>> {
pub async fn begin_with_options(&self, options: TransactionOptions) -> Result<Transaction> {
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<Cod, PdRpcClient<Cod>> {
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<Cod: Codec> Client<Cod> {
// 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<Cod: Codec> Client<Cod> {
range: impl Into<BoundRange>,
batch_size: u32,
) -> Result<Vec<crate::proto::kvrpcpb::LockInfo>> {
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<Cod: Codec> Client<Cod> {
///
/// 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<BoundRange>) -> 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<Cod, PdRpcClient<Cod>> {
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)
}
}

View File

@ -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<kvrpcpb::LockInfo>,
pd_client: Arc<impl PdClient>,
keyspace: Keyspace,
) -> Result<Vec<kvrpcpb::LockInfo> /* 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<impl PdClient>,
keyspace: Keyspace,
) -> Result<RegionVerId> {
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<kvrpcpb::LockInfo>,
pd_client: Arc<impl PdClient>, // 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<impl PdClient>,
keyspace: Keyspace,
txn_id: u64,
primary: Vec<u8>,
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<impl PdClient>,
keyspace: Keyspace,
keys: Vec<Vec<u8>>,
txn_id: u64,
) -> Result<SecondaryLocksStatus> {
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<impl PdClient>,
keyspace: Keyspace,
store: RegionStore,
txn_infos: Vec<TxnInfo>,
) -> Result<RegionVerId> {
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");
}

View File

@ -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;

Some files were not shown because too many files have changed in this diff Show More