implement the complete keyspace feature (#439)

Signed-off-by: Andy Lok <andylokandy@hotmail.com>
This commit is contained in:
Andy Lok 2023-12-27 18:12:40 +08:00 committed by GitHub
parent bbaf317b05
commit 1178d79ed6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 953 additions and 562 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,54 +0,0 @@
// Copyright 2023 TiKV Project Authors. Licensed under Apache-2.0.
use crate::proto::kvrpcpb;
use crate::request::KvRequest;
pub trait Codec: Clone + Sync + Send + 'static {
fn encode_request<R: KvRequest>(&self, _req: &mut R) {}
// TODO: fn decode_response()
}
#[derive(Clone, Default)]
pub struct ApiV1TxnCodec {}
impl Codec for ApiV1TxnCodec {}
#[derive(Clone, Default)]
pub struct ApiV1RawCodec {}
impl Codec for ApiV1RawCodec {}
#[derive(Clone)]
pub struct ApiV2TxnCodec {
_keyspace_id: u32,
}
impl ApiV2TxnCodec {
pub fn new(keyspace_id: u32) -> Self {
Self {
_keyspace_id: keyspace_id,
}
}
}
impl Codec for ApiV2TxnCodec {
fn encode_request<R: KvRequest>(&self, req: &mut R) {
req.set_api_version(kvrpcpb::ApiVersion::V2);
// TODO: req.encode_request(self);
}
}
// TODO: pub struct ApiV2RawCodec
// EncodeRequest is just a type wrapper to avoid passing not encoded request to `PlanBuilder` by mistake.
#[derive(Clone)]
pub struct EncodedRequest<Req: KvRequest> {
pub inner: Req,
}
impl<Req: KvRequest> EncodedRequest<Req> {
pub fn new<C: Codec>(mut req: Req, codec: &C) -> Self {
codec.encode_request(&mut req);
Self { inner: req }
}
}

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

@ -0,0 +1,280 @@
// Copyright 2023 TiKV Project Authors. Licensed under Apache-2.0.
use std::ops::{Bound, Range};
use serde_derive::{Deserialize, Serialize};
use crate::transaction::Mutation;
use crate::{proto::kvrpcpb, Key};
use crate::{BoundRange, KvPair};
pub const RAW_KEY_PREFIX: u8 = b'r';
pub const TXN_KEY_PREFIX: u8 = b'x';
pub const KEYSPACE_PREFIX_LEN: usize = 4;
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize)]
pub enum Keyspace {
Disable,
Enable { keyspace_id: u32 },
}
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize)]
pub enum KeyMode {
Raw,
Txn,
}
impl Keyspace {
pub fn api_version(&self) -> kvrpcpb::ApiVersion {
match self {
Keyspace::Disable => kvrpcpb::ApiVersion::V1,
Keyspace::Enable { .. } => kvrpcpb::ApiVersion::V2,
}
}
}
pub trait EncodeKeyspace {
fn encode_keyspace(self, keyspace: Keyspace, key_mode: KeyMode) -> Self;
}
pub trait TruncateKeyspace {
fn truncate_keyspace(self, keyspace: Keyspace) -> Self;
}
impl EncodeKeyspace for Key {
fn encode_keyspace(mut self, keyspace: Keyspace, key_mode: KeyMode) -> Self {
let prefix = match keyspace {
Keyspace::Disable => {
return self;
}
Keyspace::Enable { keyspace_id } => keyspace_prefix(keyspace_id, key_mode),
};
prepend_bytes(&mut self.0, &prefix);
self
}
}
impl EncodeKeyspace for KvPair {
fn encode_keyspace(mut self, keyspace: Keyspace, key_mode: KeyMode) -> Self {
self.0 = self.0.encode_keyspace(keyspace, key_mode);
self
}
}
impl EncodeKeyspace for BoundRange {
fn encode_keyspace(mut self, keyspace: Keyspace, key_mode: KeyMode) -> Self {
self.from = match self.from {
Bound::Included(key) => Bound::Included(key.encode_keyspace(keyspace, key_mode)),
Bound::Excluded(key) => Bound::Excluded(key.encode_keyspace(keyspace, key_mode)),
Bound::Unbounded => {
let key = Key::from(vec![]);
Bound::Included(key.encode_keyspace(keyspace, key_mode))
}
};
self.to = match self.to {
Bound::Included(key) if !key.is_empty() => {
Bound::Included(key.encode_keyspace(keyspace, key_mode))
}
Bound::Excluded(key) if !key.is_empty() => {
Bound::Excluded(key.encode_keyspace(keyspace, key_mode))
}
_ => {
let key = Key::from(vec![]);
let keyspace = match keyspace {
Keyspace::Disable => Keyspace::Disable,
Keyspace::Enable { keyspace_id } => Keyspace::Enable {
keyspace_id: keyspace_id + 1,
},
};
Bound::Excluded(key.encode_keyspace(keyspace, key_mode))
}
};
self
}
}
impl EncodeKeyspace for Mutation {
fn encode_keyspace(self, keyspace: Keyspace, key_mode: KeyMode) -> Self {
match self {
Mutation::Put(key, val) => Mutation::Put(key.encode_keyspace(keyspace, key_mode), val),
Mutation::Delete(key) => Mutation::Delete(key.encode_keyspace(keyspace, key_mode)),
}
}
}
impl TruncateKeyspace for Key {
fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self {
if let Keyspace::Disable = keyspace {
return self;
}
pretruncate_bytes::<KEYSPACE_PREFIX_LEN>(&mut self.0);
self
}
}
impl TruncateKeyspace for KvPair {
fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self {
self.0 = self.0.truncate_keyspace(keyspace);
self
}
}
impl TruncateKeyspace for Range<Key> {
fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self {
self.start = self.start.truncate_keyspace(keyspace);
self.end = self.end.truncate_keyspace(keyspace);
self
}
}
impl TruncateKeyspace for Vec<Range<Key>> {
fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self {
for range in &mut self {
take_mut::take(range, |range| range.truncate_keyspace(keyspace));
}
self
}
}
impl TruncateKeyspace for Vec<KvPair> {
fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self {
for pair in &mut self {
take_mut::take(pair, |pair| pair.truncate_keyspace(keyspace));
}
self
}
}
impl TruncateKeyspace for Vec<crate::proto::kvrpcpb::LockInfo> {
fn truncate_keyspace(mut self, keyspace: Keyspace) -> Self {
for lock in &mut self {
take_mut::take(&mut lock.key, |key| {
Key::from(key).truncate_keyspace(keyspace).into()
});
take_mut::take(&mut lock.primary_lock, |primary| {
Key::from(primary).truncate_keyspace(keyspace).into()
});
for secondary in lock.secondaries.iter_mut() {
take_mut::take(secondary, |secondary| {
Key::from(secondary).truncate_keyspace(keyspace).into()
});
}
}
self
}
}
fn keyspace_prefix(keyspace_id: u32, key_mode: KeyMode) -> [u8; KEYSPACE_PREFIX_LEN] {
let mut prefix = keyspace_id.to_be_bytes();
prefix[0] = match key_mode {
KeyMode::Raw => RAW_KEY_PREFIX,
KeyMode::Txn => TXN_KEY_PREFIX,
};
prefix
}
fn prepend_bytes<const N: usize>(vec: &mut Vec<u8>, prefix: &[u8; N]) {
unsafe {
vec.reserve_exact(N);
std::ptr::copy(vec.as_ptr(), vec.as_mut_ptr().add(N), vec.len());
std::ptr::copy_nonoverlapping(prefix.as_ptr(), vec.as_mut_ptr(), N);
vec.set_len(vec.len() + N);
}
}
fn pretruncate_bytes<const N: usize>(vec: &mut Vec<u8>) {
assert!(vec.len() >= N);
unsafe {
std::ptr::copy(vec.as_ptr().add(N), vec.as_mut_ptr(), vec.len() - N);
vec.set_len(vec.len() - N);
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_keyspace_prefix() {
let key_mode = KeyMode::Raw;
assert_eq!(keyspace_prefix(0, key_mode), [b'r', 0, 0, 0]);
assert_eq!(keyspace_prefix(1, key_mode), [b'r', 0, 0, 1]);
assert_eq!(keyspace_prefix(0xFFFF, key_mode), [b'r', 0, 0xFF, 0xFF]);
let key_mode = KeyMode::Txn;
assert_eq!(keyspace_prefix(0, key_mode), [b'x', 0, 0, 0]);
assert_eq!(keyspace_prefix(1, key_mode), [b'x', 0, 0, 1]);
assert_eq!(keyspace_prefix(0xFFFF, key_mode), [b'x', 0, 0xFF, 0xFF]);
}
#[test]
fn test_encode_version() {
let keyspace = Keyspace::Enable {
keyspace_id: 0xDEAD,
};
let key_mode = KeyMode::Raw;
let key = Key::from(vec![0xBE, 0xEF]);
let expected_key = Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF]);
assert_eq!(key.encode_keyspace(keyspace, key_mode), expected_key);
let bound: BoundRange = (Key::from(vec![0xDE, 0xAD])..Key::from(vec![0xBE, 0xEF])).into();
let expected_bound: BoundRange = (Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xDE, 0xAD])
..Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF]))
.into();
assert_eq!(bound.encode_keyspace(keyspace, key_mode), expected_bound);
let bound: BoundRange = (..).into();
let expected_bound: BoundRange =
(Key::from(vec![b'r', 0, 0xDE, 0xAD])..Key::from(vec![b'r', 0, 0xDE, 0xAE])).into();
assert_eq!(bound.encode_keyspace(keyspace, key_mode), expected_bound);
let bound: BoundRange = (Key::from(vec![])..Key::from(vec![])).into();
let expected_bound: BoundRange =
(Key::from(vec![b'r', 0, 0xDE, 0xAD])..Key::from(vec![b'r', 0, 0xDE, 0xAE])).into();
assert_eq!(bound.encode_keyspace(keyspace, key_mode), expected_bound);
let bound: BoundRange = (Key::from(vec![])..=Key::from(vec![])).into();
let expected_bound: BoundRange =
(Key::from(vec![b'r', 0, 0xDE, 0xAD])..Key::from(vec![b'r', 0, 0xDE, 0xAE])).into();
assert_eq!(bound.encode_keyspace(keyspace, key_mode), expected_bound);
let mutation = Mutation::Put(Key::from(vec![0xBE, 0xEF]), vec![4, 5, 6]);
let expected_mutation = Mutation::Put(
Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF]),
vec![4, 5, 6],
);
assert_eq!(
mutation.encode_keyspace(keyspace, key_mode),
expected_mutation
);
let mutation = Mutation::Delete(Key::from(vec![0xBE, 0xEF]));
let expected_mutation = Mutation::Delete(Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF]));
assert_eq!(
mutation.encode_keyspace(keyspace, key_mode),
expected_mutation
);
}
#[test]
fn test_truncate_version() {
let key = Key::from(vec![b'r', 0, 0xDE, 0xAD, 0xBE, 0xEF]);
let keyspace = Keyspace::Enable {
keyspace_id: 0xDEAD,
};
let expected_key = Key::from(vec![0xBE, 0xEF]);
assert_eq!(key.truncate_keyspace(keyspace), expected_key);
let key = Key::from(vec![b'x', 0, 0xDE, 0xAD, 0xBE, 0xEF]);
let keyspace = Keyspace::Enable {
keyspace_id: 0xDEAD,
};
let expected_key = Key::from(vec![0xBE, 0xEF]);
assert_eq!(key.truncate_keyspace(keyspace), expected_key);
}
}

View File

@ -3,6 +3,10 @@
use async_trait::async_trait; use 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();

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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