mirror of https://github.com/tikv/client-rust.git
implement the complete keyspace feature (#439)
Signed-off-by: Andy Lok <andylokandy@hotmail.com>
This commit is contained in:
parent
bbaf317b05
commit
1178d79ed6
|
@ -38,9 +38,12 @@ prometheus = { version = "0.13", default-features = false }
|
||||||
prost = "0.12"
|
prost = "0.12"
|
||||||
rand = "0.8"
|
rand = "0.8"
|
||||||
regex = "1"
|
regex = "1"
|
||||||
|
reqwest = { version = "0.11", features = ["json", "native-tls-vendored"] }
|
||||||
semver = "1.0"
|
semver = "1.0"
|
||||||
serde = "1.0"
|
serde = "1.0"
|
||||||
serde_derive = "1.0"
|
serde_derive = "1.0"
|
||||||
|
serde_json = "1"
|
||||||
|
take_mut = "0.2.2"
|
||||||
thiserror = "1"
|
thiserror = "1"
|
||||||
tokio = { version = "1", features = ["sync", "rt-multi-thread", "macros"] }
|
tokio = { version = "1", features = ["sync", "rt-multi-thread", "macros"] }
|
||||||
tonic = { version = "0.10", features = ["tls"] }
|
tonic = { version = "0.10", features = ["tls"] }
|
||||||
|
@ -51,9 +54,7 @@ env_logger = "0.10"
|
||||||
fail = { version = "0.4", features = ["failpoints"] }
|
fail = { version = "0.4", features = ["failpoints"] }
|
||||||
proptest = "1"
|
proptest = "1"
|
||||||
proptest-derive = "0.3"
|
proptest-derive = "0.3"
|
||||||
reqwest = { version = "0.11", default-features = false, features = [
|
rstest = "0.18.2"
|
||||||
"native-tls-vendored",
|
|
||||||
] }
|
|
||||||
serde_json = "1"
|
serde_json = "1"
|
||||||
serial_test = "0.5.0"
|
serial_test = "0.5.0"
|
||||||
simple_logger = "1"
|
simple_logger = "1"
|
||||||
|
|
|
@ -15,3 +15,7 @@ max-open-files = 10000
|
||||||
|
|
||||||
[raftdb]
|
[raftdb]
|
||||||
max-open-files = 10000
|
max-open-files = 10000
|
||||||
|
|
||||||
|
[storage]
|
||||||
|
api-version = 2
|
||||||
|
enable-ttl = true
|
||||||
|
|
|
@ -24,7 +24,9 @@ async fn main() {
|
||||||
Config::default().with_security(ca, cert, key)
|
Config::default().with_security(ca, cert, key)
|
||||||
} else {
|
} else {
|
||||||
Config::default()
|
Config::default()
|
||||||
};
|
}
|
||||||
|
// This example uses the default keyspace, so api-v2 must be enabled on the server.
|
||||||
|
.with_default_keyspace();
|
||||||
|
|
||||||
// init
|
// init
|
||||||
let client = Client::new_with_config(args.pd, config)
|
let client = Client::new_with_config(args.pd, config)
|
||||||
|
|
|
@ -31,7 +31,9 @@ async fn main() -> Result<()> {
|
||||||
Config::default().with_security(ca, cert, key)
|
Config::default().with_security(ca, cert, key)
|
||||||
} else {
|
} else {
|
||||||
Config::default()
|
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
|
// When we first create a client we receive a `Connect` structure which must be resolved before
|
||||||
// the client is actually connected and usable.
|
// 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:?}");
|
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(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
|
@ -87,7 +87,9 @@ async fn main() {
|
||||||
Config::default().with_security(ca, cert, key)
|
Config::default().with_security(ca, cert, key)
|
||||||
} else {
|
} else {
|
||||||
Config::default()
|
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)
|
let txn = Client::new_with_config(args.pd, config)
|
||||||
.await
|
.await
|
||||||
|
|
|
@ -50,9 +50,14 @@ pub enum Error {
|
||||||
/// Wraps a `grpcio::Error`.
|
/// Wraps a `grpcio::Error`.
|
||||||
#[error("gRPC error: {0}")]
|
#[error("gRPC error: {0}")]
|
||||||
Grpc(#[from] tonic::transport::Error),
|
Grpc(#[from] tonic::transport::Error),
|
||||||
|
/// Wraps a `reqwest::Error`.
|
||||||
|
#[error("http error: {0}")]
|
||||||
|
Http(#[from] reqwest::Error),
|
||||||
/// Wraps a `grpcio::Error`.
|
/// Wraps a `grpcio::Error`.
|
||||||
#[error("gRPC api error: {0}")]
|
#[error("gRPC api error: {0}")]
|
||||||
GrpcAPI(#[from] tonic::Status),
|
GrpcAPI(#[from] tonic::Status),
|
||||||
|
#[error("Http request failed: unknown respond {0}")]
|
||||||
|
UnknownHttpRespond(String),
|
||||||
/// Wraps a `grpcio::Error`.
|
/// Wraps a `grpcio::Error`.
|
||||||
#[error("url error: {0}")]
|
#[error("url error: {0}")]
|
||||||
Url(#[from] tonic::codegen::http::uri::InvalidUri),
|
Url(#[from] tonic::codegen::http::uri::InvalidUri),
|
||||||
|
|
|
@ -19,6 +19,7 @@ pub struct Config {
|
||||||
pub cert_path: Option<PathBuf>,
|
pub cert_path: Option<PathBuf>,
|
||||||
pub key_path: Option<PathBuf>,
|
pub key_path: Option<PathBuf>,
|
||||||
pub timeout: Duration,
|
pub timeout: Duration,
|
||||||
|
pub keyspace: Option<String>,
|
||||||
}
|
}
|
||||||
|
|
||||||
const DEFAULT_REQUEST_TIMEOUT: Duration = Duration::from_secs(2);
|
const DEFAULT_REQUEST_TIMEOUT: Duration = Duration::from_secs(2);
|
||||||
|
@ -30,6 +31,7 @@ impl Default for Config {
|
||||||
cert_path: None,
|
cert_path: None,
|
||||||
key_path: None,
|
key_path: None,
|
||||||
timeout: DEFAULT_REQUEST_TIMEOUT,
|
timeout: DEFAULT_REQUEST_TIMEOUT,
|
||||||
|
keyspace: None,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -83,4 +85,21 @@ impl Config {
|
||||||
self.timeout = timeout;
|
self.timeout = timeout;
|
||||||
self
|
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
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -136,17 +136,11 @@ impl BoundRange {
|
||||||
pub fn into_keys(self) -> (Key, Option<Key>) {
|
pub fn into_keys(self) -> (Key, Option<Key>) {
|
||||||
let start = match self.from {
|
let start = match self.from {
|
||||||
Bound::Included(v) => v,
|
Bound::Included(v) => v,
|
||||||
Bound::Excluded(mut v) => {
|
Bound::Excluded(v) => v.next_key(),
|
||||||
v.push_zero();
|
|
||||||
v
|
|
||||||
}
|
|
||||||
Bound::Unbounded => Key::EMPTY,
|
Bound::Unbounded => Key::EMPTY,
|
||||||
};
|
};
|
||||||
let end = match self.to {
|
let end = match self.to {
|
||||||
Bound::Included(mut v) => {
|
Bound::Included(v) => Some(v.next_key()),
|
||||||
v.push_zero();
|
|
||||||
Some(v)
|
|
||||||
}
|
|
||||||
Bound::Excluded(v) => Some(v),
|
Bound::Excluded(v) => Some(v),
|
||||||
Bound::Unbounded => None,
|
Bound::Unbounded => None,
|
||||||
};
|
};
|
||||||
|
|
|
@ -71,7 +71,7 @@ pub struct Key(
|
||||||
test,
|
test,
|
||||||
proptest(strategy = "any_with::<Vec<u8>>((size_range(_PROPTEST_KEY_MAX), ()))")
|
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 {
|
impl AsRef<Key> for kvrpcpb::Mutation {
|
||||||
|
@ -98,10 +98,11 @@ impl Key {
|
||||||
|
|
||||||
/// Push a zero to the end of the 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]
|
#[inline]
|
||||||
pub(super) fn push_zero(&mut self) {
|
pub(crate) fn next_key(mut self) -> Self {
|
||||||
self.0.push(0)
|
self.0.push(0);
|
||||||
|
self
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Convert the key to a lower bound. The key is treated as inclusive.
|
/// Convert the key to a lower bound. The key is treated as inclusive.
|
||||||
|
|
|
@ -94,8 +94,6 @@
|
||||||
|
|
||||||
pub mod backoff;
|
pub mod backoff;
|
||||||
#[doc(hidden)]
|
#[doc(hidden)]
|
||||||
pub mod proto; // export `proto` to enable user customized codec
|
|
||||||
#[doc(hidden)]
|
|
||||||
pub mod raw;
|
pub mod raw;
|
||||||
pub mod request;
|
pub mod request;
|
||||||
#[doc(hidden)]
|
#[doc(hidden)]
|
||||||
|
@ -106,6 +104,7 @@ mod compat;
|
||||||
mod config;
|
mod config;
|
||||||
mod kv;
|
mod kv;
|
||||||
mod pd;
|
mod pd;
|
||||||
|
mod proto;
|
||||||
mod region;
|
mod region;
|
||||||
mod region_cache;
|
mod region_cache;
|
||||||
mod stats;
|
mod stats;
|
||||||
|
@ -146,8 +145,6 @@ pub use crate::raw::Client as RawClient;
|
||||||
#[doc(inline)]
|
#[doc(inline)]
|
||||||
pub use crate::raw::ColumnFamily;
|
pub use crate::raw::ColumnFamily;
|
||||||
#[doc(inline)]
|
#[doc(inline)]
|
||||||
pub use crate::request::codec;
|
|
||||||
#[doc(inline)]
|
|
||||||
pub use crate::request::RetryOptions;
|
pub use crate::request::RetryOptions;
|
||||||
#[doc(inline)]
|
#[doc(inline)]
|
||||||
pub use crate::timestamp::Timestamp;
|
pub use crate::timestamp::Timestamp;
|
||||||
|
|
21
src/mock.rs
21
src/mock.rs
|
@ -18,7 +18,6 @@ use crate::proto::metapb::RegionEpoch;
|
||||||
use crate::proto::metapb::{self};
|
use crate::proto::metapb::{self};
|
||||||
use crate::region::RegionId;
|
use crate::region::RegionId;
|
||||||
use crate::region::RegionWithLeader;
|
use crate::region::RegionWithLeader;
|
||||||
use crate::request::codec::ApiV1TxnCodec;
|
|
||||||
use crate::store::KvConnect;
|
use crate::store::KvConnect;
|
||||||
use crate::store::RegionStore;
|
use crate::store::RegionStore;
|
||||||
use crate::store::Request;
|
use crate::store::Request;
|
||||||
|
@ -31,7 +30,7 @@ use crate::Timestamp;
|
||||||
|
|
||||||
/// Create a `PdRpcClient` with it's internals replaced with mocks so that the
|
/// Create a `PdRpcClient` with it's internals replaced with mocks so that the
|
||||||
/// client can be tested without doing any RPC calls.
|
/// 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();
|
let config = Config::default();
|
||||||
PdRpcClient::new(
|
PdRpcClient::new(
|
||||||
config.clone(),
|
config.clone(),
|
||||||
|
@ -44,7 +43,6 @@ pub async fn pd_rpc_client() -> PdRpcClient<ApiV1TxnCodec, MockKvConnect, MockCl
|
||||||
))
|
))
|
||||||
},
|
},
|
||||||
false,
|
false,
|
||||||
Some(ApiV1TxnCodec::default()),
|
|
||||||
)
|
)
|
||||||
.await
|
.await
|
||||||
.unwrap()
|
.unwrap()
|
||||||
|
@ -73,18 +71,9 @@ pub struct MockKvConnect;
|
||||||
|
|
||||||
pub struct MockCluster;
|
pub struct MockCluster;
|
||||||
|
|
||||||
|
#[derive(new)]
|
||||||
pub struct MockPdClient {
|
pub struct MockPdClient {
|
||||||
client: MockKvClient,
|
client: MockKvClient,
|
||||||
codec: ApiV1TxnCodec,
|
|
||||||
}
|
|
||||||
|
|
||||||
impl MockPdClient {
|
|
||||||
pub fn new(client: MockKvClient) -> MockPdClient {
|
|
||||||
MockPdClient {
|
|
||||||
client,
|
|
||||||
codec: ApiV1TxnCodec::default(),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#[async_trait]
|
#[async_trait]
|
||||||
|
@ -113,7 +102,6 @@ impl MockPdClient {
|
||||||
pub fn default() -> MockPdClient {
|
pub fn default() -> MockPdClient {
|
||||||
MockPdClient {
|
MockPdClient {
|
||||||
client: MockKvClient::default(),
|
client: MockKvClient::default(),
|
||||||
codec: ApiV1TxnCodec::default(),
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -177,7 +165,6 @@ impl MockPdClient {
|
||||||
|
|
||||||
#[async_trait]
|
#[async_trait]
|
||||||
impl PdClient for MockPdClient {
|
impl PdClient for MockPdClient {
|
||||||
type Codec = ApiV1TxnCodec;
|
|
||||||
type KvClient = MockKvClient;
|
type KvClient = MockKvClient;
|
||||||
|
|
||||||
async fn map_region_to_store(self: Arc<Self>, region: RegionWithLeader) -> Result<RegionStore> {
|
async fn map_region_to_store(self: Arc<Self>, region: RegionWithLeader) -> Result<RegionStore> {
|
||||||
|
@ -228,7 +215,7 @@ impl PdClient for MockPdClient {
|
||||||
|
|
||||||
async fn invalidate_region_cache(&self, _ver_id: crate::region::RegionVerId) {}
|
async fn invalidate_region_cache(&self, _ver_id: crate::region::RegionVerId) {}
|
||||||
|
|
||||||
fn get_codec(&self) -> &Self::Codec {
|
async fn get_keyspace_id(&self, _keyspace: &str) -> Result<u32> {
|
||||||
&self.codec
|
unimplemented!()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@ use crate::region::RegionId;
|
||||||
use crate::region::RegionVerId;
|
use crate::region::RegionVerId;
|
||||||
use crate::region::RegionWithLeader;
|
use crate::region::RegionWithLeader;
|
||||||
use crate::region_cache::RegionCache;
|
use crate::region_cache::RegionCache;
|
||||||
use crate::request::codec::{ApiV1TxnCodec, Codec};
|
|
||||||
use crate::store::KvConnect;
|
use crate::store::KvConnect;
|
||||||
use crate::store::RegionStore;
|
use crate::store::RegionStore;
|
||||||
use crate::store::TikvConnect;
|
use crate::store::TikvConnect;
|
||||||
|
@ -51,7 +50,6 @@ use crate::Timestamp;
|
||||||
/// So if we use transactional APIs, keys in PD are encoded and PD does not know about the encoding stuff.
|
/// So if we use transactional APIs, keys in PD are encoded and PD does not know about the encoding stuff.
|
||||||
#[async_trait]
|
#[async_trait]
|
||||||
pub trait PdClient: Send + Sync + 'static {
|
pub trait PdClient: Send + Sync + 'static {
|
||||||
type Codec: Codec;
|
|
||||||
type KvClient: KvClient + Send + Sync + 'static;
|
type KvClient: KvClient + Send + Sync + 'static;
|
||||||
|
|
||||||
/// In transactional API, `region` is decoded (keys in raw format).
|
/// In transactional API, `region` is decoded (keys in raw format).
|
||||||
|
@ -67,6 +65,8 @@ pub trait PdClient: Send + Sync + 'static {
|
||||||
|
|
||||||
async fn update_safepoint(self: Arc<Self>, safepoint: u64) -> Result<bool>;
|
async fn update_safepoint(self: Arc<Self>, safepoint: u64) -> Result<bool>;
|
||||||
|
|
||||||
|
async fn get_keyspace_id(&self, keyspace: &str) -> Result<u32>;
|
||||||
|
|
||||||
/// In transactional API, `key` is in raw format
|
/// In transactional API, `key` is in raw format
|
||||||
async fn store_for_key(self: Arc<Self>, key: &Key) -> Result<RegionStore> {
|
async fn store_for_key(self: Arc<Self>, key: &Key) -> Result<RegionStore> {
|
||||||
let region = self.region_for_key(key).await?;
|
let region = self.region_for_key(key).await?;
|
||||||
|
@ -193,11 +193,8 @@ pub trait PdClient: Send + Sync + 'static {
|
||||||
.boxed()
|
.boxed()
|
||||||
}
|
}
|
||||||
|
|
||||||
fn decode_region(
|
fn decode_region(mut region: RegionWithLeader, enable_codec: bool) -> Result<RegionWithLeader> {
|
||||||
mut region: RegionWithLeader,
|
if enable_codec {
|
||||||
enable_mvcc_codec: bool,
|
|
||||||
) -> Result<RegionWithLeader> {
|
|
||||||
if enable_mvcc_codec {
|
|
||||||
codec::decode_bytes_in_place(&mut region.region.start_key, false)?;
|
codec::decode_bytes_in_place(&mut region.region.start_key, false)?;
|
||||||
codec::decode_bytes_in_place(&mut region.region.end_key, false)?;
|
codec::decode_bytes_in_place(&mut region.region.end_key, false)?;
|
||||||
}
|
}
|
||||||
|
@ -207,30 +204,20 @@ pub trait PdClient: Send + Sync + 'static {
|
||||||
async fn update_leader(&self, ver_id: RegionVerId, leader: metapb::Peer) -> Result<()>;
|
async fn update_leader(&self, ver_id: RegionVerId, leader: metapb::Peer) -> Result<()>;
|
||||||
|
|
||||||
async fn invalidate_region_cache(&self, ver_id: RegionVerId);
|
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
|
/// This client converts requests for the logical TiKV cluster into requests
|
||||||
/// for a single TiKV store using PD and internal logic.
|
/// for a single TiKV store using PD and internal logic.
|
||||||
pub struct PdRpcClient<
|
pub struct PdRpcClient<KvC: KvConnect + Send + Sync + 'static = TikvConnect, Cl = Cluster> {
|
||||||
Cod: Codec = ApiV1TxnCodec,
|
|
||||||
KvC: KvConnect + Send + Sync + 'static = TikvConnect,
|
|
||||||
Cl = Cluster,
|
|
||||||
> {
|
|
||||||
pd: Arc<RetryClient<Cl>>,
|
pd: Arc<RetryClient<Cl>>,
|
||||||
kv_connect: KvC,
|
kv_connect: KvC,
|
||||||
kv_client_cache: Arc<RwLock<HashMap<String, KvC::KvClient>>>,
|
kv_client_cache: Arc<RwLock<HashMap<String, KvC::KvClient>>>,
|
||||||
enable_mvcc_codec: bool,
|
enable_codec: bool,
|
||||||
region_cache: RegionCache<RetryClient<Cl>>,
|
region_cache: RegionCache<RetryClient<Cl>>,
|
||||||
codec: Option<Cod>,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#[async_trait]
|
#[async_trait]
|
||||||
impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static> PdClient for PdRpcClient<Cod, KvC> {
|
impl<KvC: KvConnect + Send + Sync + 'static> PdClient for PdRpcClient<KvC> {
|
||||||
type Codec = Cod;
|
|
||||||
type KvClient = KvC::KvClient;
|
type KvClient = KvC::KvClient;
|
||||||
|
|
||||||
async fn map_region_to_store(self: Arc<Self>, region: RegionWithLeader) -> Result<RegionStore> {
|
async fn map_region_to_store(self: Arc<Self>, region: RegionWithLeader) -> Result<RegionStore> {
|
||||||
|
@ -241,20 +228,20 @@ impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static> PdClient for PdRpcClien
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn region_for_key(&self, key: &Key) -> Result<RegionWithLeader> {
|
async fn region_for_key(&self, key: &Key) -> Result<RegionWithLeader> {
|
||||||
let enable_mvcc_codec = self.enable_mvcc_codec;
|
let enable_codec = self.enable_codec;
|
||||||
let key = if enable_mvcc_codec {
|
let key = if enable_codec {
|
||||||
key.to_encoded()
|
key.to_encoded()
|
||||||
} else {
|
} else {
|
||||||
key.clone()
|
key.clone()
|
||||||
};
|
};
|
||||||
|
|
||||||
let region = self.region_cache.get_region_by_key(&key).await?;
|
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> {
|
async fn region_for_id(&self, id: RegionId) -> Result<RegionWithLeader> {
|
||||||
let region = self.region_cache.get_region_by_id(id).await?;
|
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>> {
|
async fn all_stores(&self) -> Result<Vec<Store>> {
|
||||||
|
@ -283,39 +270,34 @@ impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static> PdClient for PdRpcClien
|
||||||
self.region_cache.invalidate_region_cache(ver_id).await
|
self.region_cache.invalidate_region_cache(ver_id).await
|
||||||
}
|
}
|
||||||
|
|
||||||
fn get_codec(&self) -> &Self::Codec {
|
async fn get_keyspace_id(&self, keyspace: &str) -> Result<u32> {
|
||||||
self.codec
|
self.pd.get_keyspace_id(keyspace).await
|
||||||
.as_ref()
|
|
||||||
.unwrap_or_else(|| panic!("codec not set"))
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<Cod: Codec> PdRpcClient<Cod, TikvConnect, Cluster> {
|
impl PdRpcClient<TikvConnect, Cluster> {
|
||||||
pub async fn connect(
|
pub async fn connect(
|
||||||
pd_endpoints: &[String],
|
pd_endpoints: &[String],
|
||||||
config: Config,
|
config: Config,
|
||||||
enable_mvcc_codec: bool, // TODO: infer from `codec`.
|
enable_codec: bool,
|
||||||
codec: Option<Cod>,
|
) -> Result<PdRpcClient> {
|
||||||
) -> Result<PdRpcClient<Cod>> {
|
|
||||||
PdRpcClient::new(
|
PdRpcClient::new(
|
||||||
config.clone(),
|
config.clone(),
|
||||||
|security_mgr| TikvConnect::new(security_mgr, config.timeout),
|
|security_mgr| TikvConnect::new(security_mgr, config.timeout),
|
||||||
|security_mgr| RetryClient::connect(pd_endpoints, security_mgr, config.timeout),
|
|security_mgr| RetryClient::connect(pd_endpoints, security_mgr, config.timeout),
|
||||||
enable_mvcc_codec,
|
enable_codec,
|
||||||
codec,
|
|
||||||
)
|
)
|
||||||
.await
|
.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>(
|
pub async fn new<PdFut, MakeKvC, MakePd>(
|
||||||
config: Config,
|
config: Config,
|
||||||
kv_connect: MakeKvC,
|
kv_connect: MakeKvC,
|
||||||
pd: MakePd,
|
pd: MakePd,
|
||||||
enable_mvcc_codec: bool,
|
enable_codec: bool,
|
||||||
codec: Option<Cod>,
|
) -> Result<PdRpcClient<KvC, Cl>>
|
||||||
) -> Result<PdRpcClient<Cod, KvC, Cl>>
|
|
||||||
where
|
where
|
||||||
PdFut: Future<Output = Result<RetryClient<Cl>>>,
|
PdFut: Future<Output = Result<RetryClient<Cl>>>,
|
||||||
MakeKvC: FnOnce(Arc<SecurityManager>) -> KvC,
|
MakeKvC: FnOnce(Arc<SecurityManager>) -> KvC,
|
||||||
|
@ -337,9 +319,8 @@ impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static, Cl> PdRpcClient<Cod, Kv
|
||||||
pd: pd.clone(),
|
pd: pd.clone(),
|
||||||
kv_client_cache,
|
kv_client_cache,
|
||||||
kv_connect: kv_connect(security_mgr),
|
kv_connect: kv_connect(security_mgr),
|
||||||
enable_mvcc_codec,
|
enable_codec,
|
||||||
region_cache: RegionCache::new(pd),
|
region_cache: RegionCache::new(pd),
|
||||||
codec,
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -359,10 +340,6 @@ impl<Cod: Codec, KvC: KvConnect + Send + Sync + 'static, Cl> PdRpcClient<Cod, Kv
|
||||||
Err(e) => Err(e),
|
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 {
|
fn make_key_range(start_key: Vec<u8>, end_key: Vec<u8>) -> kvrpcpb::KeyRange {
|
||||||
|
|
|
@ -16,6 +16,7 @@ use tonic::Request;
|
||||||
use super::timestamp::TimestampOracle;
|
use super::timestamp::TimestampOracle;
|
||||||
use crate::internal_err;
|
use crate::internal_err;
|
||||||
use crate::proto::pdpb;
|
use crate::proto::pdpb;
|
||||||
|
use crate::Error;
|
||||||
use crate::Result;
|
use crate::Result;
|
||||||
use crate::SecurityManager;
|
use crate::SecurityManager;
|
||||||
use crate::Timestamp;
|
use crate::Timestamp;
|
||||||
|
@ -24,6 +25,7 @@ use crate::Timestamp;
|
||||||
pub struct Cluster {
|
pub struct Cluster {
|
||||||
id: u64,
|
id: u64,
|
||||||
client: pdpb::pd_client::PdClient<Channel>,
|
client: pdpb::pd_client::PdClient<Channel>,
|
||||||
|
endpoint: String,
|
||||||
members: pdpb::GetMembersResponse,
|
members: pdpb::GetMembersResponse,
|
||||||
tso: TimestampOracle,
|
tso: TimestampOracle,
|
||||||
}
|
}
|
||||||
|
@ -91,6 +93,18 @@ impl Cluster {
|
||||||
req.safe_point = safepoint;
|
req.safe_point = safepoint;
|
||||||
req.send(&mut self.client, timeout).await
|
req.send(&mut self.client, timeout).await
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub async fn get_keyspace_id(&self, keyspace: &str) -> Result<u32> {
|
||||||
|
let resp =
|
||||||
|
reqwest::get(format!("{}/pd/api/v2/keyspaces/{keyspace}", self.endpoint)).await?;
|
||||||
|
let body = resp.json::<serde_json::Value>().await?;
|
||||||
|
let keyspace_id = body
|
||||||
|
.get("id")
|
||||||
|
.ok_or_else(|| Error::UnknownHttpRespond(body.to_string()))?
|
||||||
|
.as_u64()
|
||||||
|
.ok_or_else(|| Error::UnknownHttpRespond(body.to_string()))?;
|
||||||
|
Ok(keyspace_id as u32)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// An object for connecting and reconnecting to a PD cluster.
|
/// An object for connecting and reconnecting to a PD cluster.
|
||||||
|
@ -109,12 +123,13 @@ impl Connection {
|
||||||
timeout: Duration,
|
timeout: Duration,
|
||||||
) -> Result<Cluster> {
|
) -> Result<Cluster> {
|
||||||
let members = self.validate_endpoints(endpoints, timeout).await?;
|
let members = self.validate_endpoints(endpoints, timeout).await?;
|
||||||
let (client, members) = self.try_connect_leader(&members, timeout).await?;
|
let (client, endpoint, members) = self.try_connect_leader(&members, timeout).await?;
|
||||||
let id = members.header.as_ref().unwrap().cluster_id;
|
let id = members.header.as_ref().unwrap().cluster_id;
|
||||||
let tso = TimestampOracle::new(id, &client)?;
|
let tso = TimestampOracle::new(id, &client)?;
|
||||||
let cluster = Cluster {
|
let cluster = Cluster {
|
||||||
id,
|
id,
|
||||||
client,
|
client,
|
||||||
|
endpoint,
|
||||||
members,
|
members,
|
||||||
tso,
|
tso,
|
||||||
};
|
};
|
||||||
|
@ -125,11 +140,13 @@ impl Connection {
|
||||||
pub async fn reconnect(&self, cluster: &mut Cluster, timeout: Duration) -> Result<()> {
|
pub async fn reconnect(&self, cluster: &mut Cluster, timeout: Duration) -> Result<()> {
|
||||||
warn!("updating pd client");
|
warn!("updating pd client");
|
||||||
let start = Instant::now();
|
let start = Instant::now();
|
||||||
let (client, members) = self.try_connect_leader(&cluster.members, timeout).await?;
|
let (client, endpoint, members) =
|
||||||
|
self.try_connect_leader(&cluster.members, timeout).await?;
|
||||||
let tso = TimestampOracle::new(cluster.id, &client)?;
|
let tso = TimestampOracle::new(cluster.id, &client)?;
|
||||||
*cluster = Cluster {
|
*cluster = Cluster {
|
||||||
id: cluster.id,
|
id: cluster.id,
|
||||||
client,
|
client,
|
||||||
|
endpoint,
|
||||||
members,
|
members,
|
||||||
tso,
|
tso,
|
||||||
};
|
};
|
||||||
|
@ -239,7 +256,11 @@ impl Connection {
|
||||||
&self,
|
&self,
|
||||||
previous: &pdpb::GetMembersResponse,
|
previous: &pdpb::GetMembersResponse,
|
||||||
timeout: Duration,
|
timeout: Duration,
|
||||||
) -> Result<(pdpb::pd_client::PdClient<Channel>, pdpb::GetMembersResponse)> {
|
) -> Result<(
|
||||||
|
pdpb::pd_client::PdClient<Channel>,
|
||||||
|
String,
|
||||||
|
pdpb::GetMembersResponse,
|
||||||
|
)> {
|
||||||
let previous_leader = previous.leader.as_ref().unwrap();
|
let previous_leader = previous.leader.as_ref().unwrap();
|
||||||
let members = &previous.members;
|
let members = &previous.members;
|
||||||
let cluster_id = previous.header.as_ref().unwrap().cluster_id;
|
let cluster_id = previous.header.as_ref().unwrap().cluster_id;
|
||||||
|
@ -269,9 +290,10 @@ impl Connection {
|
||||||
if let Some(resp) = resp {
|
if let Some(resp) = resp {
|
||||||
let leader = resp.leader.as_ref().unwrap();
|
let leader = resp.leader.as_ref().unwrap();
|
||||||
for ep in &leader.client_urls {
|
for ep in &leader.client_urls {
|
||||||
let r = self.try_connect(ep.as_str(), cluster_id, timeout).await;
|
if let Ok((client, members)) =
|
||||||
if r.is_ok() {
|
self.try_connect(ep.as_str(), cluster_id, timeout).await
|
||||||
return r;
|
{
|
||||||
|
return Ok((client, ep.to_string(), members));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,6 +45,8 @@ pub trait RetryClientTrait {
|
||||||
async fn get_timestamp(self: Arc<Self>) -> Result<Timestamp>;
|
async fn get_timestamp(self: Arc<Self>) -> Result<Timestamp>;
|
||||||
|
|
||||||
async fn update_safepoint(self: Arc<Self>, safepoint: u64) -> Result<bool>;
|
async fn update_safepoint(self: Arc<Self>, safepoint: u64) -> Result<bool>;
|
||||||
|
|
||||||
|
async fn get_keyspace_id(&self, keyspace: &str) -> Result<u32>;
|
||||||
}
|
}
|
||||||
/// Client for communication with a PD cluster. Has the facility to reconnect to the cluster.
|
/// Client for communication with a PD cluster. Has the facility to reconnect to the cluster.
|
||||||
pub struct RetryClient<Cl = Cluster> {
|
pub struct RetryClient<Cl = Cluster> {
|
||||||
|
@ -197,6 +199,12 @@ impl RetryClientTrait for RetryClient<Cluster> {
|
||||||
.map(|resp| resp.new_safe_point == safepoint)
|
.map(|resp| resp.new_safe_point == safepoint)
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
async fn get_keyspace_id(&self, keyspace: &str) -> Result<u32> {
|
||||||
|
retry!(self, "get_keyspace_id", |cluster| async {
|
||||||
|
cluster.get_keyspace_id(keyspace).await
|
||||||
|
})
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl fmt::Debug for RetryClient {
|
impl fmt::Debug for RetryClient {
|
||||||
|
|
|
@ -15,10 +15,13 @@ use crate::pd::PdClient;
|
||||||
use crate::pd::PdRpcClient;
|
use crate::pd::PdRpcClient;
|
||||||
use crate::proto::metapb;
|
use crate::proto::metapb;
|
||||||
use crate::raw::lowering::*;
|
use crate::raw::lowering::*;
|
||||||
use crate::request::codec::{ApiV1RawCodec, Codec, EncodedRequest};
|
|
||||||
use crate::request::Collect;
|
use crate::request::Collect;
|
||||||
use crate::request::CollectSingle;
|
use crate::request::CollectSingle;
|
||||||
|
use crate::request::EncodeKeyspace;
|
||||||
|
use crate::request::KeyMode;
|
||||||
|
use crate::request::Keyspace;
|
||||||
use crate::request::Plan;
|
use crate::request::Plan;
|
||||||
|
use crate::request::TruncateKeyspace;
|
||||||
use crate::Backoff;
|
use crate::Backoff;
|
||||||
use crate::BoundRange;
|
use crate::BoundRange;
|
||||||
use crate::ColumnFamily;
|
use crate::ColumnFamily;
|
||||||
|
@ -36,16 +39,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
|
/// The returned results of raw request methods are [`Future`](std::future::Future)s that must be
|
||||||
/// awaited to execute.
|
/// awaited to execute.
|
||||||
pub struct Client<Cod = ApiV1RawCodec, PdC = PdRpcClient<Cod>>
|
pub struct Client<PdC: PdClient = PdRpcClient> {
|
||||||
where
|
|
||||||
Cod: Codec,
|
|
||||||
PdC: PdClient<Codec = Cod>,
|
|
||||||
{
|
|
||||||
rpc: Arc<PdC>,
|
rpc: Arc<PdC>,
|
||||||
cf: Option<ColumnFamily>,
|
cf: Option<ColumnFamily>,
|
||||||
backoff: Backoff,
|
backoff: Backoff,
|
||||||
/// Whether to use the [`atomic mode`](Client::with_atomic_for_cas).
|
/// Whether to use the [`atomic mode`](Client::with_atomic_for_cas).
|
||||||
atomic: bool,
|
atomic: bool,
|
||||||
|
keyspace: Keyspace,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Clone for Client {
|
impl Clone for Client {
|
||||||
|
@ -55,11 +55,12 @@ impl Clone for Client {
|
||||||
cf: self.cf.clone(),
|
cf: self.cf.clone(),
|
||||||
backoff: self.backoff.clone(),
|
backoff: self.backoff.clone(),
|
||||||
atomic: self.atomic,
|
atomic: self.atomic,
|
||||||
|
keyspace: self.keyspace,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Client<ApiV1RawCodec, PdRpcClient<ApiV1RawCodec>> {
|
impl Client<PdRpcClient> {
|
||||||
/// Create a raw [`Client`] and connect to the TiKV cluster.
|
/// 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
|
/// Because TiKV is managed by a [PD](https://github.com/pingcap/pd/) cluster, the endpoints for
|
||||||
|
@ -104,16 +105,23 @@ impl Client<ApiV1RawCodec, PdRpcClient<ApiV1RawCodec>> {
|
||||||
pd_endpoints: Vec<S>,
|
pd_endpoints: Vec<S>,
|
||||||
config: Config,
|
config: Config,
|
||||||
) -> Result<Self> {
|
) -> Result<Self> {
|
||||||
|
let enable_codec = config.keyspace.is_some();
|
||||||
let pd_endpoints: Vec<String> = pd_endpoints.into_iter().map(Into::into).collect();
|
let pd_endpoints: Vec<String> = pd_endpoints.into_iter().map(Into::into).collect();
|
||||||
let rpc = Arc::new(
|
let rpc =
|
||||||
PdRpcClient::connect(&pd_endpoints, config, false, Some(ApiV1RawCodec::default()))
|
Arc::new(PdRpcClient::connect(&pd_endpoints, config.clone(), enable_codec).await?);
|
||||||
.await?,
|
let keyspace = match config.keyspace {
|
||||||
);
|
Some(keyspace) => {
|
||||||
|
let keyspace_id = rpc.get_keyspace_id(&keyspace).await?;
|
||||||
|
Keyspace::Enable { keyspace_id }
|
||||||
|
}
|
||||||
|
None => Keyspace::Disable,
|
||||||
|
};
|
||||||
Ok(Client {
|
Ok(Client {
|
||||||
rpc,
|
rpc,
|
||||||
cf: None,
|
cf: None,
|
||||||
backoff: DEFAULT_REGION_BACKOFF,
|
backoff: DEFAULT_REGION_BACKOFF,
|
||||||
atomic: false,
|
atomic: false,
|
||||||
|
keyspace,
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -148,11 +156,10 @@ impl Client<ApiV1RawCodec, PdRpcClient<ApiV1RawCodec>> {
|
||||||
cf: Some(cf),
|
cf: Some(cf),
|
||||||
backoff: self.backoff.clone(),
|
backoff: self.backoff.clone(),
|
||||||
atomic: self.atomic,
|
atomic: self.atomic,
|
||||||
}
|
keyspace: self.keyspace,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<Cod: Codec> Client<Cod, PdRpcClient<Cod>> {
|
|
||||||
/// Set the [`Backoff`] strategy for retrying requests.
|
/// Set the [`Backoff`] strategy for retrying requests.
|
||||||
/// The default strategy is [`DEFAULT_REGION_BACKOFF`](crate::backoff::DEFAULT_REGION_BACKOFF).
|
/// The default strategy is [`DEFAULT_REGION_BACKOFF`](crate::backoff::DEFAULT_REGION_BACKOFF).
|
||||||
/// See [`Backoff`] for more information.
|
/// See [`Backoff`] for more information.
|
||||||
|
@ -178,6 +185,7 @@ impl<Cod: Codec> Client<Cod, PdRpcClient<Cod>> {
|
||||||
cf: self.cf.clone(),
|
cf: self.cf.clone(),
|
||||||
backoff,
|
backoff,
|
||||||
atomic: self.atomic,
|
atomic: self.atomic,
|
||||||
|
keyspace: self.keyspace,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -195,11 +203,12 @@ impl<Cod: Codec> Client<Cod, PdRpcClient<Cod>> {
|
||||||
cf: self.cf.clone(),
|
cf: self.cf.clone(),
|
||||||
backoff: self.backoff.clone(),
|
backoff: self.backoff.clone(),
|
||||||
atomic: true,
|
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.
|
/// Create a new 'get' request.
|
||||||
///
|
///
|
||||||
/// Once resolved this request will result in the fetching of the value associated with the
|
/// Once resolved this request will result in the fetching of the value associated with the
|
||||||
|
@ -220,9 +229,9 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
/// ```
|
/// ```
|
||||||
pub async fn get(&self, key: impl Into<Key>) -> Result<Option<Value>> {
|
pub async fn get(&self, key: impl Into<Key>) -> Result<Option<Value>> {
|
||||||
debug!("invoking raw get request");
|
debug!("invoking raw get request");
|
||||||
let request = new_raw_get_request(key.into(), self.cf.clone());
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw);
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
let request = new_raw_get_request(key, self.cf.clone());
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request)
|
||||||
.retry_multi_region(self.backoff.clone())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.post_process_default()
|
.post_process_default()
|
||||||
|
@ -253,15 +262,19 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
keys: impl IntoIterator<Item = impl Into<Key>>,
|
keys: impl IntoIterator<Item = impl Into<Key>>,
|
||||||
) -> Result<Vec<KvPair>> {
|
) -> Result<Vec<KvPair>> {
|
||||||
debug!("invoking raw batch_get request");
|
debug!("invoking raw batch_get request");
|
||||||
let request = new_raw_batch_get_request(keys.into_iter().map(Into::into), self.cf.clone());
|
let keys = keys
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
.into_iter()
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
.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())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.merge(Collect)
|
.merge(Collect)
|
||||||
.plan();
|
.plan();
|
||||||
plan.execute()
|
plan.execute().await.map(|r| {
|
||||||
.await
|
r.into_iter()
|
||||||
.map(|r| r.into_iter().map(Into::into).collect())
|
.map(|pair| pair.truncate_keyspace(self.keyspace))
|
||||||
|
.collect()
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Create a new 'put' request.
|
/// Create a new 'put' request.
|
||||||
|
@ -282,9 +295,9 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
/// ```
|
/// ```
|
||||||
pub async fn put(&self, key: impl Into<Key>, value: impl Into<Value>) -> Result<()> {
|
pub async fn put(&self, key: impl Into<Key>, value: impl Into<Value>) -> Result<()> {
|
||||||
debug!("invoking raw put request");
|
debug!("invoking raw put request");
|
||||||
let request = new_raw_put_request(key.into(), value.into(), self.cf.clone(), self.atomic);
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw);
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
let request = new_raw_put_request(key, value.into(), self.cf.clone(), self.atomic);
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request)
|
||||||
.retry_multi_region(self.backoff.clone())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
|
@ -315,13 +328,11 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
pairs: impl IntoIterator<Item = impl Into<KvPair>>,
|
pairs: impl IntoIterator<Item = impl Into<KvPair>>,
|
||||||
) -> Result<()> {
|
) -> Result<()> {
|
||||||
debug!("invoking raw batch_put request");
|
debug!("invoking raw batch_put request");
|
||||||
let request = new_raw_batch_put_request(
|
let pairs = pairs
|
||||||
pairs.into_iter().map(Into::into),
|
.into_iter()
|
||||||
self.cf.clone(),
|
.map(|pair| pair.into().encode_keyspace(self.keyspace, KeyMode::Raw));
|
||||||
self.atomic,
|
let request = new_raw_batch_put_request(pairs, self.cf.clone(), self.atomic);
|
||||||
);
|
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request)
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
|
||||||
.retry_multi_region(self.backoff.clone())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -348,9 +359,9 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
/// ```
|
/// ```
|
||||||
pub async fn delete(&self, key: impl Into<Key>) -> Result<()> {
|
pub async fn delete(&self, key: impl Into<Key>) -> Result<()> {
|
||||||
debug!("invoking raw delete request");
|
debug!("invoking raw delete request");
|
||||||
let request = new_raw_delete_request(key.into(), self.cf.clone(), self.atomic);
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw);
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
let request = new_raw_delete_request(key, self.cf.clone(), self.atomic);
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request)
|
||||||
.retry_multi_region(self.backoff.clone())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
|
@ -379,10 +390,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<()> {
|
pub async fn batch_delete(&self, keys: impl IntoIterator<Item = impl Into<Key>>) -> Result<()> {
|
||||||
debug!("invoking raw batch_delete request");
|
debug!("invoking raw batch_delete request");
|
||||||
self.assert_non_atomic()?;
|
self.assert_non_atomic()?;
|
||||||
let request =
|
let keys = keys
|
||||||
new_raw_batch_delete_request(keys.into_iter().map(Into::into), self.cf.clone());
|
.into_iter()
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
.map(|k| k.into().encode_keyspace(self.keyspace, KeyMode::Raw));
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
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())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -408,9 +420,9 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
pub async fn delete_range(&self, range: impl Into<BoundRange>) -> Result<()> {
|
pub async fn delete_range(&self, range: impl Into<BoundRange>) -> Result<()> {
|
||||||
debug!("invoking raw delete_range request");
|
debug!("invoking raw delete_range request");
|
||||||
self.assert_non_atomic()?;
|
self.assert_non_atomic()?;
|
||||||
let request = new_raw_delete_range_request(range.into(), self.cf.clone());
|
let range = range.into().encode_keyspace(self.keyspace, KeyMode::Raw);
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
let request = new_raw_delete_range_request(range, self.cf.clone());
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, request)
|
||||||
.retry_multi_region(self.backoff.clone())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -560,14 +572,14 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
) -> Result<(Option<Value>, bool)> {
|
) -> Result<(Option<Value>, bool)> {
|
||||||
debug!("invoking raw compare_and_swap request");
|
debug!("invoking raw compare_and_swap request");
|
||||||
self.assert_atomic()?;
|
self.assert_atomic()?;
|
||||||
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Raw);
|
||||||
let req = new_cas_request(
|
let req = new_cas_request(
|
||||||
key.into(),
|
key,
|
||||||
new_value.into(),
|
new_value.into(),
|
||||||
previous_value.into(),
|
previous_value.into(),
|
||||||
self.cf.clone(),
|
self.cf.clone(),
|
||||||
);
|
);
|
||||||
let encoded_req = EncodedRequest::new(req, self.rpc.get_codec());
|
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, req)
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
|
||||||
.retry_multi_region(self.backoff.clone())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.post_process_default()
|
.post_process_default()
|
||||||
|
@ -581,22 +593,39 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
copr_version_req: impl Into<String>,
|
copr_version_req: impl Into<String>,
|
||||||
ranges: impl IntoIterator<Item = impl Into<BoundRange>>,
|
ranges: impl IntoIterator<Item = impl Into<BoundRange>>,
|
||||||
request_builder: impl Fn(metapb::Region, Vec<Range<Key>>) -> Vec<u8> + Send + Sync + 'static,
|
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();
|
let copr_version_req = copr_version_req.into();
|
||||||
semver::VersionReq::from_str(&copr_version_req)?;
|
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(
|
let req = new_raw_coprocessor_request(
|
||||||
copr_name.into(),
|
copr_name.into(),
|
||||||
copr_version_req,
|
copr_version_req,
|
||||||
ranges.into_iter().map(Into::into),
|
ranges,
|
||||||
request_builder,
|
request_builder,
|
||||||
);
|
);
|
||||||
let encoded_req = EncodedRequest::new(req, self.rpc.get_codec());
|
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), self.keyspace, req)
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
|
||||||
.preserve_shard()
|
.preserve_shard()
|
||||||
.retry_multi_region(self.backoff.clone())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.post_process_default()
|
.post_process_default()
|
||||||
.plan();
|
.plan();
|
||||||
plan.execute().await
|
Ok(plan
|
||||||
|
.execute()
|
||||||
|
.await?
|
||||||
|
.into_iter()
|
||||||
|
.map(|(ranges, data)| (ranges.truncate_keyspace(keyspace), data))
|
||||||
|
.collect())
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn scan_inner(
|
async fn scan_inner(
|
||||||
|
@ -611,8 +640,9 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
max_limit: MAX_RAW_KV_SCAN_LIMIT,
|
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 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 scan_regions = self.rpc.clone().stores_for_range(cur_range.clone()).boxed();
|
||||||
let mut region_store =
|
let mut region_store =
|
||||||
scan_regions
|
scan_regions
|
||||||
|
@ -622,11 +652,11 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
range: (cur_range.clone()),
|
range: (cur_range.clone()),
|
||||||
})??;
|
})??;
|
||||||
let mut cur_limit = limit;
|
let mut cur_limit = limit;
|
||||||
|
|
||||||
while cur_limit > 0 {
|
while cur_limit > 0 {
|
||||||
let request =
|
let request =
|
||||||
new_raw_scan_request(cur_range.clone(), cur_limit, key_only, self.cf.clone());
|
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(), self.keyspace, request)
|
||||||
let resp = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
|
||||||
.single_region_with_store(region_store.clone())
|
.single_region_with_store(region_store.clone())
|
||||||
.await?
|
.await?
|
||||||
.plan()
|
.plan()
|
||||||
|
@ -639,6 +669,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
.collect::<Vec<KvPair>>();
|
.collect::<Vec<KvPair>>();
|
||||||
let res_len = region_scan_res.len();
|
let res_len = region_scan_res.len();
|
||||||
result.append(&mut region_scan_res);
|
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 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 {
|
if res_len < cur_limit as usize {
|
||||||
region_store = match scan_regions.next().await {
|
region_store = match scan_regions.next().await {
|
||||||
|
@ -650,15 +681,20 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
rs
|
rs
|
||||||
}
|
}
|
||||||
Some(Err(e)) => return Err(e),
|
Some(Err(e)) => return Err(e),
|
||||||
None => return Ok(result),
|
None => break,
|
||||||
};
|
};
|
||||||
cur_limit -= res_len as u32;
|
cur_limit -= res_len as u32;
|
||||||
} else {
|
} else {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// limit is a soft limit, so we need check the number of results
|
// limit is a soft limit, so we need check the number of results
|
||||||
result.truncate(limit as usize);
|
result.truncate(limit as usize);
|
||||||
|
|
||||||
|
// truncate the prefix of keys
|
||||||
|
let result = result.truncate_keyspace(self.keyspace);
|
||||||
|
|
||||||
Ok(result)
|
Ok(result)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -675,18 +711,20 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Client<Cod, PdC> {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
let request = new_raw_batch_scan_request(
|
let ranges = ranges
|
||||||
ranges.into_iter().map(Into::into),
|
.into_iter()
|
||||||
each_limit,
|
.map(|range| range.into().encode_keyspace(self.keyspace, KeyMode::Raw));
|
||||||
key_only,
|
|
||||||
self.cf.clone(),
|
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)
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
|
||||||
let plan = crate::request::PlanBuilder::new(self.rpc.clone(), encoded_req)
|
|
||||||
.retry_multi_region(self.backoff.clone())
|
.retry_multi_region(self.backoff.clone())
|
||||||
.merge(Collect)
|
.merge(Collect)
|
||||||
.plan();
|
.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<()> {
|
fn assert_non_atomic(&self) -> Result<()> {
|
||||||
|
@ -739,6 +777,7 @@ mod tests {
|
||||||
cf: Some(ColumnFamily::Default),
|
cf: Some(ColumnFamily::Default),
|
||||||
backoff: DEFAULT_REGION_BACKOFF,
|
backoff: DEFAULT_REGION_BACKOFF,
|
||||||
atomic: false,
|
atomic: false,
|
||||||
|
keyspace: Keyspace::Enable { keyspace_id: 0 },
|
||||||
};
|
};
|
||||||
let resps = client
|
let resps = client
|
||||||
.coprocessor(
|
.coprocessor(
|
||||||
|
@ -750,27 +789,17 @@ mod tests {
|
||||||
.await?;
|
.await?;
|
||||||
let resps: Vec<_> = resps
|
let resps: Vec<_> = resps
|
||||||
.into_iter()
|
.into_iter()
|
||||||
.map(|(data, ranges)| (String::from_utf8(data).unwrap(), ranges))
|
.map(|(ranges, data)| (ranges, String::from_utf8(data).unwrap()))
|
||||||
.collect();
|
.collect();
|
||||||
assert_eq!(
|
assert_eq!(
|
||||||
resps,
|
resps,
|
||||||
|
vec![(
|
||||||
vec![
|
vec![
|
||||||
(
|
Key::from(vec![5])..Key::from(vec![15]),
|
||||||
"1:[Key(05)..Key(0A)]".to_string(),
|
Key::from(vec![20])..Key::from(vec![])
|
||||||
vec![Key::from(vec![5])..Key::from(vec![10])]
|
],
|
||||||
),
|
"2:[Key(05)..Key(0F), Key(14)..Key()]".to_string(),
|
||||||
(
|
),]
|
||||||
"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![])]
|
|
||||||
)
|
|
||||||
]
|
|
||||||
);
|
);
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
|
@ -13,10 +13,10 @@ use super::RawRpcRequest;
|
||||||
use crate::collect_first;
|
use crate::collect_first;
|
||||||
use crate::pd::PdClient;
|
use crate::pd::PdClient;
|
||||||
use crate::proto::kvrpcpb;
|
use crate::proto::kvrpcpb;
|
||||||
use crate::proto::kvrpcpb::ApiVersion;
|
|
||||||
use crate::proto::metapb;
|
use crate::proto::metapb;
|
||||||
use crate::proto::tikvpb::tikv_client::TikvClient;
|
use crate::proto::tikvpb::tikv_client::TikvClient;
|
||||||
use crate::range_request;
|
use crate::range_request;
|
||||||
|
use crate::region::RegionWithLeader;
|
||||||
use crate::request::plan::ResponseWithShard;
|
use crate::request::plan::ResponseWithShard;
|
||||||
use crate::request::Collect;
|
use crate::request::Collect;
|
||||||
use crate::request::CollectSingle;
|
use crate::request::CollectSingle;
|
||||||
|
@ -164,7 +164,7 @@ impl Shardable for kvrpcpb::RawBatchPutRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
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.pairs = shard;
|
self.pairs = shard;
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
@ -297,7 +297,7 @@ impl Shardable for kvrpcpb::RawBatchScanRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
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;
|
self.ranges = shard;
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
@ -399,11 +399,11 @@ impl Request for RawCoprocessorRequest {
|
||||||
self.inner.as_any()
|
self.inner.as_any()
|
||||||
}
|
}
|
||||||
|
|
||||||
fn set_context(&mut self, context: kvrpcpb::Context) {
|
fn set_leader(&mut self, leader: &RegionWithLeader) -> Result<()> {
|
||||||
self.inner.set_context(context);
|
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);
|
self.inner.set_api_version(api_version);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -423,7 +423,7 @@ impl Shardable for RawCoprocessorRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
||||||
self.inner.context = Some(store.region_with_leader.context()?);
|
self.set_leader(&store.region_with_leader)?;
|
||||||
self.inner.ranges = shard.clone();
|
self.inner.ranges = shard.clone();
|
||||||
self.inner.data = (self.data_builder)(store.region_with_leader.region.clone(), shard);
|
self.inner.data = (self.data_builder)(store.region_with_leader.region.clone(), shard);
|
||||||
Ok(())
|
Ok(())
|
||||||
|
@ -435,7 +435,7 @@ impl
|
||||||
Process<Vec<Result<ResponseWithShard<kvrpcpb::RawCoprocessorResponse, Vec<kvrpcpb::KeyRange>>>>>
|
Process<Vec<Result<ResponseWithShard<kvrpcpb::RawCoprocessorResponse, Vec<kvrpcpb::KeyRange>>>>>
|
||||||
for DefaultProcessor
|
for DefaultProcessor
|
||||||
{
|
{
|
||||||
type Out = Vec<(Vec<u8>, Vec<Range<Key>>)>;
|
type Out = Vec<(Vec<Range<Key>>, Vec<u8>)>;
|
||||||
|
|
||||||
fn process(
|
fn process(
|
||||||
&self,
|
&self,
|
||||||
|
@ -448,11 +448,11 @@ impl
|
||||||
.map(|shard_resp| {
|
.map(|shard_resp| {
|
||||||
shard_resp.map(|ResponseWithShard(resp, ranges)| {
|
shard_resp.map(|ResponseWithShard(resp, ranges)| {
|
||||||
(
|
(
|
||||||
resp.data,
|
|
||||||
ranges
|
ranges
|
||||||
.into_iter()
|
.into_iter()
|
||||||
.map(|range| range.start_key.into()..range.end_key.into())
|
.map(|range| range.start_key.into()..range.end_key.into())
|
||||||
.collect(),
|
.collect(),
|
||||||
|
resp.data,
|
||||||
)
|
)
|
||||||
})
|
})
|
||||||
})
|
})
|
||||||
|
@ -497,21 +497,21 @@ impl HasLocks for kvrpcpb::RawCoprocessorResponse {}
|
||||||
mod test {
|
mod test {
|
||||||
use std::any::Any;
|
use std::any::Any;
|
||||||
|
|
||||||
use futures::executor;
|
|
||||||
|
|
||||||
use super::*;
|
use super::*;
|
||||||
use crate::backoff::DEFAULT_REGION_BACKOFF;
|
use crate::backoff::DEFAULT_REGION_BACKOFF;
|
||||||
use crate::backoff::OPTIMISTIC_BACKOFF;
|
use crate::backoff::OPTIMISTIC_BACKOFF;
|
||||||
use crate::mock::MockKvClient;
|
use crate::mock::MockKvClient;
|
||||||
use crate::mock::MockPdClient;
|
use crate::mock::MockPdClient;
|
||||||
use crate::proto::kvrpcpb;
|
use crate::proto::kvrpcpb;
|
||||||
use crate::request::codec::EncodedRequest;
|
use crate::request::Keyspace;
|
||||||
use crate::request::Plan;
|
use crate::request::Plan;
|
||||||
use crate::Key;
|
use crate::Key;
|
||||||
|
|
||||||
#[test]
|
#[rstest::rstest]
|
||||||
#[ignore]
|
#[case(Keyspace::Disable)]
|
||||||
fn test_raw_scan() {
|
#[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(
|
let client = Arc::new(MockPdClient::new(MockKvClient::with_dispatch_hook(
|
||||||
|req: &dyn Any| {
|
|req: &dyn Any| {
|
||||||
let req: &kvrpcpb::RawScanRequest = req.downcast_ref().unwrap();
|
let req: &kvrpcpb::RawScanRequest = req.downcast_ref().unwrap();
|
||||||
|
@ -540,15 +540,14 @@ mod test {
|
||||||
key_only: true,
|
key_only: true,
|
||||||
..Default::default()
|
..Default::default()
|
||||||
};
|
};
|
||||||
let encoded_scan = EncodedRequest::new(scan, client.get_codec());
|
let plan = crate::request::PlanBuilder::new(client, keyspace, scan)
|
||||||
let plan = crate::request::PlanBuilder::new(client, encoded_scan)
|
.resolve_lock(OPTIMISTIC_BACKOFF, keyspace)
|
||||||
.resolve_lock(OPTIMISTIC_BACKOFF)
|
|
||||||
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
||||||
.merge(Collect)
|
.merge(Collect)
|
||||||
.plan();
|
.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.
|
// FIXME test the keys returned.
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -2,7 +2,6 @@
|
||||||
|
|
||||||
use derive_new::new;
|
use derive_new::new;
|
||||||
|
|
||||||
use crate::proto::kvrpcpb;
|
|
||||||
use crate::proto::metapb;
|
use crate::proto::metapb;
|
||||||
use crate::Error;
|
use crate::Error;
|
||||||
use crate::Key;
|
use crate::Key;
|
||||||
|
@ -43,21 +42,6 @@ impl RegionWithLeader {
|
||||||
key >= start_key.as_slice() && (key < end_key.as_slice() || end_key.is_empty())
|
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 {
|
pub fn start_key(&self) -> Key {
|
||||||
self.region.start_key.to_vec().into()
|
self.region.start_key.to_vec().into()
|
||||||
}
|
}
|
||||||
|
|
|
@ -346,6 +346,10 @@ mod test {
|
||||||
async fn update_safepoint(self: Arc<Self>, _safepoint: u64) -> Result<bool> {
|
async fn update_safepoint(self: Arc<Self>, _safepoint: u64) -> Result<bool> {
|
||||||
todo!()
|
todo!()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
async fn get_keyspace_id(&self, _keyspace: &str) -> Result<u32> {
|
||||||
|
todo!()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
|
|
|
@ -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 }
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -3,6 +3,10 @@
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use derive_new::new;
|
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::Collect;
|
||||||
pub use self::plan::CollectError;
|
pub use self::plan::CollectError;
|
||||||
pub use self::plan::CollectSingle;
|
pub use self::plan::CollectSingle;
|
||||||
|
@ -33,7 +37,7 @@ use crate::store::Request;
|
||||||
use crate::store::{HasKeyErrors, Store};
|
use crate::store::{HasKeyErrors, Store};
|
||||||
use crate::transaction::HasLocks;
|
use crate::transaction::HasLocks;
|
||||||
|
|
||||||
pub mod codec;
|
mod keyspace;
|
||||||
pub mod plan;
|
pub mod plan;
|
||||||
mod plan_builder;
|
mod plan_builder;
|
||||||
mod shard;
|
mod shard;
|
||||||
|
@ -43,9 +47,6 @@ mod shard;
|
||||||
pub trait KvRequest: Request + Sized + Clone + Sync + Send + 'static {
|
pub trait KvRequest: Request + Sized + Clone + Sync + Send + 'static {
|
||||||
/// The expected response to the request.
|
/// The expected response to the request.
|
||||||
type Response: HasKeyErrors + HasLocks + Clone + Send + 'static;
|
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.
|
/// For requests or plans which are handled at TiKV store (other than region) level.
|
||||||
|
@ -98,12 +99,10 @@ mod test {
|
||||||
use super::*;
|
use super::*;
|
||||||
use crate::mock::MockKvClient;
|
use crate::mock::MockKvClient;
|
||||||
use crate::mock::MockPdClient;
|
use crate::mock::MockPdClient;
|
||||||
use crate::pd::PdClient;
|
|
||||||
use crate::proto::kvrpcpb;
|
use crate::proto::kvrpcpb;
|
||||||
use crate::proto::kvrpcpb::ApiVersion;
|
|
||||||
use crate::proto::pdpb::Timestamp;
|
use crate::proto::pdpb::Timestamp;
|
||||||
use crate::proto::tikvpb::tikv_client::TikvClient;
|
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::store_stream_for_keys;
|
||||||
use crate::store::HasRegionError;
|
use crate::store::HasRegionError;
|
||||||
use crate::transaction::lowering::new_commit_request;
|
use crate::transaction::lowering::new_commit_request;
|
||||||
|
@ -113,7 +112,7 @@ mod test {
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
async fn test_region_retry() {
|
async fn test_region_retry() {
|
||||||
#[derive(Clone)]
|
#[derive(Debug, Clone)]
|
||||||
struct MockRpcResponse;
|
struct MockRpcResponse;
|
||||||
|
|
||||||
impl HasKeyErrors for MockRpcResponse {
|
impl HasKeyErrors for MockRpcResponse {
|
||||||
|
@ -149,11 +148,11 @@ mod test {
|
||||||
self
|
self
|
||||||
}
|
}
|
||||||
|
|
||||||
fn set_context(&mut self, _: kvrpcpb::Context) {
|
fn set_leader(&mut self, _: &RegionWithLeader) -> Result<()> {
|
||||||
unreachable!();
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
fn set_api_version(&mut self, _api_version: ApiVersion) {}
|
fn set_api_version(&mut self, _: kvrpcpb::ApiVersion) {}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[async_trait]
|
#[async_trait]
|
||||||
|
@ -199,9 +198,8 @@ mod test {
|
||||||
|_: &dyn Any| Ok(Box::new(MockRpcResponse) as Box<dyn Any>),
|
|_: &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(), Keyspace::Disable, request)
|
||||||
let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req)
|
.resolve_lock(Backoff::no_jitter_backoff(1, 1, 3), Keyspace::Disable)
|
||||||
.resolve_lock(Backoff::no_jitter_backoff(1, 1, 3))
|
|
||||||
.retry_multi_region(Backoff::no_jitter_backoff(1, 1, 3))
|
.retry_multi_region(Backoff::no_jitter_backoff(1, 1, 3))
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -224,18 +222,18 @@ mod test {
|
||||||
|
|
||||||
let key: Key = "key".to_owned().into();
|
let key: Key = "key".to_owned().into();
|
||||||
let req = new_commit_request(iter::once(key), Timestamp::default(), Timestamp::default());
|
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
|
// does not extract error
|
||||||
let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req.clone())
|
let plan =
|
||||||
.resolve_lock(OPTIMISTIC_BACKOFF)
|
crate::request::PlanBuilder::new(pd_client.clone(), Keyspace::Disable, req.clone())
|
||||||
|
.resolve_lock(OPTIMISTIC_BACKOFF, Keyspace::Disable)
|
||||||
.retry_multi_region(OPTIMISTIC_BACKOFF)
|
.retry_multi_region(OPTIMISTIC_BACKOFF)
|
||||||
.plan();
|
.plan();
|
||||||
assert!(plan.execute().await.is_ok());
|
assert!(plan.execute().await.is_ok());
|
||||||
|
|
||||||
// extract error
|
// extract error
|
||||||
let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req)
|
let plan = crate::request::PlanBuilder::new(pd_client.clone(), Keyspace::Disable, req)
|
||||||
.resolve_lock(OPTIMISTIC_BACKOFF)
|
.resolve_lock(OPTIMISTIC_BACKOFF, Keyspace::Disable)
|
||||||
.retry_multi_region(OPTIMISTIC_BACKOFF)
|
.retry_multi_region(OPTIMISTIC_BACKOFF)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
|
|
@ -35,6 +35,8 @@ use crate::util::iter::FlatMapOkIterExt;
|
||||||
use crate::Error;
|
use crate::Error;
|
||||||
use crate::Result;
|
use crate::Result;
|
||||||
|
|
||||||
|
use super::keyspace::Keyspace;
|
||||||
|
|
||||||
/// A plan for how to execute a request. A user builds up a plan with various
|
/// A plan for how to execute a request. A user builds up a plan with various
|
||||||
/// options, then exectutes it.
|
/// options, then exectutes it.
|
||||||
#[async_trait]
|
#[async_trait]
|
||||||
|
@ -546,6 +548,7 @@ pub struct ResolveLock<P: Plan, PdC: PdClient> {
|
||||||
pub inner: P,
|
pub inner: P,
|
||||||
pub pd_client: Arc<PdC>,
|
pub pd_client: Arc<PdC>,
|
||||||
pub backoff: Backoff,
|
pub backoff: Backoff,
|
||||||
|
pub keyspace: Keyspace,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<P: Plan, PdC: PdClient> Clone for ResolveLock<P, PdC> {
|
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(),
|
inner: self.inner.clone(),
|
||||||
pd_client: self.pd_client.clone(),
|
pd_client: self.pd_client.clone(),
|
||||||
backoff: self.backoff.clone(),
|
backoff: self.backoff.clone(),
|
||||||
|
keyspace: self.keyspace,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -579,7 +583,7 @@ where
|
||||||
}
|
}
|
||||||
|
|
||||||
let pd_client = self.pd_client.clone();
|
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() {
|
if live_locks.is_empty() {
|
||||||
result = self.inner.execute().await?;
|
result = self.inner.execute().await?;
|
||||||
} else {
|
} else {
|
||||||
|
@ -595,7 +599,7 @@ where
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Default)]
|
#[derive(Debug, Default)]
|
||||||
pub struct CleanupLocksResult {
|
pub struct CleanupLocksResult {
|
||||||
pub region_error: Option<errorpb::Error>,
|
pub region_error: Option<errorpb::Error>,
|
||||||
pub key_error: Option<Vec<Error>>,
|
pub key_error: Option<Vec<Error>>,
|
||||||
|
@ -644,6 +648,7 @@ pub struct CleanupLocks<P: Plan, PdC: PdClient> {
|
||||||
pub options: ResolveLocksOptions,
|
pub options: ResolveLocksOptions,
|
||||||
pub store: Option<RegionStore>,
|
pub store: Option<RegionStore>,
|
||||||
pub pd_client: Arc<PdC>,
|
pub pd_client: Arc<PdC>,
|
||||||
|
pub keyspace: Keyspace,
|
||||||
pub backoff: Backoff,
|
pub backoff: Backoff,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -655,6 +660,7 @@ impl<P: Plan, PdC: PdClient> Clone for CleanupLocks<P, PdC> {
|
||||||
options: self.options,
|
options: self.options,
|
||||||
store: None,
|
store: None,
|
||||||
pd_client: self.pd_client.clone(),
|
pd_client: self.pd_client.clone(),
|
||||||
|
keyspace: self.keyspace,
|
||||||
backoff: self.backoff.clone(),
|
backoff: self.backoff.clone(),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -715,7 +721,12 @@ where
|
||||||
|
|
||||||
let lock_size = locks.len();
|
let lock_size = locks.len();
|
||||||
match lock_resolver
|
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
|
.await
|
||||||
{
|
{
|
||||||
Ok(()) => {
|
Ok(()) => {
|
||||||
|
@ -891,6 +902,7 @@ mod test {
|
||||||
inner: ErrPlan,
|
inner: ErrPlan,
|
||||||
backoff: Backoff::no_backoff(),
|
backoff: Backoff::no_backoff(),
|
||||||
pd_client: Arc::new(MockPdClient::default()),
|
pd_client: Arc::new(MockPdClient::default()),
|
||||||
|
keyspace: Keyspace::Disable,
|
||||||
},
|
},
|
||||||
pd_client: Arc::new(MockPdClient::default()),
|
pd_client: Arc::new(MockPdClient::default()),
|
||||||
backoff: Backoff::no_backoff(),
|
backoff: Backoff::no_backoff(),
|
||||||
|
|
|
@ -4,9 +4,9 @@ use std::marker::PhantomData;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
|
||||||
use super::plan::PreserveShard;
|
use super::plan::PreserveShard;
|
||||||
|
use super::Keyspace;
|
||||||
use crate::backoff::Backoff;
|
use crate::backoff::Backoff;
|
||||||
use crate::pd::PdClient;
|
use crate::pd::PdClient;
|
||||||
use crate::request::codec::EncodedRequest;
|
|
||||||
use crate::request::plan::{CleanupLocks, RetryableAllStores};
|
use crate::request::plan::{CleanupLocks, RetryableAllStores};
|
||||||
use crate::request::shard::HasNextBatch;
|
use crate::request::shard::HasNextBatch;
|
||||||
use crate::request::Dispatch;
|
use crate::request::Dispatch;
|
||||||
|
@ -47,11 +47,12 @@ pub struct Targetted;
|
||||||
impl PlanBuilderPhase for Targetted {}
|
impl PlanBuilderPhase for Targetted {}
|
||||||
|
|
||||||
impl<PdC: PdClient, Req: KvRequest> PlanBuilder<PdC, Dispatch<Req>, NoTarget> {
|
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 {
|
PlanBuilder {
|
||||||
pd_client,
|
pd_client,
|
||||||
plan: Dispatch {
|
plan: Dispatch {
|
||||||
request: encoded_request.inner,
|
request,
|
||||||
kv_client: None,
|
kv_client: None,
|
||||||
},
|
},
|
||||||
phantom: PhantomData,
|
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> {
|
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.
|
/// 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
|
where
|
||||||
P::Result: HasLocks,
|
P::Result: HasLocks,
|
||||||
{
|
{
|
||||||
|
@ -79,6 +84,7 @@ impl<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> PlanBuilder<PdC, P, Ph> {
|
||||||
inner: self.plan,
|
inner: self.plan,
|
||||||
backoff,
|
backoff,
|
||||||
pd_client: self.pd_client,
|
pd_client: self.pd_client,
|
||||||
|
keyspace,
|
||||||
},
|
},
|
||||||
phantom: PhantomData,
|
phantom: PhantomData,
|
||||||
}
|
}
|
||||||
|
@ -89,6 +95,7 @@ impl<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> PlanBuilder<PdC, P, Ph> {
|
||||||
ctx: ResolveLocksContext,
|
ctx: ResolveLocksContext,
|
||||||
options: ResolveLocksOptions,
|
options: ResolveLocksOptions,
|
||||||
backoff: Backoff,
|
backoff: Backoff,
|
||||||
|
keyspace: Keyspace,
|
||||||
) -> PlanBuilder<PdC, CleanupLocks<P, PdC>, Ph>
|
) -> PlanBuilder<PdC, CleanupLocks<P, PdC>, Ph>
|
||||||
where
|
where
|
||||||
P: Shardable + NextBatch,
|
P: Shardable + NextBatch,
|
||||||
|
@ -103,6 +110,7 @@ impl<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> PlanBuilder<PdC, P, Ph> {
|
||||||
store: None,
|
store: None,
|
||||||
backoff,
|
backoff,
|
||||||
pd_client: self.pd_client,
|
pd_client: self.pd_client,
|
||||||
|
keyspace,
|
||||||
},
|
},
|
||||||
phantom: PhantomData,
|
phantom: PhantomData,
|
||||||
}
|
}
|
||||||
|
@ -248,8 +256,7 @@ fn set_single_region_store<PdC: PdClient, R: KvRequest>(
|
||||||
store: RegionStore,
|
store: RegionStore,
|
||||||
pd_client: Arc<PdC>,
|
pd_client: Arc<PdC>,
|
||||||
) -> Result<PlanBuilder<PdC, Dispatch<R>, Targetted>> {
|
) -> Result<PlanBuilder<PdC, Dispatch<R>, Targetted>> {
|
||||||
plan.request
|
plan.request.set_leader(&store.region_with_leader)?;
|
||||||
.set_context(store.region_with_leader.context()?);
|
|
||||||
plan.kv_client = Some(store.client);
|
plan.kv_client = Some(store.client);
|
||||||
Ok(PlanBuilder {
|
Ok(PlanBuilder {
|
||||||
plan,
|
plan,
|
||||||
|
|
|
@ -14,6 +14,7 @@ use crate::request::ResolveLock;
|
||||||
use crate::store::RegionStore;
|
use crate::store::RegionStore;
|
||||||
use crate::store::Request;
|
use crate::store::Request;
|
||||||
use crate::Result;
|
use crate::Result;
|
||||||
|
use std::fmt::Debug;
|
||||||
|
|
||||||
macro_rules! impl_inner_shardable {
|
macro_rules! impl_inner_shardable {
|
||||||
() => {
|
() => {
|
||||||
|
@ -33,7 +34,7 @@ macro_rules! impl_inner_shardable {
|
||||||
}
|
}
|
||||||
|
|
||||||
pub trait Shardable {
|
pub trait Shardable {
|
||||||
type Shard: Clone + Send + Sync;
|
type Shard: Debug + Clone + Send + Sync;
|
||||||
|
|
||||||
fn shards(
|
fn shards(
|
||||||
&self,
|
&self,
|
||||||
|
@ -164,7 +165,7 @@ macro_rules! shardable_key {
|
||||||
mut shard: Self::Shard,
|
mut shard: Self::Shard,
|
||||||
store: &$crate::store::RegionStore,
|
store: &$crate::store::RegionStore,
|
||||||
) -> $crate::Result<()> {
|
) -> $crate::Result<()> {
|
||||||
self.set_context(store.region_with_leader.context()?);
|
self.set_leader(&store.region_with_leader)?;
|
||||||
assert!(shard.len() == 1);
|
assert!(shard.len() == 1);
|
||||||
self.key = shard.pop().unwrap();
|
self.key = shard.pop().unwrap();
|
||||||
Ok(())
|
Ok(())
|
||||||
|
@ -197,7 +198,7 @@ macro_rules! shardable_keys {
|
||||||
shard: Self::Shard,
|
shard: Self::Shard,
|
||||||
store: &$crate::store::RegionStore,
|
store: &$crate::store::RegionStore,
|
||||||
) -> $crate::Result<()> {
|
) -> $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();
|
self.keys = shard.into_iter().map(Into::into).collect();
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
@ -257,12 +258,12 @@ macro_rules! shardable_range {
|
||||||
shard: Self::Shard,
|
shard: Self::Shard,
|
||||||
store: &$crate::store::RegionStore,
|
store: &$crate::store::RegionStore,
|
||||||
) -> $crate::Result<()> {
|
) -> $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.
|
// 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.
|
// 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.start_key = shard.0;
|
||||||
self.end_key = shard.1.into();
|
self.end_key = shard.1;
|
||||||
if self.is_reverse() {
|
if self.is_reverse() {
|
||||||
std::mem::swap(&mut self.start_key, &mut self.end_key);
|
std::mem::swap(&mut self.start_key, &mut self.end_key);
|
||||||
}
|
}
|
||||||
|
|
|
@ -9,6 +9,7 @@ use tonic::IntoRequest;
|
||||||
|
|
||||||
use crate::proto::kvrpcpb;
|
use crate::proto::kvrpcpb;
|
||||||
use crate::proto::tikvpb::tikv_client::TikvClient;
|
use crate::proto::tikvpb::tikv_client::TikvClient;
|
||||||
|
use crate::store::RegionWithLeader;
|
||||||
use crate::Error;
|
use crate::Error;
|
||||||
use crate::Result;
|
use crate::Result;
|
||||||
|
|
||||||
|
@ -21,9 +22,7 @@ pub trait Request: Any + Sync + Send + 'static {
|
||||||
) -> Result<Box<dyn Any>>;
|
) -> Result<Box<dyn Any>>;
|
||||||
fn label(&self) -> &'static str;
|
fn label(&self) -> &'static str;
|
||||||
fn as_any(&self) -> &dyn Any;
|
fn as_any(&self) -> &dyn Any;
|
||||||
/// Set the context for the request.
|
fn set_leader(&mut self, leader: &RegionWithLeader) -> Result<()>;
|
||||||
/// Should always use `set_context` other than modify the `self.context` directly.
|
|
||||||
fn set_context(&mut self, context: kvrpcpb::Context);
|
|
||||||
fn set_api_version(&mut self, api_version: kvrpcpb::ApiVersion);
|
fn set_api_version(&mut self, api_version: kvrpcpb::ApiVersion);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -54,19 +53,20 @@ macro_rules! impl_request {
|
||||||
self
|
self
|
||||||
}
|
}
|
||||||
|
|
||||||
fn set_context(&mut self, context: kvrpcpb::Context) {
|
fn set_leader(&mut self, leader: &RegionWithLeader) -> Result<()> {
|
||||||
let api_version = self
|
let ctx = self.context.get_or_insert(kvrpcpb::Context::default());
|
||||||
.context
|
let leader_peer = leader.leader.as_ref().ok_or(Error::LeaderNotFound {
|
||||||
.as_ref()
|
region_id: leader.region.id,
|
||||||
.map(|c| c.api_version)
|
})?;
|
||||||
.unwrap_or_default();
|
ctx.region_id = leader.region.id;
|
||||||
self.context = Some(context);
|
ctx.region_epoch = leader.region.region_epoch.clone();
|
||||||
self.set_api_version(kvrpcpb::ApiVersion::try_from(api_version).unwrap());
|
ctx.peer = Some(leader_peer.clone());
|
||||||
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
fn set_api_version(&mut self, api_version: kvrpcpb::ApiVersion) {
|
fn set_api_version(&mut self, api_version: kvrpcpb::ApiVersion) {
|
||||||
let context = self.context.get_or_insert(kvrpcpb::Context::default());
|
let ctx = self.context.get_or_insert(kvrpcpb::Context::default());
|
||||||
context.api_version = api_version.into();
|
ctx.api_version = api_version.into();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -12,6 +12,8 @@ use crate::KvPair;
|
||||||
use crate::Result;
|
use crate::Result;
|
||||||
use crate::Value;
|
use crate::Value;
|
||||||
|
|
||||||
|
use super::transaction::Mutation;
|
||||||
|
|
||||||
/// A caching layer which buffers reads and writes in a transaction.
|
/// A caching layer which buffers reads and writes in a transaction.
|
||||||
pub struct Buffer {
|
pub struct Buffer {
|
||||||
primary_key: Option<Key>,
|
primary_key: Option<Key>,
|
||||||
|
@ -244,12 +246,10 @@ impl Buffer {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
pub(crate) fn mutate(&mut self, m: kvrpcpb::Mutation) {
|
pub(crate) fn mutate(&mut self, m: Mutation) {
|
||||||
let op = kvrpcpb::Op::try_from(m.op).unwrap();
|
match m {
|
||||||
match op {
|
Mutation::Put(key, value) => self.put(key, value),
|
||||||
kvrpcpb::Op::Put => self.put(m.key.into(), m.value),
|
Mutation::Delete(key) => self.delete(key),
|
||||||
kvrpcpb::Op::Del => self.delete(m.key.into()),
|
|
||||||
_ => unimplemented!("only put and delete are supported in mutate"),
|
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -10,8 +10,10 @@ use crate::config::Config;
|
||||||
use crate::pd::PdClient;
|
use crate::pd::PdClient;
|
||||||
use crate::pd::PdRpcClient;
|
use crate::pd::PdRpcClient;
|
||||||
use crate::proto::pdpb::Timestamp;
|
use crate::proto::pdpb::Timestamp;
|
||||||
use crate::request::codec::{ApiV1TxnCodec, ApiV2TxnCodec, Codec, EncodedRequest};
|
|
||||||
use crate::request::plan::CleanupLocksResult;
|
use crate::request::plan::CleanupLocksResult;
|
||||||
|
use crate::request::EncodeKeyspace;
|
||||||
|
use crate::request::KeyMode;
|
||||||
|
use crate::request::Keyspace;
|
||||||
use crate::request::Plan;
|
use crate::request::Plan;
|
||||||
use crate::timestamp::TimestampExt;
|
use crate::timestamp::TimestampExt;
|
||||||
use crate::transaction::lock::ResolveLocksOptions;
|
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
|
/// The returned results of transactional requests are [`Future`](std::future::Future)s that must be
|
||||||
/// awaited to execute.
|
/// awaited to execute.
|
||||||
pub struct Client<Cod: Codec = ApiV1TxnCodec> {
|
pub struct Client {
|
||||||
pd: Arc<PdRpcClient<Cod>>,
|
pd: Arc<PdRpcClient>,
|
||||||
|
keyspace: Keyspace,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<Cod: Codec> Clone for Client<Cod> {
|
impl Clone for Client {
|
||||||
fn clone(&self) -> Self {
|
fn clone(&self) -> Self {
|
||||||
Self {
|
Self {
|
||||||
pd: self.pd.clone(),
|
pd: self.pd.clone(),
|
||||||
|
keyspace: self.keyspace,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Client<ApiV1TxnCodec> {
|
impl Client {
|
||||||
/// Create a transactional [`Client`] and connect to the TiKV cluster.
|
/// 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
|
/// 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> {
|
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
|
Self::new_with_config(pd_endpoints, Config::default()).await
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -101,36 +106,17 @@ impl Client<ApiV1TxnCodec> {
|
||||||
pd_endpoints: Vec<S>,
|
pd_endpoints: Vec<S>,
|
||||||
config: Config,
|
config: Config,
|
||||||
) -> Result<Client> {
|
) -> 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");
|
debug!("creating new transactional client");
|
||||||
let pd_endpoints: Vec<String> = pd_endpoints.into_iter().map(Into::into).collect();
|
let pd_endpoints: Vec<String> = pd_endpoints.into_iter().map(Into::into).collect();
|
||||||
let pd =
|
let pd = Arc::new(PdRpcClient::connect(&pd_endpoints, config.clone(), true).await?);
|
||||||
Arc::new(PdRpcClient::<Cod>::connect(&pd_endpoints, config, true, Some(codec)).await?);
|
let keyspace = match config.keyspace {
|
||||||
Ok(Client { pd })
|
Some(keyspace) => {
|
||||||
|
let keyspace_id = pd.get_keyspace_id(&keyspace).await?;
|
||||||
|
Keyspace::Enable { keyspace_id }
|
||||||
|
}
|
||||||
|
None => Keyspace::Disable,
|
||||||
|
};
|
||||||
|
Ok(Client { pd, keyspace })
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Creates a new optimistic [`Transaction`].
|
/// Creates a new optimistic [`Transaction`].
|
||||||
|
@ -153,7 +139,7 @@ impl<Cod: Codec> Client<Cod> {
|
||||||
/// transaction.commit().await.unwrap();
|
/// 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");
|
debug!("creating new optimistic transaction");
|
||||||
let timestamp = self.current_timestamp().await?;
|
let timestamp = self.current_timestamp().await?;
|
||||||
Ok(self.new_transaction(timestamp, TransactionOptions::new_optimistic()))
|
Ok(self.new_transaction(timestamp, TransactionOptions::new_optimistic()))
|
||||||
|
@ -176,7 +162,7 @@ impl<Cod: Codec> Client<Cod> {
|
||||||
/// transaction.commit().await.unwrap();
|
/// 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");
|
debug!("creating new pessimistic transaction");
|
||||||
let timestamp = self.current_timestamp().await?;
|
let timestamp = self.current_timestamp().await?;
|
||||||
Ok(self.new_transaction(timestamp, TransactionOptions::new_pessimistic()))
|
Ok(self.new_transaction(timestamp, TransactionOptions::new_pessimistic()))
|
||||||
|
@ -199,21 +185,14 @@ impl<Cod: Codec> Client<Cod> {
|
||||||
/// transaction.commit().await.unwrap();
|
/// transaction.commit().await.unwrap();
|
||||||
/// # });
|
/// # });
|
||||||
/// ```
|
/// ```
|
||||||
pub async fn begin_with_options(
|
pub async fn begin_with_options(&self, options: TransactionOptions) -> Result<Transaction> {
|
||||||
&self,
|
|
||||||
options: TransactionOptions,
|
|
||||||
) -> Result<Transaction<Cod, PdRpcClient<Cod>>> {
|
|
||||||
debug!("creating new customized transaction");
|
debug!("creating new customized transaction");
|
||||||
let timestamp = self.current_timestamp().await?;
|
let timestamp = self.current_timestamp().await?;
|
||||||
Ok(self.new_transaction(timestamp, options))
|
Ok(self.new_transaction(timestamp, options))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Create a new [`Snapshot`](Snapshot) at the given [`Timestamp`](Timestamp).
|
/// Create a new [`Snapshot`](Snapshot) at the given [`Timestamp`](Timestamp).
|
||||||
pub fn snapshot(
|
pub fn snapshot(&self, timestamp: Timestamp, options: TransactionOptions) -> Snapshot {
|
||||||
&self,
|
|
||||||
timestamp: Timestamp,
|
|
||||||
options: TransactionOptions,
|
|
||||||
) -> Snapshot<Cod, PdRpcClient<Cod>> {
|
|
||||||
debug!("creating new snapshot");
|
debug!("creating new snapshot");
|
||||||
Snapshot::new(self.new_transaction(timestamp, options.read_only()))
|
Snapshot::new(self.new_transaction(timestamp, options.read_only()))
|
||||||
}
|
}
|
||||||
|
@ -279,10 +258,10 @@ impl<Cod: Codec> Client<Cod> {
|
||||||
// scan all locks with ts <= safepoint
|
// scan all locks with ts <= safepoint
|
||||||
let ctx = ResolveLocksContext::default();
|
let ctx = ResolveLocksContext::default();
|
||||||
let backoff = Backoff::equal_jitter_backoff(100, 10000, 50);
|
let backoff = Backoff::equal_jitter_backoff(100, 10000, 50);
|
||||||
let req = new_scan_lock_request(range.into(), safepoint, options.batch_size);
|
let range = range.into().encode_keyspace(self.keyspace, KeyMode::Txn);
|
||||||
let encoded_req = EncodedRequest::new(req, self.pd.get_codec());
|
let req = new_scan_lock_request(range, safepoint, options.batch_size);
|
||||||
let plan = crate::request::PlanBuilder::new(self.pd.clone(), encoded_req)
|
let plan = crate::request::PlanBuilder::new(self.pd.clone(), self.keyspace, req)
|
||||||
.cleanup_locks(ctx.clone(), options, backoff)
|
.cleanup_locks(ctx.clone(), options, backoff, self.keyspace)
|
||||||
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.merge(crate::request::Collect)
|
.merge(crate::request::Collect)
|
||||||
|
@ -299,13 +278,15 @@ impl<Cod: Codec> Client<Cod> {
|
||||||
range: impl Into<BoundRange>,
|
range: impl Into<BoundRange>,
|
||||||
batch_size: u32,
|
batch_size: u32,
|
||||||
) -> Result<Vec<crate::proto::kvrpcpb::LockInfo>> {
|
) -> Result<Vec<crate::proto::kvrpcpb::LockInfo>> {
|
||||||
let req = new_scan_lock_request(range.into(), safepoint, batch_size);
|
use crate::request::TruncateKeyspace;
|
||||||
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_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)
|
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
||||||
.merge(crate::request::Collect)
|
.merge(crate::request::Collect)
|
||||||
.plan();
|
.plan();
|
||||||
plan.execute().await
|
Ok(plan.execute().await?.truncate_keyspace(self.keyspace))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Cleans up all keys in a range and quickly reclaim disk space.
|
/// Cleans up all keys in a range and quickly reclaim disk space.
|
||||||
|
@ -316,20 +297,16 @@ impl<Cod: Codec> Client<Cod> {
|
||||||
///
|
///
|
||||||
/// This interface is intended for special scenarios that resemble operations like "drop table" or "drop database" in TiDB.
|
/// 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<()> {
|
pub async fn unsafe_destroy_range(&self, range: impl Into<BoundRange>) -> Result<()> {
|
||||||
let req = new_unsafe_destroy_range_request(range.into());
|
let range = range.into().encode_keyspace(self.keyspace, KeyMode::Txn);
|
||||||
let encoded_req = EncodedRequest::new(req, self.pd.get_codec());
|
let req = new_unsafe_destroy_range_request(range);
|
||||||
let plan = crate::request::PlanBuilder::new(self.pd.clone(), encoded_req)
|
let plan = crate::request::PlanBuilder::new(self.pd.clone(), self.keyspace, req)
|
||||||
.all_stores(DEFAULT_STORE_BACKOFF)
|
.all_stores(DEFAULT_STORE_BACKOFF)
|
||||||
.merge(crate::request::Collect)
|
.merge(crate::request::Collect)
|
||||||
.plan();
|
.plan();
|
||||||
plan.execute().await
|
plan.execute().await
|
||||||
}
|
}
|
||||||
|
|
||||||
fn new_transaction(
|
fn new_transaction(&self, timestamp: Timestamp, options: TransactionOptions) -> Transaction {
|
||||||
&self,
|
Transaction::new(timestamp, self.pd.clone(), options, self.keyspace)
|
||||||
timestamp: Timestamp,
|
|
||||||
options: TransactionOptions,
|
|
||||||
) -> Transaction<Cod, PdRpcClient<Cod>> {
|
|
||||||
Transaction::new(timestamp, self.pd.clone(), options)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -13,13 +13,14 @@ use crate::backoff::Backoff;
|
||||||
use crate::backoff::DEFAULT_REGION_BACKOFF;
|
use crate::backoff::DEFAULT_REGION_BACKOFF;
|
||||||
use crate::backoff::OPTIMISTIC_BACKOFF;
|
use crate::backoff::OPTIMISTIC_BACKOFF;
|
||||||
use crate::pd::PdClient;
|
use crate::pd::PdClient;
|
||||||
|
|
||||||
use crate::proto::kvrpcpb;
|
use crate::proto::kvrpcpb;
|
||||||
use crate::proto::kvrpcpb::TxnInfo;
|
use crate::proto::kvrpcpb::TxnInfo;
|
||||||
use crate::proto::pdpb::Timestamp;
|
use crate::proto::pdpb::Timestamp;
|
||||||
use crate::region::RegionVerId;
|
use crate::region::RegionVerId;
|
||||||
use crate::request::codec::EncodedRequest;
|
|
||||||
use crate::request::Collect;
|
use crate::request::Collect;
|
||||||
use crate::request::CollectSingle;
|
use crate::request::CollectSingle;
|
||||||
|
use crate::request::Keyspace;
|
||||||
use crate::request::Plan;
|
use crate::request::Plan;
|
||||||
use crate::store::RegionStore;
|
use crate::store::RegionStore;
|
||||||
use crate::timestamp::TimestampExt;
|
use crate::timestamp::TimestampExt;
|
||||||
|
@ -44,6 +45,7 @@ const RESOLVE_LOCK_RETRY_LIMIT: usize = 10;
|
||||||
pub async fn resolve_locks(
|
pub async fn resolve_locks(
|
||||||
locks: Vec<kvrpcpb::LockInfo>,
|
locks: Vec<kvrpcpb::LockInfo>,
|
||||||
pd_client: Arc<impl PdClient>,
|
pd_client: Arc<impl PdClient>,
|
||||||
|
keyspace: Keyspace,
|
||||||
) -> Result<Vec<kvrpcpb::LockInfo> /* live_locks */> {
|
) -> Result<Vec<kvrpcpb::LockInfo> /* live_locks */> {
|
||||||
debug!("resolving locks");
|
debug!("resolving locks");
|
||||||
let ts = pd_client.clone().get_timestamp().await?;
|
let ts = pd_client.clone().get_timestamp().await?;
|
||||||
|
@ -76,9 +78,8 @@ pub async fn resolve_locks(
|
||||||
Some(&commit_version) => commit_version,
|
Some(&commit_version) => commit_version,
|
||||||
None => {
|
None => {
|
||||||
let request = requests::new_cleanup_request(lock.primary_lock, lock.lock_version);
|
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(), keyspace, request)
|
||||||
let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req)
|
.resolve_lock(OPTIMISTIC_BACKOFF, keyspace)
|
||||||
.resolve_lock(OPTIMISTIC_BACKOFF)
|
|
||||||
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.post_process_default()
|
.post_process_default()
|
||||||
|
@ -94,6 +95,7 @@ pub async fn resolve_locks(
|
||||||
lock.lock_version,
|
lock.lock_version,
|
||||||
commit_version,
|
commit_version,
|
||||||
pd_client.clone(),
|
pd_client.clone(),
|
||||||
|
keyspace,
|
||||||
)
|
)
|
||||||
.await?;
|
.await?;
|
||||||
clean_regions
|
clean_regions
|
||||||
|
@ -109,6 +111,7 @@ async fn resolve_lock_with_retry(
|
||||||
start_version: u64,
|
start_version: u64,
|
||||||
commit_version: u64,
|
commit_version: u64,
|
||||||
pd_client: Arc<impl PdClient>,
|
pd_client: Arc<impl PdClient>,
|
||||||
|
keyspace: Keyspace,
|
||||||
) -> Result<RegionVerId> {
|
) -> Result<RegionVerId> {
|
||||||
debug!("resolving locks with retry");
|
debug!("resolving locks with retry");
|
||||||
// FIXME: Add backoff
|
// 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 store = pd_client.clone().store_for_key(key.into()).await?;
|
||||||
let ver_id = store.region_with_leader.ver_id();
|
let ver_id = store.region_with_leader.ver_id();
|
||||||
let request = requests::new_resolve_lock_request(start_version, commit_version);
|
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(), keyspace, request)
|
||||||
let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req)
|
|
||||||
.single_region_with_store(store)
|
.single_region_with_store(store)
|
||||||
.await?
|
.await?
|
||||||
.resolve_lock(Backoff::no_backoff())
|
.resolve_lock(Backoff::no_backoff(), keyspace)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
match plan.execute().await {
|
match plan.execute().await {
|
||||||
|
@ -214,6 +216,7 @@ impl LockResolver {
|
||||||
store: RegionStore,
|
store: RegionStore,
|
||||||
locks: Vec<kvrpcpb::LockInfo>,
|
locks: Vec<kvrpcpb::LockInfo>,
|
||||||
pd_client: Arc<impl PdClient>, // TODO: make pd_client a member of LockResolver
|
pd_client: Arc<impl PdClient>, // TODO: make pd_client a member of LockResolver
|
||||||
|
keyspace: Keyspace,
|
||||||
) -> Result<()> {
|
) -> Result<()> {
|
||||||
if locks.is_empty() {
|
if locks.is_empty() {
|
||||||
return Ok(());
|
return Ok(());
|
||||||
|
@ -235,6 +238,7 @@ impl LockResolver {
|
||||||
let mut status = self
|
let mut status = self
|
||||||
.check_txn_status(
|
.check_txn_status(
|
||||||
pd_client.clone(),
|
pd_client.clone(),
|
||||||
|
keyspace,
|
||||||
txn_id,
|
txn_id,
|
||||||
l.primary_lock.clone(),
|
l.primary_lock.clone(),
|
||||||
0,
|
0,
|
||||||
|
@ -249,7 +253,12 @@ impl LockResolver {
|
||||||
// Then we need to check the secondary locks to determine the final status of the transaction.
|
// Then we need to check the secondary locks to determine the final status of the transaction.
|
||||||
if let TransactionStatusKind::Locked(_, lock_info) = &status.kind {
|
if let TransactionStatusKind::Locked(_, lock_info) = &status.kind {
|
||||||
let secondary_status = self
|
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?;
|
.await?;
|
||||||
debug!(
|
debug!(
|
||||||
"secondary status, txn_id:{}, commit_ts:{:?}, min_commit_version:{}, fallback_2pc:{}",
|
"secondary status, txn_id:{}, commit_ts:{:?}, min_commit_version:{}, fallback_2pc:{}",
|
||||||
|
@ -267,6 +276,7 @@ impl LockResolver {
|
||||||
status = self
|
status = self
|
||||||
.check_txn_status(
|
.check_txn_status(
|
||||||
pd_client.clone(),
|
pd_client.clone(),
|
||||||
|
keyspace,
|
||||||
txn_id,
|
txn_id,
|
||||||
l.primary_lock,
|
l.primary_lock,
|
||||||
0,
|
0,
|
||||||
|
@ -315,7 +325,7 @@ impl LockResolver {
|
||||||
txn_info_vec.push(txn_info);
|
txn_info_vec.push(txn_info);
|
||||||
}
|
}
|
||||||
let cleaned_region = self
|
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?;
|
.await?;
|
||||||
for txn_id in txn_ids {
|
for txn_id in txn_ids {
|
||||||
self.ctx
|
self.ctx
|
||||||
|
@ -330,6 +340,7 @@ impl LockResolver {
|
||||||
pub async fn check_txn_status(
|
pub async fn check_txn_status(
|
||||||
&mut self,
|
&mut self,
|
||||||
pd_client: Arc<impl PdClient>,
|
pd_client: Arc<impl PdClient>,
|
||||||
|
keyspace: Keyspace,
|
||||||
txn_id: u64,
|
txn_id: u64,
|
||||||
primary: Vec<u8>,
|
primary: Vec<u8>,
|
||||||
caller_start_ts: u64,
|
caller_start_ts: u64,
|
||||||
|
@ -359,8 +370,7 @@ impl LockResolver {
|
||||||
force_sync_commit,
|
force_sync_commit,
|
||||||
resolving_pessimistic_lock,
|
resolving_pessimistic_lock,
|
||||||
);
|
);
|
||||||
let encoded_req = EncodedRequest::new(req, pd_client.get_codec());
|
let plan = crate::request::PlanBuilder::new(pd_client.clone(), keyspace, req)
|
||||||
let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req)
|
|
||||||
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
|
@ -380,12 +390,12 @@ impl LockResolver {
|
||||||
async fn check_all_secondaries(
|
async fn check_all_secondaries(
|
||||||
&mut self,
|
&mut self,
|
||||||
pd_client: Arc<impl PdClient>,
|
pd_client: Arc<impl PdClient>,
|
||||||
|
keyspace: Keyspace,
|
||||||
keys: Vec<Vec<u8>>,
|
keys: Vec<Vec<u8>>,
|
||||||
txn_id: u64,
|
txn_id: u64,
|
||||||
) -> Result<SecondaryLocksStatus> {
|
) -> Result<SecondaryLocksStatus> {
|
||||||
let req = new_check_secondary_locks_request(keys, txn_id);
|
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(), keyspace, req)
|
||||||
let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req)
|
|
||||||
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.merge(Collect)
|
.merge(Collect)
|
||||||
|
@ -396,13 +406,13 @@ impl LockResolver {
|
||||||
async fn batch_resolve_locks(
|
async fn batch_resolve_locks(
|
||||||
&mut self,
|
&mut self,
|
||||||
pd_client: Arc<impl PdClient>,
|
pd_client: Arc<impl PdClient>,
|
||||||
|
keyspace: Keyspace,
|
||||||
store: RegionStore,
|
store: RegionStore,
|
||||||
txn_infos: Vec<TxnInfo>,
|
txn_infos: Vec<TxnInfo>,
|
||||||
) -> Result<RegionVerId> {
|
) -> Result<RegionVerId> {
|
||||||
let ver_id = store.region_with_leader.ver_id();
|
let ver_id = store.region_with_leader.ver_id();
|
||||||
let request = requests::new_batch_resolve_lock_request(txn_infos.clone());
|
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(), keyspace, request)
|
||||||
let plan = crate::request::PlanBuilder::new(pd_client.clone(), encoded_req)
|
|
||||||
.single_region_with_store(store.clone())
|
.single_region_with_store(store.clone())
|
||||||
.await?
|
.await?
|
||||||
.extract_error()
|
.extract_error()
|
||||||
|
@ -422,13 +432,19 @@ pub trait HasLocks {
|
||||||
mod tests {
|
mod tests {
|
||||||
use std::any::Any;
|
use std::any::Any;
|
||||||
|
|
||||||
|
use serial_test::serial;
|
||||||
|
|
||||||
use super::*;
|
use super::*;
|
||||||
use crate::mock::MockKvClient;
|
use crate::mock::MockKvClient;
|
||||||
use crate::mock::MockPdClient;
|
use crate::mock::MockPdClient;
|
||||||
use crate::proto::errorpb;
|
use crate::proto::errorpb;
|
||||||
|
|
||||||
|
#[rstest::rstest]
|
||||||
|
#[case(Keyspace::Disable)]
|
||||||
|
#[case(Keyspace::Enable { keyspace_id: 0 })]
|
||||||
#[tokio::test]
|
#[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
|
// Test resolve lock within retry limit
|
||||||
fail::cfg("region-error", "9*return").unwrap();
|
fail::cfg("region-error", "9*return").unwrap();
|
||||||
|
|
||||||
|
@ -447,7 +463,7 @@ mod tests {
|
||||||
|
|
||||||
let key = vec![1];
|
let key = vec![1];
|
||||||
let region1 = MockPdClient::region1();
|
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
|
.await
|
||||||
.unwrap();
|
.unwrap();
|
||||||
assert_eq!(region1.ver_id(), resolved_region);
|
assert_eq!(region1.ver_id(), resolved_region);
|
||||||
|
@ -455,7 +471,7 @@ mod tests {
|
||||||
// Test resolve lock over retry limit
|
// Test resolve lock over retry limit
|
||||||
fail::cfg("region-error", "10*return").unwrap();
|
fail::cfg("region-error", "10*return").unwrap();
|
||||||
let key = vec![100];
|
let key = vec![100];
|
||||||
resolve_lock_with_retry(&key, 3, 4, client)
|
resolve_lock_with_retry(&key, 3, 4, client, keyspace)
|
||||||
.await
|
.await
|
||||||
.expect_err("should return error");
|
.expect_err("should return error");
|
||||||
}
|
}
|
||||||
|
|
|
@ -15,6 +15,7 @@ pub use snapshot::Snapshot;
|
||||||
pub use transaction::CheckLevel;
|
pub use transaction::CheckLevel;
|
||||||
#[doc(hidden)]
|
#[doc(hidden)]
|
||||||
pub use transaction::HeartbeatOption;
|
pub use transaction::HeartbeatOption;
|
||||||
|
pub use transaction::Mutation;
|
||||||
pub use transaction::Transaction;
|
pub use transaction::Transaction;
|
||||||
pub use transaction::TransactionOptions;
|
pub use transaction::TransactionOptions;
|
||||||
|
|
||||||
|
|
|
@ -298,7 +298,7 @@ impl Shardable for kvrpcpb::PrewriteRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
||||||
self.set_context(store.region_with_leader.context()?);
|
self.set_leader(&store.region_with_leader)?;
|
||||||
|
|
||||||
// Only need to set secondary keys if we're sending the primary key.
|
// Only need to set secondary keys if we're sending the primary key.
|
||||||
if self.use_async_commit && !self.mutations.iter().any(|m| m.key == self.primary_lock) {
|
if self.use_async_commit && !self.mutations.iter().any(|m| m.key == self.primary_lock) {
|
||||||
|
@ -365,7 +365,7 @@ impl Shardable for kvrpcpb::CommitRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
||||||
self.set_context(store.region_with_leader.context()?);
|
self.set_leader(&store.region_with_leader)?;
|
||||||
self.keys = shard.into_iter().map(Into::into).collect();
|
self.keys = shard.into_iter().map(Into::into).collect();
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
@ -456,7 +456,7 @@ impl Shardable for kvrpcpb::PessimisticLockRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
||||||
self.set_context(store.region_with_leader.context()?);
|
self.set_leader(&store.region_with_leader)?;
|
||||||
self.mutations = shard;
|
self.mutations = shard;
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
@ -557,7 +557,7 @@ impl Shardable for kvrpcpb::ScanLockRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
fn apply_shard(&mut self, shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
||||||
self.set_context(store.region_with_leader.context()?);
|
self.set_leader(&store.region_with_leader)?;
|
||||||
self.start_key = shard.0;
|
self.start_key = shard.0;
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
@ -618,7 +618,7 @@ impl Shardable for kvrpcpb::TxnHeartBeatRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, mut shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
fn apply_shard(&mut self, mut shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
||||||
self.set_context(store.region_with_leader.context()?);
|
self.set_leader(&store.region_with_leader)?;
|
||||||
assert!(shard.len() == 1);
|
assert!(shard.len() == 1);
|
||||||
self.primary_lock = shard.pop().unwrap();
|
self.primary_lock = shard.pop().unwrap();
|
||||||
Ok(())
|
Ok(())
|
||||||
|
@ -676,7 +676,7 @@ impl Shardable for kvrpcpb::CheckTxnStatusRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn apply_shard(&mut self, mut shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
fn apply_shard(&mut self, mut shard: Self::Shard, store: &RegionStore) -> Result<()> {
|
||||||
self.set_context(store.region_with_leader.context()?);
|
self.set_leader(&store.region_with_leader)?;
|
||||||
assert!(shard.len() == 1);
|
assert!(shard.len() == 1);
|
||||||
self.primary_key = shard.pop().unwrap();
|
self.primary_key = shard.pop().unwrap();
|
||||||
Ok(())
|
Ok(())
|
||||||
|
|
|
@ -2,11 +2,7 @@
|
||||||
|
|
||||||
use derive_new::new;
|
use derive_new::new;
|
||||||
use log::debug;
|
use log::debug;
|
||||||
use std::marker::PhantomData;
|
|
||||||
|
|
||||||
use crate::codec::ApiV1TxnCodec;
|
|
||||||
use crate::pd::{PdClient, PdRpcClient};
|
|
||||||
use crate::request::codec::Codec;
|
|
||||||
use crate::BoundRange;
|
use crate::BoundRange;
|
||||||
use crate::Key;
|
use crate::Key;
|
||||||
use crate::KvPair;
|
use crate::KvPair;
|
||||||
|
@ -22,12 +18,11 @@ use crate::Value;
|
||||||
///
|
///
|
||||||
/// See the [Transaction](struct@crate::Transaction) docs for more information on the methods.
|
/// See the [Transaction](struct@crate::Transaction) docs for more information on the methods.
|
||||||
#[derive(new)]
|
#[derive(new)]
|
||||||
pub struct Snapshot<Cod: Codec = ApiV1TxnCodec, PdC: PdClient<Codec = Cod> = PdRpcClient<Cod>> {
|
pub struct Snapshot {
|
||||||
transaction: Transaction<Cod, PdC>,
|
transaction: Transaction,
|
||||||
phantom: PhantomData<Cod>,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Snapshot<Cod, PdC> {
|
impl Snapshot {
|
||||||
/// Get the value associated with the given key.
|
/// Get the value associated with the given key.
|
||||||
pub async fn get(&mut self, key: impl Into<Key>) -> Result<Option<Value>> {
|
pub async fn get(&mut self, key: impl Into<Key>) -> Result<Option<Value>> {
|
||||||
debug!("invoking get request on snapshot");
|
debug!("invoking get request on snapshot");
|
||||||
|
|
|
@ -1,7 +1,6 @@
|
||||||
// Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0.
|
// Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0.
|
||||||
|
|
||||||
use std::iter;
|
use std::iter;
|
||||||
use std::marker::PhantomData;
|
|
||||||
use std::sync::atomic;
|
use std::sync::atomic;
|
||||||
use std::sync::atomic::AtomicU8;
|
use std::sync::atomic::AtomicU8;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
@ -16,19 +15,21 @@ use tokio::time::Duration;
|
||||||
|
|
||||||
use crate::backoff::Backoff;
|
use crate::backoff::Backoff;
|
||||||
use crate::backoff::DEFAULT_REGION_BACKOFF;
|
use crate::backoff::DEFAULT_REGION_BACKOFF;
|
||||||
use crate::codec::ApiV1TxnCodec;
|
|
||||||
use crate::pd::PdClient;
|
use crate::pd::PdClient;
|
||||||
use crate::pd::PdRpcClient;
|
use crate::pd::PdRpcClient;
|
||||||
use crate::proto::kvrpcpb;
|
use crate::proto::kvrpcpb;
|
||||||
use crate::proto::pdpb::Timestamp;
|
use crate::proto::pdpb::Timestamp;
|
||||||
use crate::request::codec::{Codec, EncodedRequest};
|
|
||||||
use crate::request::Collect;
|
use crate::request::Collect;
|
||||||
use crate::request::CollectError;
|
use crate::request::CollectError;
|
||||||
use crate::request::CollectSingle;
|
use crate::request::CollectSingle;
|
||||||
use crate::request::CollectWithShard;
|
use crate::request::CollectWithShard;
|
||||||
|
use crate::request::EncodeKeyspace;
|
||||||
|
use crate::request::KeyMode;
|
||||||
|
use crate::request::Keyspace;
|
||||||
use crate::request::Plan;
|
use crate::request::Plan;
|
||||||
use crate::request::PlanBuilder;
|
use crate::request::PlanBuilder;
|
||||||
use crate::request::RetryOptions;
|
use crate::request::RetryOptions;
|
||||||
|
use crate::request::TruncateKeyspace;
|
||||||
use crate::timestamp::TimestampExt;
|
use crate::timestamp::TimestampExt;
|
||||||
use crate::transaction::buffer::Buffer;
|
use crate::transaction::buffer::Buffer;
|
||||||
use crate::transaction::lowering::*;
|
use crate::transaction::lowering::*;
|
||||||
|
@ -77,23 +78,24 @@ use crate::Value;
|
||||||
/// txn.commit().await.unwrap();
|
/// txn.commit().await.unwrap();
|
||||||
/// # });
|
/// # });
|
||||||
/// ```
|
/// ```
|
||||||
pub struct Transaction<Cod: Codec = ApiV1TxnCodec, PdC: PdClient<Codec = Cod> = PdRpcClient<Cod>> {
|
pub struct Transaction<PdC: PdClient = PdRpcClient> {
|
||||||
status: Arc<AtomicU8>,
|
status: Arc<AtomicU8>,
|
||||||
timestamp: Timestamp,
|
timestamp: Timestamp,
|
||||||
buffer: Buffer,
|
buffer: Buffer,
|
||||||
rpc: Arc<PdC>,
|
rpc: Arc<PdC>,
|
||||||
options: TransactionOptions,
|
options: TransactionOptions,
|
||||||
|
keyspace: Keyspace,
|
||||||
is_heartbeat_started: bool,
|
is_heartbeat_started: bool,
|
||||||
start_instant: Instant,
|
start_instant: Instant,
|
||||||
phantom: PhantomData<Cod>,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
impl<PdC: PdClient> Transaction<PdC> {
|
||||||
pub(crate) fn new(
|
pub(crate) fn new(
|
||||||
timestamp: Timestamp,
|
timestamp: Timestamp,
|
||||||
rpc: Arc<PdC>,
|
rpc: Arc<PdC>,
|
||||||
options: TransactionOptions,
|
options: TransactionOptions,
|
||||||
) -> Transaction<Cod, PdC> {
|
keyspace: Keyspace,
|
||||||
|
) -> Transaction<PdC> {
|
||||||
let status = if options.read_only {
|
let status = if options.read_only {
|
||||||
TransactionStatus::ReadOnly
|
TransactionStatus::ReadOnly
|
||||||
} else {
|
} else {
|
||||||
|
@ -105,9 +107,9 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
buffer: Buffer::new(options.is_pessimistic()),
|
buffer: Buffer::new(options.is_pessimistic()),
|
||||||
rpc,
|
rpc,
|
||||||
options,
|
options,
|
||||||
|
keyspace,
|
||||||
is_heartbeat_started: false,
|
is_heartbeat_started: false,
|
||||||
start_instant: std::time::Instant::now(),
|
start_instant: std::time::Instant::now(),
|
||||||
phantom: PhantomData,
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -134,15 +136,15 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
self.check_allow_operation().await?;
|
self.check_allow_operation().await?;
|
||||||
let timestamp = self.timestamp.clone();
|
let timestamp = self.timestamp.clone();
|
||||||
let rpc = self.rpc.clone();
|
let rpc = self.rpc.clone();
|
||||||
let key = key.into();
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn);
|
||||||
let retry_options = self.options.retry_options.clone();
|
let retry_options = self.options.retry_options.clone();
|
||||||
|
let keyspace = self.keyspace;
|
||||||
|
|
||||||
self.buffer
|
self.buffer
|
||||||
.get_or_else(key, |key| async move {
|
.get_or_else(key, |key| async move {
|
||||||
let request = new_get_request(key, timestamp);
|
let request = new_get_request(key, timestamp);
|
||||||
let encoded_req = EncodedRequest::new(request, rpc.get_codec());
|
let plan = PlanBuilder::new(rpc, keyspace, request)
|
||||||
let plan = PlanBuilder::new(rpc, encoded_req)
|
.resolve_lock(retry_options.lock_backoff, keyspace)
|
||||||
.resolve_lock(retry_options.lock_backoff)
|
|
||||||
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
.retry_multi_region(DEFAULT_REGION_BACKOFF)
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.post_process_default()
|
.post_process_default()
|
||||||
|
@ -202,7 +204,8 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
self.lock_keys(iter::once(key.clone())).await?;
|
self.lock_keys(iter::once(key.clone())).await?;
|
||||||
self.get(key).await
|
self.get(key).await
|
||||||
} else {
|
} else {
|
||||||
let mut pairs = self.pessimistic_lock(iter::once(key.into()), true).await?;
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn);
|
||||||
|
let mut pairs = self.pessimistic_lock(iter::once(key), true).await?;
|
||||||
debug_assert!(pairs.len() <= 1);
|
debug_assert!(pairs.len() <= 1);
|
||||||
match pairs.pop() {
|
match pairs.pop() {
|
||||||
Some(pair) => Ok(Some(pair.1)),
|
Some(pair) => Ok(Some(pair.1)),
|
||||||
|
@ -266,14 +269,17 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
self.check_allow_operation().await?;
|
self.check_allow_operation().await?;
|
||||||
let timestamp = self.timestamp.clone();
|
let timestamp = self.timestamp.clone();
|
||||||
let rpc = self.rpc.clone();
|
let rpc = self.rpc.clone();
|
||||||
|
let keyspace = self.keyspace;
|
||||||
|
let keys = keys
|
||||||
|
.into_iter()
|
||||||
|
.map(move |k| k.into().encode_keyspace(keyspace, KeyMode::Txn));
|
||||||
let retry_options = self.options.retry_options.clone();
|
let retry_options = self.options.retry_options.clone();
|
||||||
|
|
||||||
self.buffer
|
self.buffer
|
||||||
.batch_get_or_else(keys.into_iter().map(|k| k.into()), move |keys| async move {
|
.batch_get_or_else(keys, move |keys| async move {
|
||||||
let request = new_batch_get_request(keys, timestamp);
|
let request = new_batch_get_request(keys, timestamp);
|
||||||
let encoded_req = EncodedRequest::new(request, rpc.get_codec());
|
let plan = PlanBuilder::new(rpc, keyspace, request)
|
||||||
let plan = PlanBuilder::new(rpc, encoded_req)
|
.resolve_lock(retry_options.lock_backoff, keyspace)
|
||||||
.resolve_lock(retry_options.lock_backoff)
|
|
||||||
.retry_multi_region(retry_options.region_backoff)
|
.retry_multi_region(retry_options.region_backoff)
|
||||||
.merge(Collect)
|
.merge(Collect)
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -282,6 +288,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
.map(|r| r.into_iter().map(Into::into).collect())
|
.map(|r| r.into_iter().map(Into::into).collect())
|
||||||
})
|
})
|
||||||
.await
|
.await
|
||||||
|
.map(move |pairs| pairs.map(move |pair| pair.truncate_keyspace(keyspace)))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Create a new 'batch get for update' request.
|
/// Create a new 'batch get for update' request.
|
||||||
|
@ -317,12 +324,20 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
) -> Result<Vec<KvPair>> {
|
) -> Result<Vec<KvPair>> {
|
||||||
debug!("invoking transactional batch_get_for_update request");
|
debug!("invoking transactional batch_get_for_update request");
|
||||||
self.check_allow_operation().await?;
|
self.check_allow_operation().await?;
|
||||||
let keys: Vec<Key> = keys.into_iter().map(|k| k.into()).collect();
|
|
||||||
if !self.is_pessimistic() {
|
if !self.is_pessimistic() {
|
||||||
|
let keys: Vec<Key> = keys.into_iter().map(|k| k.into()).collect();
|
||||||
self.lock_keys(keys.clone()).await?;
|
self.lock_keys(keys.clone()).await?;
|
||||||
Ok(self.batch_get(keys).await?.collect())
|
Ok(self.batch_get(keys).await?.collect())
|
||||||
} else {
|
} else {
|
||||||
self.pessimistic_lock(keys, true).await
|
let keyspace = self.keyspace;
|
||||||
|
let keys = keys
|
||||||
|
.into_iter()
|
||||||
|
.map(move |k| k.into().encode_keyspace(keyspace, KeyMode::Txn));
|
||||||
|
let pairs = self
|
||||||
|
.pessimistic_lock(keys, true)
|
||||||
|
.await?
|
||||||
|
.truncate_keyspace(keyspace);
|
||||||
|
Ok(pairs)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -448,7 +463,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
pub async fn put(&mut self, key: impl Into<Key>, value: impl Into<Value>) -> Result<()> {
|
pub async fn put(&mut self, key: impl Into<Key>, value: impl Into<Value>) -> Result<()> {
|
||||||
debug!("invoking transactional put request");
|
debug!("invoking transactional put request");
|
||||||
self.check_allow_operation().await?;
|
self.check_allow_operation().await?;
|
||||||
let key = key.into();
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn);
|
||||||
if self.is_pessimistic() {
|
if self.is_pessimistic() {
|
||||||
self.pessimistic_lock(iter::once(key.clone()), false)
|
self.pessimistic_lock(iter::once(key.clone()), false)
|
||||||
.await?;
|
.await?;
|
||||||
|
@ -479,7 +494,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
pub async fn insert(&mut self, key: impl Into<Key>, value: impl Into<Value>) -> Result<()> {
|
pub async fn insert(&mut self, key: impl Into<Key>, value: impl Into<Value>) -> Result<()> {
|
||||||
debug!("invoking transactional insert request");
|
debug!("invoking transactional insert request");
|
||||||
self.check_allow_operation().await?;
|
self.check_allow_operation().await?;
|
||||||
let key = key.into();
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn);
|
||||||
if self.buffer.get(&key).is_some() {
|
if self.buffer.get(&key).is_some() {
|
||||||
return Err(Error::DuplicateKeyInsertion);
|
return Err(Error::DuplicateKeyInsertion);
|
||||||
}
|
}
|
||||||
|
@ -514,7 +529,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
pub async fn delete(&mut self, key: impl Into<Key>) -> Result<()> {
|
pub async fn delete(&mut self, key: impl Into<Key>) -> Result<()> {
|
||||||
debug!("invoking transactional delete request");
|
debug!("invoking transactional delete request");
|
||||||
self.check_allow_operation().await?;
|
self.check_allow_operation().await?;
|
||||||
let key = key.into();
|
let key = key.into().encode_keyspace(self.keyspace, KeyMode::Txn);
|
||||||
if self.is_pessimistic() {
|
if self.is_pessimistic() {
|
||||||
self.pessimistic_lock(iter::once(key.clone()), false)
|
self.pessimistic_lock(iter::once(key.clone()), false)
|
||||||
.await?;
|
.await?;
|
||||||
|
@ -530,23 +545,14 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
/// # Examples
|
/// # Examples
|
||||||
///
|
///
|
||||||
/// ```rust,no_run
|
/// ```rust,no_run
|
||||||
/// # use tikv_client::{Key, Config, TransactionClient, proto::kvrpcpb};
|
/// # use tikv_client::{Key, Config, TransactionClient, transaction::Mutation};
|
||||||
/// # use futures::prelude::*;
|
/// # use futures::prelude::*;
|
||||||
/// # futures::executor::block_on(async {
|
/// # futures::executor::block_on(async {
|
||||||
/// # let client = TransactionClient::new(vec!["192.168.0.100", "192.168.0.101"]).await.unwrap();
|
/// # let client = TransactionClient::new(vec!["192.168.0.100", "192.168.0.101"]).await.unwrap();
|
||||||
/// let mut txn = client.begin_optimistic().await.unwrap();
|
/// let mut txn = client.begin_optimistic().await.unwrap();
|
||||||
/// let mutations = vec![
|
/// let mutations = vec![
|
||||||
/// kvrpcpb::Mutation {
|
/// Mutation::Delete("k0".to_owned().into()),
|
||||||
/// op: kvrpcpb::Op::Del.into(),
|
/// Mutation::Put("k1".to_owned().into(), b"v1".to_vec()),
|
||||||
/// key: b"k0".to_vec(),
|
|
||||||
/// ..Default::default()
|
|
||||||
/// },
|
|
||||||
/// kvrpcpb::Mutation {
|
|
||||||
/// op: kvrpcpb::Op::Put.into(),
|
|
||||||
/// key: b"k1".to_vec(),
|
|
||||||
/// value: b"v1".to_vec(),
|
|
||||||
/// ..Default::default()
|
|
||||||
/// },
|
|
||||||
/// ];
|
/// ];
|
||||||
/// txn.batch_mutate(mutations).await.unwrap();
|
/// txn.batch_mutate(mutations).await.unwrap();
|
||||||
/// txn.commit().await.unwrap();
|
/// txn.commit().await.unwrap();
|
||||||
|
@ -554,13 +560,16 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
/// ```
|
/// ```
|
||||||
pub async fn batch_mutate(
|
pub async fn batch_mutate(
|
||||||
&mut self,
|
&mut self,
|
||||||
mutations: impl IntoIterator<Item = kvrpcpb::Mutation>,
|
mutations: impl IntoIterator<Item = Mutation>,
|
||||||
) -> Result<()> {
|
) -> Result<()> {
|
||||||
debug!("invoking transactional batch mutate request");
|
debug!("invoking transactional batch mutate request");
|
||||||
self.check_allow_operation().await?;
|
self.check_allow_operation().await?;
|
||||||
|
let mutations: Vec<Mutation> = mutations
|
||||||
|
.into_iter()
|
||||||
|
.map(|mutation| mutation.encode_keyspace(self.keyspace, KeyMode::Txn))
|
||||||
|
.collect();
|
||||||
if self.is_pessimistic() {
|
if self.is_pessimistic() {
|
||||||
let mutations: Vec<kvrpcpb::Mutation> = mutations.into_iter().collect();
|
self.pessimistic_lock(mutations.iter().map(|m| m.key().clone()), false)
|
||||||
self.pessimistic_lock(mutations.iter().map(|m| Key::from(m.key.clone())), false)
|
|
||||||
.await?;
|
.await?;
|
||||||
for m in mutations {
|
for m in mutations {
|
||||||
self.buffer.mutate(m);
|
self.buffer.mutate(m);
|
||||||
|
@ -602,15 +611,18 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
) -> Result<()> {
|
) -> Result<()> {
|
||||||
debug!("invoking transactional lock_keys request");
|
debug!("invoking transactional lock_keys request");
|
||||||
self.check_allow_operation().await?;
|
self.check_allow_operation().await?;
|
||||||
|
let keyspace = self.keyspace;
|
||||||
|
let keys = keys
|
||||||
|
.into_iter()
|
||||||
|
.map(move |k| k.into().encode_keyspace(keyspace, KeyMode::Txn));
|
||||||
match self.options.kind {
|
match self.options.kind {
|
||||||
TransactionKind::Optimistic => {
|
TransactionKind::Optimistic => {
|
||||||
for key in keys {
|
for key in keys {
|
||||||
self.buffer.lock(key.into());
|
self.buffer.lock(key);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
TransactionKind::Pessimistic(_) => {
|
TransactionKind::Pessimistic(_) => {
|
||||||
self.pessimistic_lock(keys.into_iter().map(|k| k.into()), false)
|
self.pessimistic_lock(keys, false).await?;
|
||||||
.await?;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
Ok(())
|
Ok(())
|
||||||
|
@ -660,6 +672,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
self.timestamp.clone(),
|
self.timestamp.clone(),
|
||||||
self.rpc.clone(),
|
self.rpc.clone(),
|
||||||
self.options.clone(),
|
self.options.clone(),
|
||||||
|
self.keyspace,
|
||||||
self.buffer.get_write_size() as u64,
|
self.buffer.get_write_size() as u64,
|
||||||
self.start_instant,
|
self.start_instant,
|
||||||
)
|
)
|
||||||
|
@ -712,6 +725,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
self.timestamp.clone(),
|
self.timestamp.clone(),
|
||||||
self.rpc.clone(),
|
self.rpc.clone(),
|
||||||
self.options.clone(),
|
self.options.clone(),
|
||||||
|
self.keyspace,
|
||||||
self.buffer.get_write_size() as u64,
|
self.buffer.get_write_size() as u64,
|
||||||
self.start_instant,
|
self.start_instant,
|
||||||
)
|
)
|
||||||
|
@ -745,10 +759,13 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
primary_key,
|
primary_key,
|
||||||
self.start_instant.elapsed().as_millis() as u64 + MAX_TTL,
|
self.start_instant.elapsed().as_millis() as u64 + MAX_TTL,
|
||||||
);
|
);
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, request)
|
||||||
let plan = PlanBuilder::new(self.rpc.clone(), encoded_req)
|
.resolve_lock(
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff.clone())
|
self.options.retry_options.lock_backoff.clone(),
|
||||||
|
self.keyspace,
|
||||||
|
)
|
||||||
.retry_multi_region(self.options.retry_options.region_backoff.clone())
|
.retry_multi_region(self.options.retry_options.region_backoff.clone())
|
||||||
|
.extract_error()
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.post_process_default()
|
.post_process_default()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -766,19 +783,20 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
let timestamp = self.timestamp.clone();
|
let timestamp = self.timestamp.clone();
|
||||||
let rpc = self.rpc.clone();
|
let rpc = self.rpc.clone();
|
||||||
let retry_options = self.options.retry_options.clone();
|
let retry_options = self.options.retry_options.clone();
|
||||||
|
let keyspace = self.keyspace;
|
||||||
|
let range = range.into().encode_keyspace(self.keyspace, KeyMode::Txn);
|
||||||
|
|
||||||
self.buffer
|
self.buffer
|
||||||
.scan_and_fetch(
|
.scan_and_fetch(
|
||||||
range.into(),
|
range,
|
||||||
limit,
|
limit,
|
||||||
!key_only,
|
!key_only,
|
||||||
reverse,
|
reverse,
|
||||||
move |new_range, new_limit| async move {
|
move |new_range, new_limit| async move {
|
||||||
let request =
|
let request =
|
||||||
new_scan_request(new_range, timestamp, new_limit, key_only, reverse);
|
new_scan_request(new_range, timestamp, new_limit, key_only, reverse);
|
||||||
let encoded_req = EncodedRequest::new(request, rpc.get_codec());
|
let plan = PlanBuilder::new(rpc, keyspace, request)
|
||||||
let plan = PlanBuilder::new(rpc, encoded_req)
|
.resolve_lock(retry_options.lock_backoff, keyspace)
|
||||||
.resolve_lock(retry_options.lock_backoff)
|
|
||||||
.retry_multi_region(retry_options.region_backoff)
|
.retry_multi_region(retry_options.region_backoff)
|
||||||
.merge(Collect)
|
.merge(Collect)
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -788,6 +806,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
},
|
},
|
||||||
)
|
)
|
||||||
.await
|
.await
|
||||||
|
.map(move |pairs| pairs.map(move |pair| pair.truncate_keyspace(keyspace)))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Pessimistically lock the keys, and optionally retrieve corresponding values.
|
/// Pessimistically lock the keys, and optionally retrieve corresponding values.
|
||||||
|
@ -832,9 +851,11 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
for_update_ts.clone(),
|
for_update_ts.clone(),
|
||||||
need_value,
|
need_value,
|
||||||
);
|
);
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, request)
|
||||||
let plan = PlanBuilder::new(self.rpc.clone(), encoded_req)
|
.resolve_lock(
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff.clone())
|
self.options.retry_options.lock_backoff.clone(),
|
||||||
|
self.keyspace,
|
||||||
|
)
|
||||||
.preserve_shard()
|
.preserve_shard()
|
||||||
.retry_multi_region_preserve_results(self.options.retry_options.region_backoff.clone())
|
.retry_multi_region_preserve_results(self.options.retry_options.region_backoff.clone())
|
||||||
.merge(CollectWithShard)
|
.merge(CollectWithShard)
|
||||||
|
@ -887,9 +908,11 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
start_version,
|
start_version,
|
||||||
for_update_ts,
|
for_update_ts,
|
||||||
);
|
);
|
||||||
let encoded_req = EncodedRequest::new(req, self.rpc.get_codec());
|
let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, req)
|
||||||
let plan = PlanBuilder::new(self.rpc.clone(), encoded_req)
|
.resolve_lock(
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff.clone())
|
self.options.retry_options.lock_backoff.clone(),
|
||||||
|
self.keyspace,
|
||||||
|
)
|
||||||
.retry_multi_region(self.options.retry_options.region_backoff.clone())
|
.retry_multi_region(self.options.retry_options.region_backoff.clone())
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -937,6 +960,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
HeartbeatOption::FixedTime(heartbeat_interval) => heartbeat_interval,
|
HeartbeatOption::FixedTime(heartbeat_interval) => heartbeat_interval,
|
||||||
};
|
};
|
||||||
let start_instant = self.start_instant;
|
let start_instant = self.start_instant;
|
||||||
|
let keyspace = self.keyspace;
|
||||||
|
|
||||||
let heartbeat_task = async move {
|
let heartbeat_task = async move {
|
||||||
loop {
|
loop {
|
||||||
|
@ -957,8 +981,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
primary_key.clone(),
|
primary_key.clone(),
|
||||||
start_instant.elapsed().as_millis() as u64 + MAX_TTL,
|
start_instant.elapsed().as_millis() as u64 + MAX_TTL,
|
||||||
);
|
);
|
||||||
let encoded_req = EncodedRequest::new(request, rpc.get_codec());
|
let plan = PlanBuilder::new(rpc.clone(), keyspace, request)
|
||||||
let plan = PlanBuilder::new(rpc.clone(), encoded_req)
|
|
||||||
.retry_multi_region(region_backoff.clone())
|
.retry_multi_region(region_backoff.clone())
|
||||||
.merge(CollectSingle)
|
.merge(CollectSingle)
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -1005,7 +1028,7 @@ impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Transaction<Cod, PdC> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<Cod: Codec, PdC: PdClient<Codec = Cod>> Drop for Transaction<Cod, PdC> {
|
impl<PdC: PdClient> Drop for Transaction<PdC> {
|
||||||
fn drop(&mut self) {
|
fn drop(&mut self) {
|
||||||
debug!("dropping transaction");
|
debug!("dropping transaction");
|
||||||
if std::thread::panicking() {
|
if std::thread::panicking() {
|
||||||
|
@ -1204,6 +1227,21 @@ impl HeartbeatOption {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[derive(Clone, Eq, PartialEq, Debug)]
|
||||||
|
pub enum Mutation {
|
||||||
|
Put(Key, Value),
|
||||||
|
Delete(Key),
|
||||||
|
}
|
||||||
|
|
||||||
|
impl Mutation {
|
||||||
|
pub fn key(&self) -> &Key {
|
||||||
|
match self {
|
||||||
|
Mutation::Put(key, _) => key,
|
||||||
|
Mutation::Delete(key) => key,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// A struct wrapping the details of two-phase commit protocol (2PC).
|
/// A struct wrapping the details of two-phase commit protocol (2PC).
|
||||||
///
|
///
|
||||||
/// The two phases are `prewrite` and `commit`.
|
/// The two phases are `prewrite` and `commit`.
|
||||||
|
@ -1219,6 +1257,7 @@ struct Committer<PdC: PdClient = PdRpcClient> {
|
||||||
start_version: Timestamp,
|
start_version: Timestamp,
|
||||||
rpc: Arc<PdC>,
|
rpc: Arc<PdC>,
|
||||||
options: TransactionOptions,
|
options: TransactionOptions,
|
||||||
|
keyspace: Keyspace,
|
||||||
#[new(default)]
|
#[new(default)]
|
||||||
undetermined: bool,
|
undetermined: bool,
|
||||||
write_size: u64,
|
write_size: u64,
|
||||||
|
@ -1296,9 +1335,11 @@ impl<PdC: PdClient> Committer<PdC> {
|
||||||
.collect();
|
.collect();
|
||||||
// FIXME set max_commit_ts and min_commit_ts
|
// FIXME set max_commit_ts and min_commit_ts
|
||||||
|
|
||||||
let encoded_req = EncodedRequest::new(request, self.rpc.get_codec());
|
let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, request)
|
||||||
let plan = PlanBuilder::new(self.rpc.clone(), encoded_req)
|
.resolve_lock(
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff.clone())
|
self.options.retry_options.lock_backoff.clone(),
|
||||||
|
self.keyspace,
|
||||||
|
)
|
||||||
.retry_multi_region(self.options.retry_options.region_backoff.clone())
|
.retry_multi_region(self.options.retry_options.region_backoff.clone())
|
||||||
.merge(CollectError)
|
.merge(CollectError)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
|
@ -1337,9 +1378,11 @@ impl<PdC: PdClient> Committer<PdC> {
|
||||||
self.start_version.clone(),
|
self.start_version.clone(),
|
||||||
commit_version.clone(),
|
commit_version.clone(),
|
||||||
);
|
);
|
||||||
let encoded_req = EncodedRequest::new(req, self.rpc.get_codec());
|
let plan = PlanBuilder::new(self.rpc.clone(), self.keyspace, req)
|
||||||
let plan = PlanBuilder::new(self.rpc.clone(), encoded_req)
|
.resolve_lock(
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff.clone())
|
self.options.retry_options.lock_backoff.clone(),
|
||||||
|
self.keyspace,
|
||||||
|
)
|
||||||
.retry_multi_region(self.options.retry_options.region_backoff.clone())
|
.retry_multi_region(self.options.retry_options.region_backoff.clone())
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -1402,9 +1445,8 @@ impl<PdC: PdClient> Committer<PdC> {
|
||||||
.filter(|key| &primary_key != key);
|
.filter(|key| &primary_key != key);
|
||||||
new_commit_request(keys, self.start_version, commit_version)
|
new_commit_request(keys, self.start_version, commit_version)
|
||||||
};
|
};
|
||||||
let encoded_req = EncodedRequest::new(req, self.rpc.get_codec());
|
let plan = PlanBuilder::new(self.rpc, self.keyspace, req)
|
||||||
let plan = PlanBuilder::new(self.rpc, encoded_req)
|
.resolve_lock(self.options.retry_options.lock_backoff, self.keyspace)
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff)
|
|
||||||
.retry_multi_region(self.options.retry_options.region_backoff)
|
.retry_multi_region(self.options.retry_options.region_backoff)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -1424,9 +1466,8 @@ impl<PdC: PdClient> Committer<PdC> {
|
||||||
match self.options.kind {
|
match self.options.kind {
|
||||||
TransactionKind::Optimistic => {
|
TransactionKind::Optimistic => {
|
||||||
let req = new_batch_rollback_request(keys, self.start_version);
|
let req = new_batch_rollback_request(keys, self.start_version);
|
||||||
let encoded_req = EncodedRequest::new(req, self.rpc.get_codec());
|
let plan = PlanBuilder::new(self.rpc, self.keyspace, req)
|
||||||
let plan = PlanBuilder::new(self.rpc, encoded_req)
|
.resolve_lock(self.options.retry_options.lock_backoff, self.keyspace)
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff)
|
|
||||||
.retry_multi_region(self.options.retry_options.region_backoff)
|
.retry_multi_region(self.options.retry_options.region_backoff)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -1434,9 +1475,8 @@ impl<PdC: PdClient> Committer<PdC> {
|
||||||
}
|
}
|
||||||
TransactionKind::Pessimistic(for_update_ts) => {
|
TransactionKind::Pessimistic(for_update_ts) => {
|
||||||
let req = new_pessimistic_rollback_request(keys, self.start_version, for_update_ts);
|
let req = new_pessimistic_rollback_request(keys, self.start_version, for_update_ts);
|
||||||
let encoded_req = EncodedRequest::new(req, self.rpc.get_codec());
|
let plan = PlanBuilder::new(self.rpc, self.keyspace, req)
|
||||||
let plan = PlanBuilder::new(self.rpc, encoded_req)
|
.resolve_lock(self.options.retry_options.lock_backoff, self.keyspace)
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff)
|
|
||||||
.retry_multi_region(self.options.retry_options.region_backoff)
|
.retry_multi_region(self.options.retry_options.region_backoff)
|
||||||
.extract_error()
|
.extract_error()
|
||||||
.plan();
|
.plan();
|
||||||
|
@ -1506,12 +1546,16 @@ mod tests {
|
||||||
use crate::mock::MockPdClient;
|
use crate::mock::MockPdClient;
|
||||||
use crate::proto::kvrpcpb;
|
use crate::proto::kvrpcpb;
|
||||||
use crate::proto::pdpb::Timestamp;
|
use crate::proto::pdpb::Timestamp;
|
||||||
|
use crate::request::Keyspace;
|
||||||
use crate::transaction::HeartbeatOption;
|
use crate::transaction::HeartbeatOption;
|
||||||
use crate::Transaction;
|
use crate::Transaction;
|
||||||
use crate::TransactionOptions;
|
use crate::TransactionOptions;
|
||||||
|
|
||||||
|
#[rstest::rstest]
|
||||||
|
#[case(Keyspace::Disable)]
|
||||||
|
#[case(Keyspace::Enable { keyspace_id: 0 })]
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
async fn test_optimistic_heartbeat() -> Result<(), io::Error> {
|
async fn test_optimistic_heartbeat(#[case] keyspace: Keyspace) -> Result<(), io::Error> {
|
||||||
let scenario = FailScenario::setup();
|
let scenario = FailScenario::setup();
|
||||||
fail::cfg("after-prewrite", "sleep(1500)").unwrap();
|
fail::cfg("after-prewrite", "sleep(1500)").unwrap();
|
||||||
let heartbeats = Arc::new(AtomicUsize::new(0));
|
let heartbeats = Arc::new(AtomicUsize::new(0));
|
||||||
|
@ -1534,6 +1578,7 @@ mod tests {
|
||||||
pd_client,
|
pd_client,
|
||||||
TransactionOptions::new_optimistic()
|
TransactionOptions::new_optimistic()
|
||||||
.heartbeat_option(HeartbeatOption::FixedTime(Duration::from_secs(1))),
|
.heartbeat_option(HeartbeatOption::FixedTime(Duration::from_secs(1))),
|
||||||
|
keyspace,
|
||||||
);
|
);
|
||||||
heartbeat_txn.put(key1.clone(), "foo").await.unwrap();
|
heartbeat_txn.put(key1.clone(), "foo").await.unwrap();
|
||||||
let heartbeat_txn_handle = tokio::task::spawn_blocking(move || {
|
let heartbeat_txn_handle = tokio::task::spawn_blocking(move || {
|
||||||
|
@ -1546,8 +1591,11 @@ mod tests {
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[rstest::rstest]
|
||||||
|
#[case(Keyspace::Disable)]
|
||||||
|
#[case(Keyspace::Enable { keyspace_id: 0 })]
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
async fn test_pessimistic_heartbeat() -> Result<(), io::Error> {
|
async fn test_pessimistic_heartbeat(#[case] keyspace: Keyspace) -> Result<(), io::Error> {
|
||||||
let heartbeats = Arc::new(AtomicUsize::new(0));
|
let heartbeats = Arc::new(AtomicUsize::new(0));
|
||||||
let heartbeats_cloned = heartbeats.clone();
|
let heartbeats_cloned = heartbeats.clone();
|
||||||
let pd_client = Arc::new(MockPdClient::new(MockKvClient::with_dispatch_hook(
|
let pd_client = Arc::new(MockPdClient::new(MockKvClient::with_dispatch_hook(
|
||||||
|
@ -1573,6 +1621,7 @@ mod tests {
|
||||||
pd_client,
|
pd_client,
|
||||||
TransactionOptions::new_pessimistic()
|
TransactionOptions::new_pessimistic()
|
||||||
.heartbeat_option(HeartbeatOption::FixedTime(Duration::from_secs(1))),
|
.heartbeat_option(HeartbeatOption::FixedTime(Duration::from_secs(1))),
|
||||||
|
keyspace,
|
||||||
);
|
);
|
||||||
heartbeat_txn.put(key1.clone(), "foo").await.unwrap();
|
heartbeat_txn.put(key1.clone(), "foo").await.unwrap();
|
||||||
assert_eq!(heartbeats.load(Ordering::SeqCst), 0);
|
assert_eq!(heartbeats.load(Ordering::SeqCst), 0);
|
||||||
|
|
|
@ -2,20 +2,20 @@
|
||||||
|
|
||||||
mod ctl;
|
mod ctl;
|
||||||
|
|
||||||
|
use log::info;
|
||||||
|
use log::warn;
|
||||||
|
use rand::Rng;
|
||||||
use std::collections::HashSet;
|
use std::collections::HashSet;
|
||||||
use std::convert::TryInto;
|
use std::convert::TryInto;
|
||||||
use std::env;
|
use std::env;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
|
use tikv_client::Config;
|
||||||
use log::info;
|
|
||||||
use log::warn;
|
|
||||||
use rand::Rng;
|
|
||||||
use tikv_client::Key;
|
use tikv_client::Key;
|
||||||
use tikv_client::RawClient;
|
use tikv_client::RawClient;
|
||||||
use tikv_client::Result;
|
use tikv_client::Result;
|
||||||
use tikv_client::Transaction;
|
use tikv_client::Transaction;
|
||||||
use tikv_client::TransactionClient;
|
use tikv_client::TransactionClient;
|
||||||
use tikv_client::{ColumnFamily, Snapshot, TransactionOptions};
|
use tikv_client::{Snapshot, TransactionOptions};
|
||||||
use tokio::time::sleep;
|
use tokio::time::sleep;
|
||||||
|
|
||||||
const ENV_PD_ADDRS: &str = "PD_ADDRS";
|
const ENV_PD_ADDRS: &str = "PD_ADDRS";
|
||||||
|
@ -24,21 +24,23 @@ const REGION_SPLIT_TIME_LIMIT: Duration = Duration::from_secs(15);
|
||||||
|
|
||||||
// Delete all entries in TiKV to leave a clean space for following tests.
|
// Delete all entries in TiKV to leave a clean space for following tests.
|
||||||
pub async fn clear_tikv() {
|
pub async fn clear_tikv() {
|
||||||
let cfs = vec![
|
|
||||||
ColumnFamily::Default,
|
|
||||||
ColumnFamily::Lock,
|
|
||||||
ColumnFamily::Write,
|
|
||||||
];
|
|
||||||
// DEFAULT_REGION_BACKOFF is not long enough for CI environment. So set a longer backoff.
|
// DEFAULT_REGION_BACKOFF is not long enough for CI environment. So set a longer backoff.
|
||||||
let backoff = tikv_client::Backoff::no_jitter_backoff(100, 30000, 20);
|
let backoff = tikv_client::Backoff::no_jitter_backoff(100, 30000, 20);
|
||||||
for cf in cfs {
|
let raw_client =
|
||||||
let raw_client = RawClient::new(pd_addrs()).await.unwrap().with_cf(cf);
|
RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
raw_client
|
|
||||||
.with_backoff(backoff.clone())
|
|
||||||
.delete_range(vec![]..)
|
|
||||||
.await
|
.await
|
||||||
.unwrap();
|
.unwrap();
|
||||||
}
|
raw_client
|
||||||
|
.with_backoff(backoff)
|
||||||
|
.delete_range(..)
|
||||||
|
.await
|
||||||
|
.unwrap();
|
||||||
|
|
||||||
|
let txn_client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await
|
||||||
|
.unwrap();
|
||||||
|
txn_client.unsafe_destroy_range(..).await.unwrap();
|
||||||
}
|
}
|
||||||
|
|
||||||
// To test with multiple regions, prewrite some data. Tests that hope to test
|
// To test with multiple regions, prewrite some data. Tests that hope to test
|
||||||
|
@ -78,14 +80,16 @@ async fn ensure_region_split(
|
||||||
// 1. write plenty transactional keys
|
// 1. write plenty transactional keys
|
||||||
// 2. wait until regions split
|
// 2. wait until regions split
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let mut txn = client.begin_optimistic().await?;
|
let mut txn = client.begin_optimistic().await?;
|
||||||
for key in keys.into_iter() {
|
for key in keys.into_iter() {
|
||||||
txn.put(key.into(), vec![0, 0, 0, 0]).await?;
|
txn.put(key.into(), vec![0, 0, 0, 0]).await?;
|
||||||
}
|
}
|
||||||
txn.commit().await?;
|
txn.commit().await?;
|
||||||
let mut txn = client.begin_optimistic().await?;
|
let mut txn = client.begin_optimistic().await?;
|
||||||
let _ = txn.scan(vec![].., 2048).await?;
|
let _ = txn.scan(.., 2048).await?;
|
||||||
txn.commit().await?;
|
txn.commit().await?;
|
||||||
|
|
||||||
info!("splitting regions...");
|
info!("splitting regions...");
|
||||||
|
|
|
@ -17,6 +17,7 @@ use tikv_client::transaction::HeartbeatOption;
|
||||||
use tikv_client::transaction::ResolveLocksOptions;
|
use tikv_client::transaction::ResolveLocksOptions;
|
||||||
use tikv_client::Backoff;
|
use tikv_client::Backoff;
|
||||||
use tikv_client::CheckLevel;
|
use tikv_client::CheckLevel;
|
||||||
|
use tikv_client::Config;
|
||||||
use tikv_client::Result;
|
use tikv_client::Result;
|
||||||
use tikv_client::RetryOptions;
|
use tikv_client::RetryOptions;
|
||||||
use tikv_client::TransactionClient;
|
use tikv_client::TransactionClient;
|
||||||
|
@ -34,7 +35,9 @@ async fn txn_optimistic_heartbeat() -> Result<()> {
|
||||||
|
|
||||||
let key1 = "key1".to_owned();
|
let key1 = "key1".to_owned();
|
||||||
let key2 = "key2".to_owned();
|
let key2 = "key2".to_owned();
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
// CheckLevel::Panic makes the case unstable, change to Warn level for now.
|
// CheckLevel::Panic makes the case unstable, change to Warn level for now.
|
||||||
// See https://github.com/tikv/client-rust/issues/389
|
// See https://github.com/tikv/client-rust/issues/389
|
||||||
|
@ -110,7 +113,9 @@ async fn txn_cleanup_locks_batch_size() -> Result<()> {
|
||||||
fail::cfg("before-cleanup-locks", "off").unwrap();
|
fail::cfg("before-cleanup-locks", "off").unwrap();
|
||||||
}}
|
}}
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let keys = write_data(&client, true, true).await?;
|
let keys = write_data(&client, true, true).await?;
|
||||||
assert_eq!(count_locks(&client).await?, keys.len());
|
assert_eq!(count_locks(&client).await?, keys.len());
|
||||||
|
|
||||||
|
@ -145,7 +150,11 @@ async fn txn_cleanup_async_commit_locks() -> Result<()> {
|
||||||
fail::cfg("after-prewrite", "off").unwrap()
|
fail::cfg("after-prewrite", "off").unwrap()
|
||||||
}
|
}
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client = TransactionClient::new_with_config(
|
||||||
|
pd_addrs(),
|
||||||
|
Config::default().with_default_keyspace(),
|
||||||
|
)
|
||||||
|
.await?;
|
||||||
let keys = write_data(&client, true, true).await?;
|
let keys = write_data(&client, true, true).await?;
|
||||||
assert_eq!(count_locks(&client).await?, keys.len());
|
assert_eq!(count_locks(&client).await?, keys.len());
|
||||||
|
|
||||||
|
@ -171,7 +180,11 @@ async fn txn_cleanup_async_commit_locks() -> Result<()> {
|
||||||
fail::cfg("before-commit-secondary", "off").unwrap()
|
fail::cfg("before-commit-secondary", "off").unwrap()
|
||||||
}
|
}
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client = TransactionClient::new_with_config(
|
||||||
|
pd_addrs(),
|
||||||
|
Config::default().with_default_keyspace(),
|
||||||
|
)
|
||||||
|
.await?;
|
||||||
let keys = write_data(&client, true, false).await?;
|
let keys = write_data(&client, true, false).await?;
|
||||||
thread::sleep(Duration::from_secs(1)); // Wait for async commit to complete.
|
thread::sleep(Duration::from_secs(1)); // Wait for async commit to complete.
|
||||||
assert_eq!(count_locks(&client).await?, keys.len() * percent / 100);
|
assert_eq!(count_locks(&client).await?, keys.len() * percent / 100);
|
||||||
|
@ -192,7 +205,11 @@ async fn txn_cleanup_async_commit_locks() -> Result<()> {
|
||||||
// all committed
|
// all committed
|
||||||
{
|
{
|
||||||
info!("test all committed");
|
info!("test all committed");
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client = TransactionClient::new_with_config(
|
||||||
|
pd_addrs(),
|
||||||
|
Config::default().with_default_keyspace(),
|
||||||
|
)
|
||||||
|
.await?;
|
||||||
let keys = write_data(&client, true, false).await?;
|
let keys = write_data(&client, true, false).await?;
|
||||||
|
|
||||||
let safepoint = client.current_timestamp().await?;
|
let safepoint = client.current_timestamp().await?;
|
||||||
|
@ -227,7 +244,9 @@ async fn txn_cleanup_range_async_commit_locks() -> Result<()> {
|
||||||
fail::cfg("after-prewrite", "off").unwrap()
|
fail::cfg("after-prewrite", "off").unwrap()
|
||||||
}
|
}
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let keys = write_data(&client, true, true).await?;
|
let keys = write_data(&client, true, true).await?;
|
||||||
assert_eq!(count_locks(&client).await?, keys.len());
|
assert_eq!(count_locks(&client).await?, keys.len());
|
||||||
|
|
||||||
|
@ -276,7 +295,11 @@ async fn txn_cleanup_2pc_locks() -> Result<()> {
|
||||||
fail::cfg("after-prewrite", "off").unwrap()
|
fail::cfg("after-prewrite", "off").unwrap()
|
||||||
}
|
}
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client = TransactionClient::new_with_config(
|
||||||
|
pd_addrs(),
|
||||||
|
Config::default().with_default_keyspace(),
|
||||||
|
)
|
||||||
|
.await?;
|
||||||
let keys = write_data(&client, false, true).await?;
|
let keys = write_data(&client, false, true).await?;
|
||||||
assert_eq!(count_locks(&client).await?, keys.len());
|
assert_eq!(count_locks(&client).await?, keys.len());
|
||||||
|
|
||||||
|
@ -306,7 +329,11 @@ async fn txn_cleanup_2pc_locks() -> Result<()> {
|
||||||
// all committed
|
// all committed
|
||||||
{
|
{
|
||||||
info!("test all committed");
|
info!("test all committed");
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client = TransactionClient::new_with_config(
|
||||||
|
pd_addrs(),
|
||||||
|
Config::default().with_default_keyspace(),
|
||||||
|
)
|
||||||
|
.await?;
|
||||||
let keys = write_data(&client, false, false).await?;
|
let keys = write_data(&client, false, false).await?;
|
||||||
assert_eq!(count_locks(&client).await?, 0);
|
assert_eq!(count_locks(&client).await?, 0);
|
||||||
|
|
||||||
|
@ -347,7 +374,7 @@ async fn must_rollbacked(client: &TransactionClient, keys: HashSet<Vec<u8>>) {
|
||||||
|
|
||||||
async fn count_locks(client: &TransactionClient) -> Result<usize> {
|
async fn count_locks(client: &TransactionClient) -> Result<usize> {
|
||||||
let ts = client.current_timestamp().await.unwrap();
|
let ts = client.current_timestamp().await.unwrap();
|
||||||
let locks = client.scan_locks(&ts, vec![].., 1024).await?;
|
let locks = client.scan_locks(&ts, .., 1024).await?;
|
||||||
// De-duplicated as `scan_locks` will return duplicated locks due to retry on region changes.
|
// De-duplicated as `scan_locks` will return duplicated locks due to retry on region changes.
|
||||||
let locks_set: HashSet<Vec<u8>> = HashSet::from_iter(locks.into_iter().map(|l| l.key));
|
let locks_set: HashSet<Vec<u8>> = HashSet::from_iter(locks.into_iter().map(|l| l.key));
|
||||||
Ok(locks_set.len())
|
Ok(locks_set.len())
|
||||||
|
|
|
@ -12,18 +12,18 @@
|
||||||
//! requirements on the region boundaries.
|
//! requirements on the region boundaries.
|
||||||
|
|
||||||
mod common;
|
mod common;
|
||||||
use std::collections::HashMap;
|
|
||||||
use std::iter;
|
|
||||||
|
|
||||||
use common::*;
|
use common::*;
|
||||||
use futures::prelude::*;
|
use futures::prelude::*;
|
||||||
use rand::seq::IteratorRandom;
|
use rand::seq::IteratorRandom;
|
||||||
use rand::thread_rng;
|
use rand::thread_rng;
|
||||||
use rand::Rng;
|
use rand::Rng;
|
||||||
use serial_test::serial;
|
use serial_test::serial;
|
||||||
|
use std::collections::HashMap;
|
||||||
|
use std::iter;
|
||||||
use tikv_client::backoff::DEFAULT_REGION_BACKOFF;
|
use tikv_client::backoff::DEFAULT_REGION_BACKOFF;
|
||||||
use tikv_client::proto::kvrpcpb;
|
|
||||||
use tikv_client::transaction::HeartbeatOption;
|
use tikv_client::transaction::HeartbeatOption;
|
||||||
|
use tikv_client::transaction::Mutation;
|
||||||
|
use tikv_client::Config;
|
||||||
use tikv_client::Error;
|
use tikv_client::Error;
|
||||||
use tikv_client::Key;
|
use tikv_client::Key;
|
||||||
use tikv_client::KvPair;
|
use tikv_client::KvPair;
|
||||||
|
@ -42,7 +42,9 @@ const NUM_TRNASFER: u32 = 100;
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_get_timestamp() -> Result<()> {
|
async fn txn_get_timestamp() -> Result<()> {
|
||||||
const COUNT: usize = 1 << 8; // use a small number to make test fast
|
const COUNT: usize = 1 << 8; // use a small number to make test fast
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
let mut versions = future::join_all((0..COUNT).map(|_| client.current_timestamp()))
|
let mut versions = future::join_all((0..COUNT).map(|_| client.current_timestamp()))
|
||||||
.await
|
.await
|
||||||
|
@ -63,7 +65,9 @@ async fn txn_get_timestamp() -> Result<()> {
|
||||||
async fn txn_crud() -> Result<()> {
|
async fn txn_crud() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let mut txn = client.begin_optimistic().await?;
|
let mut txn = client.begin_optimistic().await?;
|
||||||
|
|
||||||
// Get non-existent keys
|
// Get non-existent keys
|
||||||
|
@ -147,7 +151,9 @@ async fn txn_crud() -> Result<()> {
|
||||||
async fn txn_insert_duplicate_keys() -> Result<()> {
|
async fn txn_insert_duplicate_keys() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
// Initialize TiKV store with {foo => bar}
|
// Initialize TiKV store with {foo => bar}
|
||||||
let mut txn = client.begin_optimistic().await?;
|
let mut txn = client.begin_optimistic().await?;
|
||||||
txn.put("foo".to_owned(), "bar".to_owned()).await?;
|
txn.put("foo".to_owned(), "bar".to_owned()).await?;
|
||||||
|
@ -171,7 +177,9 @@ async fn txn_insert_duplicate_keys() -> Result<()> {
|
||||||
async fn txn_pessimistic() -> Result<()> {
|
async fn txn_pessimistic() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let mut txn = client.begin_pessimistic().await?;
|
let mut txn = client.begin_pessimistic().await?;
|
||||||
txn.put("foo".to_owned(), "foo".to_owned()).await.unwrap();
|
txn.put("foo".to_owned(), "foo".to_owned()).await.unwrap();
|
||||||
|
|
||||||
|
@ -188,7 +196,9 @@ async fn txn_pessimistic() -> Result<()> {
|
||||||
async fn txn_split_batch() -> Result<()> {
|
async fn txn_split_batch() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
|
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let mut txn = client.begin_optimistic().await?;
|
let mut txn = client.begin_optimistic().await?;
|
||||||
let mut rng = thread_rng();
|
let mut rng = thread_rng();
|
||||||
|
|
||||||
|
@ -226,7 +236,8 @@ async fn txn_split_batch() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn raw_bank_transfer() -> Result<()> {
|
async fn raw_bank_transfer() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = RawClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?;
|
||||||
let mut rng = thread_rng();
|
let mut rng = thread_rng();
|
||||||
|
|
||||||
let people = gen_u32_keys(NUM_PEOPLE, &mut rng);
|
let people = gen_u32_keys(NUM_PEOPLE, &mut rng);
|
||||||
|
@ -278,7 +289,9 @@ async fn txn_read() -> Result<()> {
|
||||||
let value = "large_value".repeat(10);
|
let value = "large_value".repeat(10);
|
||||||
|
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
for i in 0..2u32.pow(NUM_BITS_TXN) {
|
for i in 0..2u32.pow(NUM_BITS_TXN) {
|
||||||
let mut cur = i * 2u32.pow(32 - NUM_BITS_TXN);
|
let mut cur = i * 2u32.pow(32 - NUM_BITS_TXN);
|
||||||
|
@ -370,7 +383,9 @@ async fn txn_read() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_bank_transfer() -> Result<()> {
|
async fn txn_bank_transfer() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let mut rng = thread_rng();
|
let mut rng = thread_rng();
|
||||||
let options = TransactionOptions::new_optimistic()
|
let options = TransactionOptions::new_optimistic()
|
||||||
.use_async_commit()
|
.use_async_commit()
|
||||||
|
@ -423,7 +438,8 @@ async fn txn_bank_transfer() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn raw_req() -> Result<()> {
|
async fn raw_req() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = RawClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?;
|
||||||
|
|
||||||
// empty; get non-existent key
|
// empty; get non-existent key
|
||||||
let res = client.get("k1".to_owned()).await;
|
let res = client.get("k1".to_owned()).await;
|
||||||
|
@ -553,7 +569,9 @@ async fn raw_req() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_update_safepoint() -> Result<()> {
|
async fn txn_update_safepoint() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let res = client.gc(client.current_timestamp().await?).await?;
|
let res = client.gc(client.current_timestamp().await?).await?;
|
||||||
assert!(res);
|
assert!(res);
|
||||||
Ok(())
|
Ok(())
|
||||||
|
@ -568,7 +586,8 @@ async fn raw_write_million() -> Result<()> {
|
||||||
let interval = 2u32.pow(32 - NUM_BITS_TXN - NUM_BITS_KEY_PER_TXN);
|
let interval = 2u32.pow(32 - NUM_BITS_TXN - NUM_BITS_KEY_PER_TXN);
|
||||||
|
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = RawClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?;
|
||||||
|
|
||||||
for i in 0..2u32.pow(NUM_BITS_TXN) {
|
for i in 0..2u32.pow(NUM_BITS_TXN) {
|
||||||
let mut cur = i * 2u32.pow(32 - NUM_BITS_TXN);
|
let mut cur = i * 2u32.pow(32 - NUM_BITS_TXN);
|
||||||
|
@ -702,7 +721,9 @@ async fn raw_write_million() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_pessimistic_rollback() -> Result<()> {
|
async fn txn_pessimistic_rollback() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let mut preload_txn = client.begin_optimistic().await?;
|
let mut preload_txn = client.begin_optimistic().await?;
|
||||||
let key1 = vec![1];
|
let key1 = vec![1];
|
||||||
let key2 = vec![2];
|
let key2 = vec![2];
|
||||||
|
@ -734,7 +755,9 @@ async fn txn_pessimistic_rollback() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_pessimistic_delete() -> Result<()> {
|
async fn txn_pessimistic_delete() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
// The transaction will lock the keys and must release the locks on commit,
|
// The transaction will lock the keys and must release the locks on commit,
|
||||||
// even when values are not written to the DB.
|
// even when values are not written to the DB.
|
||||||
|
@ -785,7 +808,9 @@ async fn txn_pessimistic_delete() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_lock_keys() -> Result<()> {
|
async fn txn_lock_keys() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
let k1 = b"key1".to_vec();
|
let k1 = b"key1".to_vec();
|
||||||
let k2 = b"key2".to_vec();
|
let k2 = b"key2".to_vec();
|
||||||
|
@ -819,7 +844,9 @@ async fn txn_lock_keys() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_lock_keys_error_handle() -> Result<()> {
|
async fn txn_lock_keys_error_handle() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
// Keys in `k` should locate in different regions. See `init()` for boundary of regions.
|
// Keys in `k` should locate in different regions. See `init()` for boundary of regions.
|
||||||
let k: Vec<Key> = vec![
|
let k: Vec<Key> = vec![
|
||||||
|
@ -856,7 +883,9 @@ async fn txn_lock_keys_error_handle() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_get_for_update() -> Result<()> {
|
async fn txn_get_for_update() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let key1 = "key".to_owned();
|
let key1 = "key".to_owned();
|
||||||
let key2 = "another key".to_owned();
|
let key2 = "another key".to_owned();
|
||||||
let value1 = b"some value".to_owned();
|
let value1 = b"some value".to_owned();
|
||||||
|
@ -903,7 +932,9 @@ async fn txn_pessimistic_heartbeat() -> Result<()> {
|
||||||
|
|
||||||
let key1 = "key1".to_owned();
|
let key1 = "key1".to_owned();
|
||||||
let key2 = "key2".to_owned();
|
let key2 = "key2".to_owned();
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
let mut heartbeat_txn = client
|
let mut heartbeat_txn = client
|
||||||
.begin_with_options(TransactionOptions::new_pessimistic())
|
.begin_with_options(TransactionOptions::new_pessimistic())
|
||||||
|
@ -943,7 +974,9 @@ async fn txn_pessimistic_heartbeat() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn raw_cas() -> Result<()> {
|
async fn raw_cas() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = RawClient::new(pd_addrs()).await?.with_atomic_for_cas();
|
let client = RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?
|
||||||
|
.with_atomic_for_cas();
|
||||||
let key = "key".to_owned();
|
let key = "key".to_owned();
|
||||||
let value = "value".to_owned();
|
let value = "value".to_owned();
|
||||||
let new_value = "new value".to_owned();
|
let new_value = "new value".to_owned();
|
||||||
|
@ -986,7 +1019,8 @@ async fn raw_cas() -> Result<()> {
|
||||||
client.batch_delete(vec![key.clone()]).await.err().unwrap(),
|
client.batch_delete(vec![key.clone()]).await.err().unwrap(),
|
||||||
Error::UnsupportedMode
|
Error::UnsupportedMode
|
||||||
));
|
));
|
||||||
let client = RawClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
RawClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace()).await?;
|
||||||
assert!(matches!(
|
assert!(matches!(
|
||||||
client
|
client
|
||||||
.compare_and_swap(key.clone(), None, vec![])
|
.compare_and_swap(key.clone(), None, vec![])
|
||||||
|
@ -1003,7 +1037,9 @@ async fn raw_cas() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_scan() -> Result<()> {
|
async fn txn_scan() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
let k1 = b"a".to_vec();
|
let k1 = b"a".to_vec();
|
||||||
let v = b"b".to_vec();
|
let v = b"b".to_vec();
|
||||||
|
@ -1026,7 +1062,9 @@ async fn txn_scan() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_scan_reverse() -> Result<()> {
|
async fn txn_scan_reverse() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
let k1 = b"k1".to_vec();
|
let k1 = b"k1".to_vec();
|
||||||
let k2 = b"k2".to_vec();
|
let k2 = b"k2".to_vec();
|
||||||
|
@ -1099,7 +1137,9 @@ async fn txn_scan_reverse() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_scan_reverse_multi_regions() -> Result<()> {
|
async fn txn_scan_reverse_multi_regions() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
// Keys in `keys` should locate in different regions. See `init()` for boundary of regions.
|
// Keys in `keys` should locate in different regions. See `init()` for boundary of regions.
|
||||||
let keys: Vec<Key> = vec![
|
let keys: Vec<Key> = vec![
|
||||||
|
@ -1143,7 +1183,9 @@ async fn txn_scan_reverse_multi_regions() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_key_exists() -> Result<()> {
|
async fn txn_key_exists() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let key = "key".to_owned();
|
let key = "key".to_owned();
|
||||||
let value = "value".to_owned();
|
let value = "value".to_owned();
|
||||||
let mut t1 = client.begin_optimistic().await?;
|
let mut t1 = client.begin_optimistic().await?;
|
||||||
|
@ -1166,7 +1208,9 @@ async fn txn_key_exists() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_batch_mutate_optimistic() -> Result<()> {
|
async fn txn_batch_mutate_optimistic() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
// Put k0
|
// Put k0
|
||||||
{
|
{
|
||||||
|
@ -1177,7 +1221,7 @@ async fn txn_batch_mutate_optimistic() -> Result<()> {
|
||||||
// Delete k0 and put k1, k2
|
// Delete k0 and put k1, k2
|
||||||
do_mutate(false).await.unwrap();
|
do_mutate(false).await.unwrap();
|
||||||
// Read and verify
|
// Read and verify
|
||||||
verify_mutate(false).await;
|
verify_mutate(false).await?;
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1185,7 +1229,9 @@ async fn txn_batch_mutate_optimistic() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_batch_mutate_pessimistic() -> Result<()> {
|
async fn txn_batch_mutate_pessimistic() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new(pd_addrs()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
// Put k0
|
// Put k0
|
||||||
{
|
{
|
||||||
|
@ -1210,7 +1256,7 @@ async fn txn_batch_mutate_pessimistic() -> Result<()> {
|
||||||
txn3_handle.await?.unwrap();
|
txn3_handle.await?.unwrap();
|
||||||
|
|
||||||
// Read and verify
|
// Read and verify
|
||||||
verify_mutate(true).await;
|
verify_mutate(true).await?;
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1227,27 +1273,15 @@ async fn begin_mutate(client: &TransactionClient, is_pessimistic: bool) -> Resul
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn do_mutate(is_pessimistic: bool) -> Result<()> {
|
async fn do_mutate(is_pessimistic: bool) -> Result<()> {
|
||||||
let client = TransactionClient::new(pd_addrs()).await.unwrap();
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let mut txn = begin_mutate(&client, is_pessimistic).await.unwrap();
|
let mut txn = begin_mutate(&client, is_pessimistic).await.unwrap();
|
||||||
|
|
||||||
let mutations = vec![
|
let mutations = vec![
|
||||||
kvrpcpb::Mutation {
|
Mutation::Delete(Key::from("k0".to_owned())),
|
||||||
op: kvrpcpb::Op::Del.into(),
|
Mutation::Put(Key::from("k1".to_owned()), Value::from("v1".to_owned())),
|
||||||
key: b"k0".to_vec(),
|
Mutation::Put(Key::from("k2".to_owned()), Value::from("v2".to_owned())),
|
||||||
..Default::default()
|
|
||||||
},
|
|
||||||
kvrpcpb::Mutation {
|
|
||||||
op: kvrpcpb::Op::Put.into(),
|
|
||||||
key: b"k1".to_vec(),
|
|
||||||
value: b"v1".to_vec(),
|
|
||||||
..Default::default()
|
|
||||||
},
|
|
||||||
kvrpcpb::Mutation {
|
|
||||||
op: kvrpcpb::Op::Put.into(),
|
|
||||||
key: b"k2".to_vec(),
|
|
||||||
value: b"v2".to_vec(),
|
|
||||||
..Default::default()
|
|
||||||
},
|
|
||||||
];
|
];
|
||||||
|
|
||||||
match txn.batch_mutate(mutations).await {
|
match txn.batch_mutate(mutations).await {
|
||||||
|
@ -1262,8 +1296,10 @@ async fn do_mutate(is_pessimistic: bool) -> Result<()> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn verify_mutate(is_pessimistic: bool) {
|
async fn verify_mutate(is_pessimistic: bool) -> Result<()> {
|
||||||
let client = TransactionClient::new(pd_addrs()).await.unwrap();
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
let mut snapshot = snapshot(&client, is_pessimistic).await.unwrap();
|
let mut snapshot = snapshot(&client, is_pessimistic).await.unwrap();
|
||||||
let res: HashMap<Key, Value> = snapshot
|
let res: HashMap<Key, Value> = snapshot
|
||||||
.batch_get(vec!["k0".to_owned(), "k1".to_owned(), "k2".to_owned()])
|
.batch_get(vec!["k0".to_owned(), "k1".to_owned(), "k2".to_owned()])
|
||||||
|
@ -1280,13 +1316,16 @@ async fn verify_mutate(is_pessimistic: bool) {
|
||||||
res.get(&Key::from("k2".to_owned())),
|
res.get(&Key::from("k2".to_owned())),
|
||||||
Some(Value::from("v2".to_owned())).as_ref()
|
Some(Value::from("v2".to_owned())).as_ref()
|
||||||
);
|
);
|
||||||
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn txn_unsafe_destroy_range() -> Result<()> {
|
async fn txn_unsafe_destroy_range() -> Result<()> {
|
||||||
init().await?;
|
init().await?;
|
||||||
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
let client =
|
||||||
|
TransactionClient::new_with_config(pd_addrs(), Config::default().with_default_keyspace())
|
||||||
|
.await?;
|
||||||
|
|
||||||
const DATA_COUNT: usize = 10;
|
const DATA_COUNT: usize = 10;
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue