mirror of https://github.com/tikv/client-rust.git
feat: fix batch_get_for_update
Signed-off-by: ekexium <ekexium@gmail.com>
This commit is contained in:
parent
405535c777
commit
a3f814ab1f
|
@ -128,6 +128,10 @@ impl Key {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub trait HasKeys {
|
||||||
|
fn get_keys(&self) -> Vec<Key>;
|
||||||
|
}
|
||||||
|
|
||||||
impl From<Vec<u8>> for Key {
|
impl From<Vec<u8>> for Key {
|
||||||
fn from(v: Vec<u8>) -> Self {
|
fn from(v: Vec<u8>) -> Self {
|
||||||
Key(v)
|
Key(v)
|
||||||
|
|
|
@ -8,7 +8,7 @@ mod kvpair;
|
||||||
mod value;
|
mod value;
|
||||||
|
|
||||||
pub use bound_range::{BoundRange, IntoOwnedRange};
|
pub use bound_range::{BoundRange, IntoOwnedRange};
|
||||||
pub use key::Key;
|
pub use key::{HasKeys, Key};
|
||||||
pub use kvpair::KvPair;
|
pub use kvpair::KvPair;
|
||||||
pub use value::Value;
|
pub use value::Value;
|
||||||
|
|
||||||
|
|
|
@ -10,8 +10,8 @@ use tikv_client_store::{HasError, Request};
|
||||||
|
|
||||||
pub use self::{
|
pub use self::{
|
||||||
plan::{
|
plan::{
|
||||||
Collect, CollectError, DefaultProcessor, Dispatch, ExtractError, Merge, MergeResponse,
|
Collect, CollectAndMatchKey, CollectError, DefaultProcessor, Dispatch, ExtractError, Merge,
|
||||||
MultiRegion, Plan, Process, ProcessResponse, ResolveLock, RetryRegion,
|
MergeResponse, MultiRegion, Plan, Process, ProcessResponse, ResolveLock, RetryRegion,
|
||||||
},
|
},
|
||||||
plan_builder::{PlanBuilder, SingleKey},
|
plan_builder::{PlanBuilder, SingleKey},
|
||||||
shard::Shardable,
|
shard::Shardable,
|
||||||
|
|
|
@ -2,15 +2,18 @@
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
backoff::Backoff,
|
backoff::Backoff,
|
||||||
|
kv::HasKeys,
|
||||||
pd::PdClient,
|
pd::PdClient,
|
||||||
request::{KvRequest, Shardable},
|
request::{KvRequest, Shardable},
|
||||||
stats::tikv_stats,
|
stats::tikv_stats,
|
||||||
transaction::{resolve_locks, HasLocks},
|
transaction::{resolve_locks, HasLocks},
|
||||||
Error, Result,
|
util::iter::FlatMapOkIterExt,
|
||||||
|
Error, Key, KvPair, Result, Value,
|
||||||
};
|
};
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use futures::{prelude::*, stream::StreamExt};
|
use futures::{prelude::*, stream::StreamExt};
|
||||||
use std::{marker::PhantomData, sync::Arc};
|
use std::{marker::PhantomData, sync::Arc};
|
||||||
|
use tikv_client_proto::kvrpcpb;
|
||||||
use tikv_client_store::{HasError, HasRegionError, KvClient};
|
use tikv_client_store::{HasError, HasRegionError, KvClient};
|
||||||
|
|
||||||
/// 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
|
||||||
|
@ -55,6 +58,12 @@ impl<Req: KvRequest> Plan for Dispatch<Req> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<Req: KvRequest + HasKeys> HasKeys for Dispatch<Req> {
|
||||||
|
fn get_keys(&self) -> Vec<Key> {
|
||||||
|
self.request.get_keys()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
pub struct MultiRegion<P: Plan, PdC: PdClient> {
|
pub struct MultiRegion<P: Plan, PdC: PdClient> {
|
||||||
pub(super) inner: P,
|
pub(super) inner: P,
|
||||||
pub pd_client: Arc<PdC>,
|
pub pd_client: Arc<PdC>,
|
||||||
|
@ -123,6 +132,9 @@ impl<In: Clone + Send + Sync + 'static, P: Plan<Result = Vec<Result<In>>>, M: Me
|
||||||
#[derive(Clone, Copy)]
|
#[derive(Clone, Copy)]
|
||||||
pub struct Collect;
|
pub struct Collect;
|
||||||
|
|
||||||
|
#[derive(Clone, Debug)]
|
||||||
|
pub struct CollectAndMatchKey;
|
||||||
|
|
||||||
/// A merge strategy which returns an error if any response is an error and
|
/// A merge strategy which returns an error if any response is an error and
|
||||||
/// otherwise returns a Vec of the results.
|
/// otherwise returns a Vec of the results.
|
||||||
#[derive(Clone, Copy)]
|
#[derive(Clone, Copy)]
|
||||||
|
@ -256,6 +268,12 @@ where
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<P: Plan + HasKeys, PdC: PdClient> HasKeys for ResolveLock<P, PdC> {
|
||||||
|
fn get_keys(&self) -> Vec<Key> {
|
||||||
|
self.inner.get_keys()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
pub struct ExtractError<P: Plan> {
|
pub struct ExtractError<P: Plan> {
|
||||||
pub inner: P,
|
pub inner: P,
|
||||||
}
|
}
|
||||||
|
@ -292,6 +310,34 @@ where
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Requires: len(inner.keys) == len(inner.result)
|
||||||
|
pub struct PreserveKey<P: Plan + HasKeys> {
|
||||||
|
pub inner: P,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<P: Plan + HasKeys> Clone for PreserveKey<P> {
|
||||||
|
fn clone(&self) -> Self {
|
||||||
|
PreserveKey {
|
||||||
|
inner: self.inner.clone(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
#[async_trait]
|
||||||
|
impl<P> Plan for PreserveKey<P>
|
||||||
|
where
|
||||||
|
P: Plan + HasKeys,
|
||||||
|
{
|
||||||
|
type Result = ResponseAndKeys<P::Result>;
|
||||||
|
|
||||||
|
async fn execute(&self) -> Result<Self::Result> {
|
||||||
|
let keys = self.inner.get_keys();
|
||||||
|
let res = self.inner.execute().await?;
|
||||||
|
// TODO: should we check they have the same length?
|
||||||
|
Ok(ResponseAndKeys(res, keys))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod test {
|
mod test {
|
||||||
use super::*;
|
use super::*;
|
||||||
|
@ -349,3 +395,60 @@ mod test {
|
||||||
.for_each(|r| assert!(r.is_err()));
|
.for_each(|r| assert!(r.is_err()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// contains a response and the corresponding keys
|
||||||
|
// currently only used for matching keys and values in pessimistic lock requests
|
||||||
|
#[derive(Debug, Clone)]
|
||||||
|
pub struct ResponseAndKeys<Resp>(Resp, Vec<Key>);
|
||||||
|
|
||||||
|
impl<Resp: HasError> HasError for ResponseAndKeys<Resp> {
|
||||||
|
fn error(&mut self) -> Option<Error> {
|
||||||
|
self.0.error()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<Resp: HasLocks> HasLocks for ResponseAndKeys<Resp> {
|
||||||
|
fn take_locks(&mut self) -> Vec<tikv_client_proto::kvrpcpb::LockInfo> {
|
||||||
|
self.0.take_locks()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<Resp: HasRegionError> HasRegionError for ResponseAndKeys<Resp> {
|
||||||
|
fn region_error(&mut self) -> Option<Error> {
|
||||||
|
self.0.region_error()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl Merge<ResponseAndKeys<kvrpcpb::PessimisticLockResponse>> for CollectAndMatchKey {
|
||||||
|
type Out = Vec<KvPair>;
|
||||||
|
|
||||||
|
fn merge(
|
||||||
|
&self,
|
||||||
|
input: Vec<Result<ResponseAndKeys<kvrpcpb::PessimisticLockResponse>>>,
|
||||||
|
) -> Result<Self::Out> {
|
||||||
|
input
|
||||||
|
.into_iter()
|
||||||
|
.flat_map_ok(|ResponseAndKeys(mut resp, keys)| {
|
||||||
|
let values = resp.take_values();
|
||||||
|
let not_founds = resp.take_not_founds();
|
||||||
|
let v: Vec<_> = if not_founds.is_empty() {
|
||||||
|
// Legacy TiKV does not distiguish not existing key and existing key
|
||||||
|
// that with empty value. We assume that key does not exist if value
|
||||||
|
// is empty.
|
||||||
|
let values: Vec<Value> = values.into_iter().filter(|v| v.is_empty()).collect();
|
||||||
|
keys.into_iter().zip(values).map(From::from).collect()
|
||||||
|
} else {
|
||||||
|
assert_eq!(values.len(), not_founds.len());
|
||||||
|
let values: Vec<Value> = values
|
||||||
|
.into_iter()
|
||||||
|
.zip(not_founds.into_iter())
|
||||||
|
.filter_map(|(v, not_found)| if not_found { None } else { Some(v) })
|
||||||
|
.collect();
|
||||||
|
keys.into_iter().zip(values).map(From::from).collect()
|
||||||
|
};
|
||||||
|
// FIXME sucks to collect and re-iterate, but the iterators have different types
|
||||||
|
v.into_iter()
|
||||||
|
})
|
||||||
|
.collect()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -2,6 +2,7 @@
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
backoff::Backoff,
|
backoff::Backoff,
|
||||||
|
kv::HasKeys,
|
||||||
pd::PdClient,
|
pd::PdClient,
|
||||||
request::{
|
request::{
|
||||||
DefaultProcessor, Dispatch, ExtractError, KvRequest, Merge, MergeResponse, MultiRegion,
|
DefaultProcessor, Dispatch, ExtractError, KvRequest, Merge, MergeResponse, MultiRegion,
|
||||||
|
@ -14,6 +15,8 @@ use crate::{
|
||||||
use std::{marker::PhantomData, sync::Arc};
|
use std::{marker::PhantomData, sync::Arc};
|
||||||
use tikv_client_store::HasError;
|
use tikv_client_store::HasError;
|
||||||
|
|
||||||
|
use super::plan::PreserveKey;
|
||||||
|
|
||||||
/// Builder type for plans (see that module for more).
|
/// Builder type for plans (see that module for more).
|
||||||
pub struct PlanBuilder<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> {
|
pub struct PlanBuilder<PdC: PdClient, P: Plan, Ph: PlanBuilderPhase> {
|
||||||
pd_client: Arc<PdC>,
|
pd_client: Arc<PdC>,
|
||||||
|
@ -161,6 +164,19 @@ impl<PdC: PdClient, R: KvRequest> PlanBuilder<PdC, Dispatch<R>, NoTarget> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<PdC: PdClient, P: Plan + HasKeys> PlanBuilder<PdC, P, NoTarget>
|
||||||
|
where
|
||||||
|
P::Result: HasError,
|
||||||
|
{
|
||||||
|
pub fn preserve_keys(self) -> PlanBuilder<PdC, PreserveKey<P>, NoTarget> {
|
||||||
|
PlanBuilder {
|
||||||
|
pd_client: self.pd_client.clone(),
|
||||||
|
plan: PreserveKey { inner: self.plan },
|
||||||
|
phantom: PhantomData,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl<PdC: PdClient, P: Plan> PlanBuilder<PdC, P, Targetted>
|
impl<PdC: PdClient, P: Plan> PlanBuilder<PdC, P, Targetted>
|
||||||
where
|
where
|
||||||
P::Result: HasError,
|
P::Result: HasError,
|
||||||
|
|
|
@ -1,6 +1,7 @@
|
||||||
// Copyright 2021 TiKV Project Authors. Licensed under Apache-2.0.
|
// Copyright 2021 TiKV Project Authors. Licensed under Apache-2.0.
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
|
kv::HasKeys,
|
||||||
pd::PdClient,
|
pd::PdClient,
|
||||||
request::{Dispatch, KvRequest, Plan, ResolveLock, RetryRegion},
|
request::{Dispatch, KvRequest, Plan, ResolveLock, RetryRegion},
|
||||||
store::Store,
|
store::Store,
|
||||||
|
@ -9,6 +10,8 @@ use crate::{
|
||||||
use futures::stream::BoxStream;
|
use futures::stream::BoxStream;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
|
||||||
|
use super::plan::PreserveKey;
|
||||||
|
|
||||||
pub trait Shardable {
|
pub trait Shardable {
|
||||||
type Shard: Send;
|
type Shard: Send;
|
||||||
|
|
||||||
|
@ -51,6 +54,21 @@ impl<P: Plan + Shardable, PdC: PdClient> Shardable for ResolveLock<P, PdC> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<P: Plan + HasKeys + Shardable> Shardable for PreserveKey<P> {
|
||||||
|
type Shard = P::Shard;
|
||||||
|
|
||||||
|
fn shards(
|
||||||
|
&self,
|
||||||
|
pd_client: &Arc<impl PdClient>,
|
||||||
|
) -> BoxStream<'static, Result<(Self::Shard, Store)>> {
|
||||||
|
self.inner.shards(pd_client)
|
||||||
|
}
|
||||||
|
|
||||||
|
fn apply_shard(&mut self, shard: Self::Shard, store: &Store) -> Result<()> {
|
||||||
|
self.inner.apply_shard(shard, store)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl<P: Plan + Shardable, PdC: PdClient> Shardable for RetryRegion<P, PdC> {
|
impl<P: Plan + Shardable, PdC: PdClient> Shardable for RetryRegion<P, PdC> {
|
||||||
type Shard = P::Shard;
|
type Shard = P::Shard;
|
||||||
|
|
||||||
|
|
|
@ -1,6 +1,7 @@
|
||||||
// Copyright 2020 TiKV Project Authors. Licensed under Apache-2.0.
|
// Copyright 2020 TiKV Project Authors. Licensed under Apache-2.0.
|
||||||
|
|
||||||
use crate::{
|
use crate::{
|
||||||
|
kv::HasKeys,
|
||||||
pd::PdClient,
|
pd::PdClient,
|
||||||
request::{Collect, DefaultProcessor, KvRequest, Merge, Process, Shardable, SingleKey},
|
request::{Collect, DefaultProcessor, KvRequest, Merge, Process, Shardable, SingleKey},
|
||||||
store::{store_stream_for_keys, store_stream_for_range_by_start_key, Store},
|
store::{store_stream_for_keys, store_stream_for_range_by_start_key, Store},
|
||||||
|
@ -359,6 +360,15 @@ impl Shardable for kvrpcpb::PessimisticLockRequest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl HasKeys for kvrpcpb::PessimisticLockRequest {
|
||||||
|
fn get_keys(&self) -> Vec<Key> {
|
||||||
|
self.mutations
|
||||||
|
.iter()
|
||||||
|
.map(|m| m.key.clone().into())
|
||||||
|
.collect()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
impl Merge<kvrpcpb::PessimisticLockResponse> for Collect {
|
impl Merge<kvrpcpb::PessimisticLockResponse> for Collect {
|
||||||
// FIXME: PessimisticLockResponse only contains values.
|
// FIXME: PessimisticLockResponse only contains values.
|
||||||
// We need to pair keys and values returned somewhere.
|
// We need to pair keys and values returned somewhere.
|
||||||
|
|
|
@ -3,7 +3,7 @@
|
||||||
use crate::{
|
use crate::{
|
||||||
backoff::Backoff,
|
backoff::Backoff,
|
||||||
pd::{PdClient, PdRpcClient},
|
pd::{PdClient, PdRpcClient},
|
||||||
request::{Collect, CollectError, Plan, PlanBuilder, RetryOptions},
|
request::{Collect, CollectAndMatchKey, CollectError, Plan, PlanBuilder, RetryOptions},
|
||||||
timestamp::TimestampExt,
|
timestamp::TimestampExt,
|
||||||
transaction::{buffer::Buffer, lowering::*},
|
transaction::{buffer::Buffer, lowering::*},
|
||||||
BoundRange, Error, Key, KvPair, Result, Value,
|
BoundRange, Error, Key, KvPair, Result, Value,
|
||||||
|
@ -150,10 +150,12 @@ impl Transaction {
|
||||||
if !self.is_pessimistic() {
|
if !self.is_pessimistic() {
|
||||||
Err(Error::InvalidTransactionType)
|
Err(Error::InvalidTransactionType)
|
||||||
} else {
|
} else {
|
||||||
let key = key.into();
|
let mut pairs = self.pessimistic_lock(iter::once(key.into()), true).await?;
|
||||||
let mut values = self.pessimistic_lock(iter::once(key.clone()), true).await?;
|
debug_assert!(pairs.len() <= 1);
|
||||||
assert!(values.len() == 1);
|
match pairs.pop() {
|
||||||
Ok(values.pop().unwrap())
|
Some(pair) => Ok(Some(pair.1)),
|
||||||
|
None => Ok(None),
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -242,7 +244,7 @@ impl Transaction {
|
||||||
/// It can only be used in pessimistic mode.
|
/// It can only be used in pessimistic mode.
|
||||||
///
|
///
|
||||||
/// # Examples
|
/// # Examples
|
||||||
/// ```rust,no_run,compile_fail
|
/// ```rust,no_run
|
||||||
/// # use tikv_client::{Key, Value, Config, TransactionClient};
|
/// # use tikv_client::{Key, Value, Config, TransactionClient};
|
||||||
/// # use futures::prelude::*;
|
/// # use futures::prelude::*;
|
||||||
/// # use std::collections::HashMap;
|
/// # use std::collections::HashMap;
|
||||||
|
@ -263,19 +265,16 @@ impl Transaction {
|
||||||
/// ```
|
/// ```
|
||||||
// This is temporarily disabled because we cannot correctly match the keys and values.
|
// This is temporarily disabled because we cannot correctly match the keys and values.
|
||||||
// See `impl KvRequest for kvrpcpb::PessimisticLockRequest` for details.
|
// See `impl KvRequest for kvrpcpb::PessimisticLockRequest` for details.
|
||||||
#[allow(dead_code)]
|
pub async fn batch_get_for_update(
|
||||||
async fn batch_get_for_update(
|
|
||||||
&mut self,
|
&mut self,
|
||||||
keys: impl IntoIterator<Item = impl Into<Key>>,
|
keys: impl IntoIterator<Item = impl Into<Key>>,
|
||||||
) -> Result<impl Iterator<Item = KvPair>> {
|
) -> Result<impl Iterator<Item = KvPair>> {
|
||||||
self.check_allow_operation()?;
|
self.check_allow_operation()?;
|
||||||
if !self.is_pessimistic() {
|
if !self.is_pessimistic() {
|
||||||
Err(Error::InvalidTransactionType)
|
return Err(Error::InvalidTransactionType);
|
||||||
} else {
|
|
||||||
let keys: Vec<Key> = keys.into_iter().map(|it| it.into()).collect();
|
|
||||||
self.pessimistic_lock(keys.clone(), false).await?;
|
|
||||||
self.batch_get(keys).await
|
|
||||||
}
|
}
|
||||||
|
let keys: Vec<Key> = keys.into_iter().map(|it| it.into()).collect();
|
||||||
|
Ok(self.pessimistic_lock(keys, true).await?.into_iter())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Create a new 'scan' request.
|
/// Create a new 'scan' request.
|
||||||
|
@ -618,42 +617,43 @@ impl Transaction {
|
||||||
&mut self,
|
&mut self,
|
||||||
keys: impl IntoIterator<Item = Key>,
|
keys: impl IntoIterator<Item = Key>,
|
||||||
need_value: bool,
|
need_value: bool,
|
||||||
) -> Result<Vec<Option<Value>>> {
|
) -> Result<Vec<KvPair>> {
|
||||||
assert!(
|
assert!(
|
||||||
matches!(self.options.kind, TransactionKind::Pessimistic(_)),
|
matches!(self.options.kind, TransactionKind::Pessimistic(_)),
|
||||||
"`pessimistic_lock` is only valid to use with pessimistic transactions"
|
"`pessimistic_lock` is only valid to use with pessimistic transactions"
|
||||||
);
|
);
|
||||||
|
|
||||||
let keys: Vec<Key> = keys.into_iter().collect();
|
let keys: Vec<Key> = keys.into_iter().collect();
|
||||||
|
if keys.is_empty() {
|
||||||
|
return Ok(vec![]);
|
||||||
|
}
|
||||||
|
|
||||||
let first_key = keys[0].clone();
|
let first_key = keys[0].clone();
|
||||||
let primary_lock = self.buffer.get_primary_key_or(&first_key).await;
|
let primary_lock = self.buffer.get_primary_key_or(&first_key).await;
|
||||||
let lock_ttl = DEFAULT_LOCK_TTL;
|
|
||||||
let for_update_ts = self.rpc.clone().get_timestamp().await?;
|
let for_update_ts = self.rpc.clone().get_timestamp().await?;
|
||||||
self.options.push_for_update_ts(for_update_ts.clone());
|
self.options.push_for_update_ts(for_update_ts.clone());
|
||||||
let request = new_pessimistic_lock_request(
|
let request = new_pessimistic_lock_request(
|
||||||
keys.clone().into_iter(),
|
keys.clone().into_iter(),
|
||||||
primary_lock,
|
primary_lock,
|
||||||
self.timestamp.clone(),
|
self.timestamp.clone(),
|
||||||
lock_ttl,
|
DEFAULT_LOCK_TTL,
|
||||||
for_update_ts,
|
for_update_ts,
|
||||||
need_value,
|
need_value,
|
||||||
);
|
);
|
||||||
let plan = PlanBuilder::new(self.rpc.clone(), request)
|
let plan = PlanBuilder::new(self.rpc.clone(), request)
|
||||||
.resolve_lock(self.options.retry_options.lock_backoff.clone())
|
.resolve_lock(self.options.retry_options.lock_backoff.clone())
|
||||||
|
.preserve_keys()
|
||||||
.multi_region()
|
.multi_region()
|
||||||
.retry_region(self.options.retry_options.region_backoff.clone())
|
.retry_region(self.options.retry_options.region_backoff.clone())
|
||||||
.merge(Collect)
|
.merge(CollectAndMatchKey)
|
||||||
.plan();
|
.plan();
|
||||||
let values = plan
|
let pairs = plan.execute().await;
|
||||||
.execute()
|
|
||||||
.await
|
|
||||||
.map(|r| r.into_iter().map(Into::into).collect());
|
|
||||||
|
|
||||||
for key in keys {
|
for key in keys {
|
||||||
self.buffer.lock(key).await;
|
self.buffer.lock(key).await;
|
||||||
}
|
}
|
||||||
|
|
||||||
values
|
pairs
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Checks if the transaction can perform arbitrary operations.
|
/// Checks if the transaction can perform arbitrary operations.
|
||||||
|
|
|
@ -555,8 +555,7 @@ async fn raw_write_million() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn pessimistic_rollback() -> Result<()> {
|
async fn pessimistic_rollback() -> Result<()> {
|
||||||
clear_tikv().await;
|
clear_tikv().await;
|
||||||
let client =
|
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
||||||
TransactionClient::new_with_config(vec!["127.0.0.1:2379"], Default::default()).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 value = key1.clone();
|
let value = key1.clone();
|
||||||
|
@ -587,8 +586,7 @@ async fn pessimistic_rollback() -> Result<()> {
|
||||||
#[serial]
|
#[serial]
|
||||||
async fn lock_keys() -> Result<()> {
|
async fn lock_keys() -> Result<()> {
|
||||||
clear_tikv().await;
|
clear_tikv().await;
|
||||||
let client =
|
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
||||||
TransactionClient::new_with_config(vec!["127.0.0.1:2379"], Default::default()).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();
|
||||||
|
@ -617,6 +615,43 @@ async fn lock_keys() -> Result<()> {
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[tokio::test]
|
||||||
|
#[serial]
|
||||||
|
async fn get_for_update() -> Result<()> {
|
||||||
|
clear_tikv().await;
|
||||||
|
let client = TransactionClient::new_with_config(pd_addrs(), Default::default()).await?;
|
||||||
|
let key1 = "key".to_owned();
|
||||||
|
let key2 = "another key".to_owned();
|
||||||
|
let value1 = b"some value".to_owned();
|
||||||
|
let value2 = b"another value".to_owned();
|
||||||
|
let keys = vec![key1.clone(), key2.clone()];
|
||||||
|
|
||||||
|
let mut t1 = client.begin_pessimistic().await?;
|
||||||
|
let mut t2 = client.begin_pessimistic().await?;
|
||||||
|
|
||||||
|
let mut t0 = client.begin_pessimistic().await?;
|
||||||
|
t0.put(key1.clone(), value1).await?;
|
||||||
|
t0.put(key2.clone(), value2).await?;
|
||||||
|
t0.commit().await?;
|
||||||
|
|
||||||
|
assert!(t1.get(key1.clone()).await?.is_none());
|
||||||
|
assert!(t1.get_for_update(key1.clone()).await?.unwrap() == value1);
|
||||||
|
t1.commit().await?;
|
||||||
|
|
||||||
|
assert!(t2.batch_get(keys.clone()).await?.collect::<Vec<_>>().len() == 0);
|
||||||
|
let res: HashMap<_, _> = t2
|
||||||
|
.batch_get_for_update(keys.clone())
|
||||||
|
.await?
|
||||||
|
.map(From::from)
|
||||||
|
.collect();
|
||||||
|
t2.commit().await?;
|
||||||
|
assert!(res.get(&key1.into()).unwrap() == &value1);
|
||||||
|
assert!(res.get(&key2.into()).unwrap() == &value2);
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
// helper function
|
// helper function
|
||||||
async fn get_u32(client: &RawClient, key: Vec<u8>) -> Result<u32> {
|
async fn get_u32(client: &RawClient, key: Vec<u8>) -> Result<u32> {
|
||||||
let x = client.get(key).await?.unwrap();
|
let x = client.get(key).await?.unwrap();
|
||||||
|
|
Loading…
Reference in New Issue