diff --git a/examples/raw.rs b/examples/raw.rs index 9491701..f35fcc8 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -6,7 +6,7 @@ mod common; use crate::common::parse_args; -use tikv_client::{raw::Client, Config, Key, KvPair, Result, ToOwnedRange, Value}; +use tikv_client::{Config, Key, KvPair, RawClient as Client, Result, ToOwnedRange, Value}; const KEY: &str = "TiKV"; const VALUE: &str = "Rust"; @@ -27,8 +27,7 @@ async fn main() -> Result<()> { // When we first create a client we receive a `Connect` structure which must be resolved before // the client is actually connected and usable. - let unconnnected_client = Client::connect(config); - let client = unconnnected_client.await?; + let client = Client::new(config)?; // Requests are created from the connected client. These calls return structures which // implement `Future`. This means the `Future` must be resolved before the action ever takes diff --git a/examples/transaction.rs b/examples/transaction.rs index efc5d7c..e7ad532 100644 --- a/examples/transaction.rs +++ b/examples/transaction.rs @@ -7,7 +7,7 @@ mod common; use crate::common::parse_args; use futures::prelude::*; use std::ops::RangeBounds; -use tikv_client::{transaction::Client, Config, Key, KvPair, Value}; +use tikv_client::{Config, Key, KvPair, TransactionClient as Client, Value}; async fn puts(client: &Client, pairs: impl IntoIterator>) { let mut txn = client.begin().await.expect("Could not begin a transaction"); diff --git a/src/kv/bound_range.rs b/src/kv/bound_range.rs index b6c1e03..6d53dea 100644 --- a/src/kv/bound_range.rs +++ b/src/kv/bound_range.rs @@ -1,6 +1,7 @@ // Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. use super::Key; +use kvproto::kvrpcpb; #[cfg(test)] use proptest_derive::Arbitrary; use std::borrow::Borrow; @@ -182,6 +183,17 @@ impl + Eq> TryFrom<(Bound, Bound)> for BoundRange { } } +impl Into for BoundRange { + fn into(self) -> kvrpcpb::KeyRange { + let (start, end) = self.into_keys(); + let mut range = kvrpcpb::KeyRange::default(); + range.set_start_key(start.into()); + // FIXME handle end = None rather than unwrapping + end.map(|k| range.set_end_key(k.into())).unwrap(); + range + } +} + /// A convenience trait for converting ranges of borrowed types into a `BoundRange`. pub trait ToOwnedRange { /// Transform a borrowed range of some form into an owned `BoundRange`. diff --git a/src/kv/key.rs b/src/kv/key.rs index 41fe3d0..81cf2b7 100644 --- a/src/kv/key.rs +++ b/src/kv/key.rs @@ -111,6 +111,12 @@ impl<'a> Into<&'a [u8]> for &'a Key { } } +impl AsRef for Key { + fn as_ref(&self) -> &Key { + self + } +} + impl fmt::Debug for Key { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { write!(f, "Key({})", HexRepr(&self.0)) diff --git a/src/kv/kvpair.rs b/src/kv/kvpair.rs index 80de107..616b18d 100644 --- a/src/kv/kvpair.rs +++ b/src/kv/kvpair.rs @@ -1,6 +1,7 @@ // Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. use super::{HexRepr, Key, Value}; +use kvproto::kvrpcpb; #[cfg(test)] use proptest_derive::Arbitrary; use std::{fmt, str}; @@ -20,7 +21,7 @@ use std::{fmt, str}; /// types (Like a `(Key, Value)`) can be passed directly to those functions. #[derive(Default, Clone, Eq, PartialEq)] #[cfg_attr(test, derive(Arbitrary))] -pub struct KvPair(Key, Value); +pub struct KvPair(pub Key, pub Value); impl KvPair { /// Create a new `KvPair`. @@ -92,6 +93,34 @@ impl Into<(Key, Value)> for KvPair { } } +impl From for KvPair { + fn from(mut pair: kvrpcpb::KvPair) -> Self { + KvPair(Key::from(pair.take_key()), Value::from(pair.take_value())) + } +} + +impl Into for KvPair { + fn into(self) -> kvrpcpb::KvPair { + let mut result = kvrpcpb::KvPair::default(); + let (key, value) = self.into(); + result.set_key(key.into()); + result.set_value(value.into()); + result + } +} + +impl AsRef for KvPair { + fn as_ref(&self) -> &Key { + &self.0 + } +} + +impl AsRef for KvPair { + fn as_ref(&self) -> &Value { + &self.1 + } +} + impl fmt::Debug for KvPair { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { let KvPair(key, value) = self; diff --git a/src/kv_client/client.rs b/src/kv_client/client.rs new file mode 100644 index 0000000..dc63ea8 --- /dev/null +++ b/src/kv_client/client.rs @@ -0,0 +1,310 @@ +// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. + +// FIXME: Remove this when txn is done. +#![allow(dead_code)] + +use derive_new::new; +use futures::compat::Compat01As03; +use futures::future::BoxFuture; +use futures::prelude::*; +use grpcio::CallOption; +use kvproto::kvrpcpb; +use kvproto::tikvpb::TikvClient; +use std::{sync::Arc, time::Duration}; + +use crate::{ + kv_client::HasError, pd::Region, raw::RawRequest, stats::tikv_stats, transaction::TxnInfo, + ErrorKind, Key, Result, +}; + +/// This client handles requests for a single TiKV node. It converts the data +/// types and abstractions of the client program into the grpc data types. +#[derive(new, Clone)] +pub struct KvRpcClient { + rpc_client: Arc, +} + +impl super::KvClient for KvRpcClient { + fn dispatch( + &self, + request: &T::RpcRequest, + opt: CallOption, + ) -> BoxFuture<'static, Result> { + map_errors_and_trace(T::REQUEST_NAME, T::RPC_FN(&self.rpc_client, request, opt)).boxed() + } +} + +pub struct TransactionRegionClient { + region: Region, + timeout: Duration, + client: Arc, +} + +// FIXME use `request` method instead. +macro_rules! txn_request { + ($region:expr, $type:ty) => {{ + let mut req = <$type>::default(); + // FIXME don't unwrap + req.set_context($region.context().unwrap()); + req + }}; +} + +impl From for kvrpcpb::TxnInfo { + fn from(txn_info: TxnInfo) -> kvrpcpb::TxnInfo { + let mut pb = kvrpcpb::TxnInfo::default(); + pb.set_txn(txn_info.txn); + pb.set_status(txn_info.status); + pb + } +} + +impl TransactionRegionClient { + pub fn kv_get( + &self, + version: u64, + key: Key, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::GetRequest); + req.set_key(key.into()); + req.set_version(version); + + map_errors_and_trace( + "kv_get", + self.client + .clone() + .kv_get_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_scan( + &self, + version: u64, + start_key: Key, + end_key: Key, + limit: u32, + key_only: bool, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::ScanRequest); + req.set_start_key(start_key.into()); + req.set_end_key(end_key.into()); + req.set_version(version); + req.set_limit(limit); + req.set_key_only(key_only); + + map_errors_and_trace( + "kv_scan", + self.client + .clone() + .kv_scan_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_prewrite( + &self, + mutations: impl Iterator, + primary_lock: Key, + start_version: u64, + lock_ttl: u64, + skip_constraint_check: bool, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::PrewriteRequest); + req.set_mutations(mutations.collect()); + req.set_primary_lock(primary_lock.into()); + req.set_start_version(start_version); + req.set_lock_ttl(lock_ttl); + req.set_skip_constraint_check(skip_constraint_check); + + map_errors_and_trace( + "kv_prewrite", + self.client + .clone() + .kv_prewrite_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_commit( + &self, + keys: impl Iterator, + start_version: u64, + commit_version: u64, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::CommitRequest); + req.set_keys(keys.map(|x| x.into()).collect()); + req.set_start_version(start_version); + req.set_commit_version(commit_version); + + map_errors_and_trace( + "kv_commit", + self.client + .clone() + .kv_commit_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_import( + &self, + mutations: impl Iterator, + commit_version: u64, + ) -> impl Future> { + let mut req = kvrpcpb::ImportRequest::default(); + req.set_mutations(mutations.collect()); + req.set_commit_version(commit_version); + + map_errors_and_trace( + "kv_import", + self.client + .clone() + .kv_import_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_cleanup( + &self, + key: Key, + start_version: u64, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::CleanupRequest); + req.set_key(key.into()); + req.set_start_version(start_version); + + map_errors_and_trace( + "kv_cleanup", + self.client + .clone() + .kv_cleanup_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_batch_get( + &self, + keys: impl Iterator, + version: u64, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::BatchGetRequest); + req.set_keys(keys.map(|x| x.into()).collect()); + req.set_version(version); + + map_errors_and_trace( + "kv_batch_get", + self.client + .clone() + .kv_batch_get_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_batch_rollback( + &self, + keys: impl Iterator, + start_version: u64, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::BatchRollbackRequest); + req.set_keys(keys.map(|x| x.into()).collect()); + req.set_start_version(start_version); + + map_errors_and_trace( + "kv_batch_rollback", + self.client + .clone() + .kv_batch_rollback_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_scan_lock( + &self, + start_key: Key, + max_version: u64, + limit: u32, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::ScanLockRequest); + req.set_start_key(start_key.into()); + req.set_max_version(max_version); + req.set_limit(limit); + + map_errors_and_trace( + "kv_scan_lock", + self.client + .clone() + .kv_scan_lock_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_resolve_lock( + &self, + txn_infos: impl Iterator, + start_version: u64, + commit_version: u64, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::ResolveLockRequest); + req.set_start_version(start_version); + req.set_commit_version(commit_version); + req.set_txn_infos(txn_infos.map(Into::into).collect()); + + map_errors_and_trace( + "kv_resolve_lock", + self.client + .clone() + .kv_resolve_lock_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_gc(&self, safe_point: u64) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::GcRequest); + req.set_safe_point(safe_point); + + map_errors_and_trace( + "kv_gc", + self.client + .clone() + .kv_gc_async_opt(&req, self.call_options()), + ) + } + + pub fn kv_delete_range( + &self, + start_key: Key, + end_key: Key, + ) -> impl Future> { + let mut req = txn_request!(self.region, kvrpcpb::DeleteRangeRequest); + req.set_start_key(start_key.into()); + req.set_end_key(end_key.into()); + + map_errors_and_trace( + "kv_delete_range", + self.client + .clone() + .kv_delete_range_async_opt(&req, self.call_options()), + ) + } + + fn call_options(&self) -> CallOption { + CallOption::default().timeout(self.timeout) + } +} + +fn map_errors_and_trace( + request_name: &'static str, + fut: ::grpcio::Result, +) -> impl Future> +where + Compat01As03: Future>, + Resp: HasError + Sized + Clone + Send + 'static, +{ + let context = tikv_stats(request_name); + + // FIXME should handle the error, not unwrap. + Compat01As03::new(fut.unwrap()) + .map(|r| match r { + Err(e) => Err(ErrorKind::Grpc(e).into()), + Ok(mut r) => { + if let Some(e) = r.region_error() { + Err(e) + } else if let Some(e) = r.error() { + Err(e) + } else { + Ok(r) + } + } + }) + .map(move |r| context.done(r)) +} diff --git a/src/kv_client/errors.rs b/src/kv_client/errors.rs new file mode 100644 index 0000000..743b29b --- /dev/null +++ b/src/kv_client/errors.rs @@ -0,0 +1,142 @@ +// Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. + +use crate::Error; +use kvproto::{errorpb, kvrpcpb}; + +pub trait HasRegionError { + fn region_error(&mut self) -> Option; +} + +pub trait HasError: HasRegionError { + fn error(&mut self) -> Option; +} + +impl From for Error { + fn from(mut e: errorpb::Error) -> Error { + let message = e.take_message(); + if e.has_not_leader() { + let e = e.get_not_leader(); + let message = format!("{}. Leader: {:?}", message, e.get_leader()); + Error::not_leader(e.get_region_id(), Some(message)) + } else if e.has_region_not_found() { + Error::region_not_found(e.get_region_not_found().get_region_id(), Some(message)) + } else if e.has_key_not_in_region() { + let e = e.take_key_not_in_region(); + Error::key_not_in_region(e) + } else if e.has_epoch_not_match() { + Error::stale_epoch(Some(format!( + "{}. New epoch: {:?}", + message, + e.get_epoch_not_match().get_current_regions() + ))) + } else if e.has_server_is_busy() { + Error::server_is_busy(e.take_server_is_busy()) + } else if e.has_stale_command() { + Error::stale_command(message) + } else if e.has_store_not_match() { + Error::store_not_match(e.take_store_not_match(), message) + } else if e.has_raft_entry_too_large() { + Error::raft_entry_too_large(e.take_raft_entry_too_large(), message) + } else { + Error::internal_error(message) + } + } +} + +macro_rules! has_region_error { + ($type:ty) => { + impl HasRegionError for $type { + fn region_error(&mut self) -> Option { + if self.has_region_error() { + Some(self.take_region_error().into()) + } else { + None + } + } + } + }; +} + +has_region_error!(kvrpcpb::GetResponse); +has_region_error!(kvrpcpb::ScanResponse); +has_region_error!(kvrpcpb::PrewriteResponse); +has_region_error!(kvrpcpb::CommitResponse); +has_region_error!(kvrpcpb::ImportResponse); +has_region_error!(kvrpcpb::BatchRollbackResponse); +has_region_error!(kvrpcpb::CleanupResponse); +has_region_error!(kvrpcpb::BatchGetResponse); +has_region_error!(kvrpcpb::ScanLockResponse); +has_region_error!(kvrpcpb::ResolveLockResponse); +has_region_error!(kvrpcpb::GcResponse); +has_region_error!(kvrpcpb::RawGetResponse); +has_region_error!(kvrpcpb::RawBatchGetResponse); +has_region_error!(kvrpcpb::RawPutResponse); +has_region_error!(kvrpcpb::RawBatchPutResponse); +has_region_error!(kvrpcpb::RawDeleteResponse); +has_region_error!(kvrpcpb::RawBatchDeleteResponse); +has_region_error!(kvrpcpb::DeleteRangeResponse); +has_region_error!(kvrpcpb::RawDeleteRangeResponse); +has_region_error!(kvrpcpb::RawScanResponse); +has_region_error!(kvrpcpb::RawBatchScanResponse); + +macro_rules! has_key_error { + ($type:ty) => { + impl HasError for $type { + fn error(&mut self) -> Option { + if self.has_error() { + Some(self.take_error().into()) + } else { + None + } + } + } + }; +} + +has_key_error!(kvrpcpb::GetResponse); +has_key_error!(kvrpcpb::CommitResponse); +has_key_error!(kvrpcpb::BatchRollbackResponse); +has_key_error!(kvrpcpb::CleanupResponse); +has_key_error!(kvrpcpb::ScanLockResponse); +has_key_error!(kvrpcpb::ResolveLockResponse); +has_key_error!(kvrpcpb::GcResponse); + +macro_rules! has_str_error { + ($type:ty) => { + impl HasError for $type { + fn error(&mut self) -> Option { + if self.get_error().is_empty() { + None + } else { + Some(Error::kv_error(self.take_error())) + } + } + } + }; +} + +has_str_error!(kvrpcpb::RawGetResponse); +has_str_error!(kvrpcpb::RawPutResponse); +has_str_error!(kvrpcpb::RawBatchPutResponse); +has_str_error!(kvrpcpb::RawDeleteResponse); +has_str_error!(kvrpcpb::RawBatchDeleteResponse); +has_str_error!(kvrpcpb::RawDeleteRangeResponse); +has_str_error!(kvrpcpb::ImportResponse); +has_str_error!(kvrpcpb::DeleteRangeResponse); + +macro_rules! has_no_error { + ($type:ty) => { + impl HasError for $type { + fn error(&mut self) -> Option { + None + } + } + }; +} + +has_no_error!(kvrpcpb::ScanResponse); +has_no_error!(kvrpcpb::PrewriteResponse); +has_no_error!(kvrpcpb::BatchGetResponse); +has_no_error!(kvrpcpb::RawBatchGetResponse); +has_no_error!(kvrpcpb::RawScanResponse); +has_no_error!(kvrpcpb::RawBatchScanResponse); diff --git a/src/kv_client/mod.rs b/src/kv_client/mod.rs new file mode 100644 index 0000000..40ecac8 --- /dev/null +++ b/src/kv_client/mod.rs @@ -0,0 +1,125 @@ +// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. + +mod client; +mod errors; + +pub use self::client::KvRpcClient; +pub use self::errors::HasError; +pub use kvproto::tikvpb::TikvClient; + +use crate::pd::Region; +use crate::raw::{ColumnFamily, RawRequest}; +use crate::security::SecurityManager; +use crate::Result; +use derive_new::new; +use futures::future::BoxFuture; +use grpcio::CallOption; +use grpcio::Environment; +use kvproto::kvrpcpb; +use std::sync::Arc; +use std::time::Duration; + +/// A trait for connecting to TiKV stores. +pub trait KvConnect: Sized { + type KvClient: KvClient + Clone + Send + Sync + 'static; + + fn connect(&self, address: &str) -> Result; +} + +pub type RpcFnType = + for<'a, 'b> fn( + &'a TikvClient, + &'b Req, + CallOption, + ) + -> std::result::Result<::grpcio::ClientUnaryReceiver, ::grpcio::Error>; + +#[derive(new, Clone)] +pub struct TikvConnect { + env: Arc, + security_mgr: Arc, +} + +impl KvConnect for TikvConnect { + type KvClient = KvRpcClient; + + fn connect(&self, address: &str) -> Result { + self.security_mgr + .connect(self.env.clone(), address, TikvClient::new) + .map(|c| KvRpcClient::new(Arc::new(c))) + } +} + +pub trait KvClient { + fn dispatch( + &self, + request: &T::RpcRequest, + opt: CallOption, + ) -> BoxFuture<'static, Result>; +} + +#[derive(new)] +pub struct Store { + pub region: Region, + client: Client, + timeout: Duration, +} + +impl Store { + pub fn call_options(&self) -> CallOption { + CallOption::default().timeout(self.timeout) + } + + pub fn request(&self) -> T { + let mut request = T::default(); + // FIXME propagate the error instead of using `expect` + request.set_context( + self.region + .context() + .expect("Cannot create context from region"), + ); + request + } + + pub fn dispatch( + &self, + request: &T::RpcRequest, + opt: CallOption, + ) -> BoxFuture<'static, Result> { + self.client.dispatch::(request, opt) + } +} + +pub trait KvRawRequest: Default { + fn set_cf(&mut self, cf: String); + fn set_context(&mut self, context: kvrpcpb::Context); + + fn maybe_set_cf(&mut self, cf: Option) { + if let Some(cf) = cf { + self.set_cf(cf.to_string()); + } + } +} + +macro_rules! impl_raw_request { + ($name: ident) => { + impl KvRawRequest for kvrpcpb::$name { + fn set_cf(&mut self, cf: String) { + self.set_cf(cf); + } + fn set_context(&mut self, context: kvrpcpb::Context) { + self.set_context(context); + } + } + }; +} + +impl_raw_request!(RawGetRequest); +impl_raw_request!(RawBatchGetRequest); +impl_raw_request!(RawPutRequest); +impl_raw_request!(RawBatchPutRequest); +impl_raw_request!(RawDeleteRequest); +impl_raw_request!(RawBatchDeleteRequest); +impl_raw_request!(RawScanRequest); +impl_raw_request!(RawBatchScanRequest); +impl_raw_request!(RawDeleteRangeRequest); diff --git a/src/lib.rs b/src/lib.rs index 9016871..3f8d4d1 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -3,7 +3,9 @@ // Long and nested future chains can quickly result in large generic types. #![type_length_limit = "16777216"] #![allow(clippy::redundant_closure)] +#![allow(clippy::type_complexity)] #![feature(async_await)] +#![cfg_attr(test, feature(specialization))] //! This crate provides a clean, ready to use client for [TiKV](https://github.com/tikv/tikv), a //! distributed transactional Key-Value database written in Rust. @@ -33,7 +35,7 @@ //! operations affecting multiple keys or values, or operations that depend on strong ordering. //! //! ```rust -//! use tikv_client::{*, transaction::*}; +//! use tikv_client::*; //! ``` //! //! ### Raw @@ -45,7 +47,7 @@ //! key) requirements. You will not be able to use transactions with this API. //! //! ```rust -//! use tikv_client::{*, raw::*}; +//! use tikv_client::*; //! ``` //! //! ## Connect @@ -55,7 +57,7 @@ //! //! ```rust //! # #![feature(async_await)] -//! # use tikv_client::{*, raw::*}; +//! # use tikv_client::*; //! # use futures::prelude::*; //! //! # futures::executor::block_on(async { @@ -66,7 +68,7 @@ //! ]).with_security("root.ca", "internal.cert", "internal.key"); //! //! // Get an unresolved connection. -//! let connect = Client::connect(config); +//! let connect = TransactionClient::connect(config); //! //! // Resolve the connection into a client. //! let client = connect.into_future().await; @@ -75,15 +77,22 @@ //! //! At this point, you should seek the documentation in the related API modules. +#[macro_use] +mod util; + mod compat; mod config; mod errors; mod kv; +mod kv_client; +mod pd; +pub mod raw; +mod security; +mod stats; +pub mod transaction; + #[cfg(test)] mod proptests; -pub mod raw; -mod rpc; -pub mod transaction; #[macro_use] extern crate lazy_static; @@ -102,3 +111,9 @@ pub use crate::errors::ErrorKind; pub use crate::errors::Result; #[doc(inline)] pub use crate::kv::{BoundRange, Key, KvPair, ToOwnedRange, Value}; +#[doc(inline)] +pub use crate::raw::{Client as RawClient, ColumnFamily}; +#[doc(inline)] +pub use crate::transaction::{ + Client as TransactionClient, Connect, Snapshot, Timestamp, Transaction, +}; diff --git a/src/pd/client.rs b/src/pd/client.rs new file mode 100644 index 0000000..036a851 --- /dev/null +++ b/src/pd/client.rs @@ -0,0 +1,371 @@ +// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. + +use std::{ + collections::HashMap, + sync::{Arc, RwLock}, + time::Duration, +}; + +use futures::future::BoxFuture; +use futures::future::{ready, Either}; +use futures::prelude::*; +use futures::stream::BoxStream; +use grpcio::EnvBuilder; + +use crate::{ + compat::{stream_fn, ClientFutureExt}, + kv::BoundRange, + kv_client::{KvClient, KvConnect, Store, TikvConnect}, + pd::{Region, RegionId, RetryClient}, + security::SecurityManager, + transaction::Timestamp, + Config, Key, Result, +}; + +const CQ_COUNT: usize = 1; +const CLIENT_PREFIX: &str = "tikv-client"; + +pub trait PdClient: Send + Sync + 'static { + type KvClient: KvClient + Send + Sync + 'static; + + fn map_region_to_store( + self: Arc, + region: Region, + ) -> BoxFuture<'static, Result>>; + + fn region_for_key(&self, key: &Key) -> BoxFuture<'static, Result>; + + fn region_for_id(&self, id: RegionId) -> BoxFuture<'static, Result>; + + fn get_timestamp(self: Arc) -> BoxFuture<'static, Result>; + + fn store_for_key( + self: Arc, + key: &Key, + ) -> BoxFuture<'static, Result>> { + self.region_for_key(key) + .and_then(move |region| self.clone().map_region_to_store(region)) + .boxed() + } + + fn store_for_id( + self: Arc, + id: RegionId, + ) -> BoxFuture<'static, Result>> { + self.region_for_id(id) + .and_then(move |region| self.clone().map_region_to_store(region).boxed()) + .boxed() + } + + fn group_keys_by_region + Send + Sync + 'static>( + self: Arc, + keys: impl Iterator + Send + Sync + 'static, + ) -> BoxStream<'static, Result<(RegionId, Vec)>> { + let keys = keys.peekable(); + stream_fn(keys, move |mut keys| { + if let Some(key) = keys.next() { + Either::Left(self.region_for_key(key.as_ref()).map_ok(move |region| { + let id = region.id(); + let mut grouped = vec![key]; + while let Some(key) = keys.peek() { + if !region.contains(key.as_ref()) { + break; + } + grouped.push(keys.next().unwrap()); + } + Some((keys, (id, grouped))) + })) + } else { + Either::Right(ready(Ok(None))) + } + }) + .boxed() + } + + // Returns a Steam which iterates over the contexts for each region covered by range. + fn stores_for_range( + self: Arc, + range: BoundRange, + ) -> BoxStream<'static, Result>> { + let (start_key, end_key) = range.into_keys(); + stream_fn(Some(start_key), move |start_key| { + let start_key = match start_key { + None => return Either::Right(ready(Ok(None))), + Some(sk) => sk, + }; + let end_key = end_key.clone(); + + let this = self.clone(); + Either::Left(self.region_for_key(&start_key).and_then(move |region| { + let region_end = region.end_key(); + this.map_region_to_store(region).map_ok(move |store| { + if end_key.map(|x| x < region_end).unwrap_or(false) || region_end.is_empty() { + return Some((None, store)); + } + Some((Some(region_end), store)) + }) + })) + }) + .boxed() + } +} + +/// This client converts requests for the logical TiKV cluster into requests +/// for a single TiKV store using PD and internal logic. +pub struct PdRpcClient { + pd: Arc, + kv_connect: KvC, + kv_client_cache: Arc>>, + timeout: Duration, +} + +impl PdClient for PdRpcClient { + type KvClient = KvC::KvClient; + + fn map_region_to_store( + self: Arc, + region: Region, + ) -> BoxFuture<'static, Result>> { + let timeout = self.timeout; + // FIXME propagate this error instead of using `unwrap`. + let store_id = region.get_store_id().unwrap(); + self.pd + .clone() + .get_store(store_id) + .ok_and_then(move |store| self.kv_client(store.get_address())) + .map_ok(move |kv_client| Store::new(region, kv_client, timeout)) + .boxed() + } + + fn region_for_id(&self, id: RegionId) -> BoxFuture<'static, Result> { + self.pd.clone().get_region_by_id(id).boxed() + } + + fn region_for_key(&self, key: &Key) -> BoxFuture<'static, Result> { + self.pd.clone().get_region(key.into()).boxed() + } + + fn get_timestamp(self: Arc) -> BoxFuture<'static, Result> { + self.pd.clone().get_timestamp() + } +} + +impl PdRpcClient { + pub fn connect(config: &Config) -> Result { + let env = Arc::new( + EnvBuilder::new() + .cq_count(CQ_COUNT) + .name_prefix(thread_name!(CLIENT_PREFIX)) + .build(), + ); + let security_mgr = Arc::new( + if let (Some(ca_path), Some(cert_path), Some(key_path)) = + (&config.ca_path, &config.cert_path, &config.key_path) + { + SecurityManager::load(ca_path, cert_path, key_path)? + } else { + SecurityManager::default() + }, + ); + + let pd = Arc::new(RetryClient::connect( + env.clone(), + &config.pd_endpoints, + security_mgr.clone(), + config.timeout, + )?); + let kv_client_cache = Default::default(); + let kv_connect = TikvConnect::new(env, security_mgr); + Ok(PdRpcClient { + pd, + kv_client_cache, + kv_connect, + timeout: config.timeout, + }) + } +} + +impl PdRpcClient { + fn kv_client(&self, address: &str) -> Result { + if let Some(client) = self.kv_client_cache.read().unwrap().get(address) { + return Ok(client.clone()); + }; + info!("connect to tikv endpoint: {:?}", address); + self.kv_connect.connect(address).map(|client| { + self.kv_client_cache + .write() + .unwrap() + .insert(address.to_owned(), client.clone()); + client + }) + } +} + +#[cfg(test)] +pub mod test { + use super::*; + use crate::raw::{MockDispatch, RawRequest, RawScan}; + use crate::Error; + + use futures::executor; + use futures::future::{ready, BoxFuture}; + use grpcio::CallOption; + use kvproto::kvrpcpb; + use kvproto::metapb; + + // FIXME move all the mocks to their own module + pub struct MockKvClient; + + impl KvClient for MockKvClient { + fn dispatch( + &self, + _request: &T::RpcRequest, + _opt: CallOption, + ) -> BoxFuture<'static, Result> { + unreachable!() + } + } + + impl MockDispatch for RawScan { + fn mock_dispatch( + &self, + request: &kvrpcpb::RawScanRequest, + _opt: CallOption, + ) -> Option>> { + assert!(request.key_only); + assert_eq!(request.limit, 10); + + let mut resp = kvrpcpb::RawScanResponse::default(); + for i in request.start_key[0]..request.end_key[0] { + let mut kv = kvrpcpb::KvPair::default(); + kv.key = vec![i]; + resp.kvs.push(kv); + } + + Some(Box::pin(ready(Ok(resp)))) + } + } + + pub struct MockPdClient; + + impl PdClient for MockPdClient { + type KvClient = MockKvClient; + + fn map_region_to_store( + self: Arc, + region: Region, + ) -> BoxFuture<'static, Result>> { + Box::pin(ready(Ok(Store::new( + region, + MockKvClient, + Duration::from_secs(60), + )))) + } + + fn region_for_key(&self, key: &Key) -> BoxFuture<'static, Result> { + let bytes: &[_] = key.into(); + let region = if bytes.is_empty() || bytes[0] < 10 { + Self::region1() + } else { + Self::region2() + }; + + Box::pin(ready(Ok(region))) + } + fn region_for_id(&self, id: RegionId) -> BoxFuture<'static, Result> { + let result = match id { + 1 => Ok(Self::region1()), + 2 => Ok(Self::region2()), + _ => Err(Error::region_not_found(id, None)), + }; + + Box::pin(ready(result)) + } + + fn get_timestamp(self: Arc) -> BoxFuture<'static, Result> { + unimplemented!() + } + } + + // fn get_store(self: Arc, id: StoreId) -> BoxFuture<'static, Result> { + // let mut result = metapb::Store::default(); + // result.set_address(format!("store-address-{}", id)); + // Box::pin(ready(Ok(result))) + // } + + impl MockPdClient { + fn region1() -> Region { + let mut region = Region::default(); + region.region.id = 1; + region.region.set_start_key(vec![0]); + region.region.set_end_key(vec![10]); + + let mut leader = metapb::Peer::default(); + leader.store_id = 41; + region.leader = Some(leader); + + region + } + + fn region2() -> Region { + let mut region = Region::default(); + region.region.id = 2; + region.region.set_start_key(vec![10]); + region.region.set_end_key(vec![250, 250]); + + let mut leader = metapb::Peer::default(); + leader.store_id = 42; + region.leader = Some(leader); + + region + } + } + + // TODO needs us to mock out the KvConnect in PdRpcClient + // #[test] + // fn test_kv_client() { + // let client = MockPdClient; + // let addr1 = "foo"; + // let addr2 = "bar"; + + // let kv1 = client.kv_client(&addr1).unwrap(); + // let kv2 = client.kv_client(&addr2).unwrap(); + // let kv3 = client.kv_client(&addr2).unwrap(); + // assert!(&*kv1 as *const _ != &*kv2 as *const _); + // assert_eq!(&*kv2 as *const _, &*kv3 as *const _); + // } + + #[test] + fn test_group_keys_by_region() { + let client = MockPdClient; + + // FIXME This only works if the keys are in order of regions. Not sure if + // that is a reasonable constraint. + let tasks: Vec = vec![ + vec![1].into(), + vec![2].into(), + vec![3].into(), + vec![5, 2].into(), + vec![12].into(), + vec![11, 4].into(), + ]; + + let stream = Arc::new(client).group_keys_by_region(tasks.into_iter()); + let mut stream = executor::block_on_stream(stream); + + assert_eq!( + stream.next().unwrap().unwrap().1, + vec![ + vec![1].into(), + vec![2].into(), + vec![3].into(), + vec![5, 2].into() + ] + ); + assert_eq!( + stream.next().unwrap().unwrap().1, + vec![vec![12].into(), vec![11, 4].into()] + ); + assert!(stream.next().is_none()); + } +} diff --git a/src/rpc/pd/client.rs b/src/pd/cluster.rs similarity index 71% rename from src/rpc/pd/client.rs rename to src/pd/cluster.rs index fc6f083..31f1f33 100644 --- a/src/rpc/pd/client.rs +++ b/src/pd/cluster.rs @@ -1,26 +1,24 @@ // Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. +// FIXME: Remove this when txn is done. +#![allow(dead_code)] + use std::{ collections::HashSet, - fmt, sync::{Arc, RwLock}, time::{Duration, Instant}, }; use futures::compat::Compat01As03; -use futures::future::BoxFuture; use futures::prelude::*; use grpcio::{CallOption, Environment}; use kvproto::{metapb, pdpb}; use crate::{ - rpc::{ - pd::{ - context::request_context, request::retry_request, timestamp::TimestampOracle, Region, - RegionId, StoreId, Timestamp, - }, - security::SecurityManager, - }, + pd::{timestamp::TimestampOracle, Region, RegionId, StoreId}, + security::SecurityManager, + stats::pd_stats, + transaction::Timestamp, Error, Result, }; @@ -34,110 +32,6 @@ macro_rules! pd_request { }}; } -pub trait PdClient: Sized { - fn connect( - env: Arc, - endpoints: &[String], - security_mgr: Arc, - timeout: Duration, - ) -> Result; - - fn get_region(self: Arc, key: &[u8]) -> BoxFuture<'static, Result>; - - fn get_region_by_id(self: Arc, id: RegionId) -> BoxFuture<'static, Result>; - - fn get_store(self: Arc, id: StoreId) -> BoxFuture<'static, Result>; - - fn get_all_stores(self: Arc) -> BoxFuture<'static, Result>>; - - /// Request a timestamp from the PD cluster. - fn get_timestamp(self: Arc) -> BoxFuture<'static, Result>; -} - -/// Client for communication with a PD cluster. Has the facility to reconnect to the cluster. -pub struct RetryClient { - cluster: RwLock, - connection: Connection, - timeout: Duration, -} - -impl PdClient for RetryClient { - fn connect( - env: Arc, - endpoints: &[String], - security_mgr: Arc, - timeout: Duration, - ) -> Result { - let connection = Connection::new(env, security_mgr); - let cluster = RwLock::new(connection.connect_cluster(endpoints, timeout)?); - Ok(RetryClient { - cluster, - connection, - timeout, - }) - } - - // These get_* functions will try multiple times to make a request, reconnecting as necessary. - fn get_region(self: Arc, key: &[u8]) -> BoxFuture<'static, Result> { - let key = key.to_owned(); - let timeout = self.timeout; - Box::pin(retry_request(self, move |cluster| { - cluster.get_region(key.clone(), timeout) - })) - } - - fn get_region_by_id(self: Arc, id: RegionId) -> BoxFuture<'static, Result> { - let timeout = self.timeout; - Box::pin(retry_request(self, move |cluster| { - cluster.get_region_by_id(id, timeout) - })) - } - - fn get_store(self: Arc, id: StoreId) -> BoxFuture<'static, Result> { - let timeout = self.timeout; - Box::pin(retry_request(self, move |cluster| { - cluster.get_store(id, timeout) - })) - } - - fn get_all_stores(self: Arc) -> BoxFuture<'static, Result>> { - let timeout = self.timeout; - Box::pin(retry_request(self, move |cluster| { - cluster.get_all_stores(timeout) - })) - } - - fn get_timestamp(self: Arc) -> BoxFuture<'static, Result> { - // FIXME: retry or reconnect on error - Box::pin(self.cluster.read().unwrap().get_timestamp()) - } -} - -impl RetryClient { - pub fn reconnect(&self, interval: u64) -> Result<()> { - if let Some(cluster) = - self.connection - .reconnect(&self.cluster.read().unwrap(), interval, self.timeout)? - { - *self.cluster.write().unwrap() = cluster; - } - Ok(()) - } - - pub fn with_cluster T>(&self, f: F) -> T { - f(&self.cluster.read().unwrap()) - } -} - -impl fmt::Debug for RetryClient { - fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { - fmt.debug_struct("pd::RetryClient") - .field("cluster_id", &self.cluster.read().unwrap().id) - .field("timeout", &self.timeout) - .finish() - } -} - /// A PD cluster. pub struct Cluster { pub id: u64, @@ -148,8 +42,12 @@ pub struct Cluster { // These methods make a single attempt to make a request. impl Cluster { - fn get_region(&self, key: Vec, timeout: Duration) -> impl Future> { - let context = request_context("get_region"); + pub fn get_region( + &self, + key: Vec, + timeout: Duration, + ) -> impl Future> { + let context = pd_stats("get_region"); let option = CallOption::default().timeout(timeout); let mut req = pd_request!(self.id, pdpb::GetRegionRequest); @@ -175,12 +73,12 @@ impl Cluster { }) } - fn get_region_by_id( + pub fn get_region_by_id( &self, id: RegionId, timeout: Duration, ) -> impl Future> { - let context = request_context("get_region_by_id"); + let context = pd_stats("get_region_by_id"); let option = CallOption::default().timeout(timeout); let mut req = pd_request!(self.id, pdpb::GetRegionByIdRequest); @@ -204,12 +102,12 @@ impl Cluster { }) } - fn get_store( + pub fn get_store( &self, id: StoreId, timeout: Duration, ) -> impl Future> { - let context = request_context("get_store"); + let context = pd_stats("get_store"); let option = CallOption::default().timeout(timeout); let mut req = pd_request!(self.id, pdpb::GetStoreRequest); @@ -231,11 +129,11 @@ impl Cluster { }) } - fn get_all_stores( + pub fn get_all_stores( &self, timeout: Duration, ) -> impl Future>> { - let context = request_context("get_all_stores"); + let context = pd_stats("get_all_stores"); let option = CallOption::default().timeout(timeout); let req = pd_request!(self.id, pdpb::GetAllStoresRequest); @@ -256,20 +154,20 @@ impl Cluster { }) } - fn get_timestamp(&self) -> impl Future> { + pub fn get_timestamp(&self) -> impl Future> { self.tso.clone().get_timestamp() } } /// An object for connecting and reconnecting to a PD cluster. -struct Connection { +pub struct Connection { env: Arc, security_mgr: Arc, last_update: RwLock, } impl Connection { - fn new(env: Arc, security_mgr: Arc) -> Connection { + pub fn new(env: Arc, security_mgr: Arc) -> Connection { Connection { env, security_mgr, @@ -277,7 +175,7 @@ impl Connection { } } - fn connect_cluster(&self, endpoints: &[String], timeout: Duration) -> Result { + pub fn connect_cluster(&self, endpoints: &[String], timeout: Duration) -> Result { let members = self.validate_endpoints(endpoints, timeout)?; let (client, members) = self.try_connect_leader(&members, timeout)?; @@ -293,7 +191,7 @@ impl Connection { } // Re-establish connection with PD leader in synchronized fashion. - fn reconnect( + pub fn reconnect( &self, old_cluster: &Cluster, interval: u64, diff --git a/src/rpc/pd/mod.rs b/src/pd/mod.rs similarity index 79% rename from src/rpc/pd/mod.rs rename to src/pd/mod.rs index 0a05f8d..c300d9d 100644 --- a/src/rpc/pd/mod.rs +++ b/src/pd/mod.rs @@ -1,18 +1,18 @@ // Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. -// TODO: Remove this when txn is done. -#![allow(dead_code)] - use derive_new::new; use kvproto::{kvrpcpb, metapb, pdpb}; -pub use crate::rpc::pd::client::{PdClient, RetryClient}; use crate::{Error, Key, Result}; +#[cfg(test)] +pub use client::test::{MockKvClient, MockPdClient}; +pub use client::{PdClient, PdRpcClient}; +pub use retry::RetryClient; -#[macro_use] mod client; -mod context; -mod request; +#[macro_use] +mod cluster; +mod retry; mod timestamp; pub type RegionId = u64; @@ -32,6 +32,7 @@ pub struct Region { } impl Region { + #[allow(dead_code)] pub fn switch_peer(&mut self, _to: StoreId) -> Result<()> { unimplemented!() } @@ -56,14 +57,19 @@ impl Region { }) } - pub fn start_key(&self) -> &[u8] { - self.region.get_start_key() + pub fn start_key(&self) -> Key { + self.region.get_start_key().to_vec().into() } - pub fn end_key(&self) -> &[u8] { - self.region.get_end_key() + pub fn end_key(&self) -> Key { + self.region.get_end_key().to_vec().into() } + pub fn range(&self) -> (Key, Key) { + (self.start_key(), self.end_key()) + } + + #[allow(dead_code)] pub fn ver_id(&self) -> RegionVerId { let region = &self.region; let epoch = region.get_region_epoch(); @@ -78,21 +84,15 @@ impl Region { self.region.get_id() } - pub fn peer(&self) -> Result { + pub fn get_store_id(&self) -> Result { self.leader .as_ref() - .map(Clone::clone) - .map(Into::into) + .cloned() .ok_or_else(|| Error::stale_epoch(None)) + .map(|s| s.get_store_id()) } } -#[derive(Eq, PartialEq, Debug, Clone, Copy)] -pub struct Timestamp { - pub physical: i64, - pub logical: i64, -} - trait PdResponse { fn header(&self) -> &pdpb::ResponseHeader; } diff --git a/src/rpc/pd/request.rs b/src/pd/request.rs similarity index 97% rename from src/rpc/pd/request.rs rename to src/pd/request.rs index 73c0134..861d923 100644 --- a/src/rpc/pd/request.rs +++ b/src/pd/request.rs @@ -15,8 +15,8 @@ use std::pin::Pin; use tokio_timer::timer::Handle; use crate::{ - rpc::pd::client::{Cluster, RetryClient}, - rpc::util::GLOBAL_TIMER_HANDLE, + pd::client::{Cluster, RetryClient}, + util::GLOBAL_TIMER_HANDLE, Result, }; diff --git a/src/pd/retry.rs b/src/pd/retry.rs new file mode 100644 index 0000000..3bb416b --- /dev/null +++ b/src/pd/retry.rs @@ -0,0 +1,220 @@ +// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. + +//! A utility module for managing and retrying PD requests. + +use std::{ + fmt, + sync::{Arc, RwLock}, + time::{Duration, Instant}, +}; + +use futures::compat::Compat01As03; +use futures::future::BoxFuture; +use futures::prelude::*; +use futures::ready; +use futures::task::{Context, Poll}; +use grpcio::Environment; +use kvproto::metapb; +use std::pin::Pin; +use tokio_timer::timer::Handle; + +use crate::{ + pd::{ + cluster::{Cluster, Connection}, + Region, RegionId, StoreId, + }, + security::SecurityManager, + transaction::Timestamp, + util::GLOBAL_TIMER_HANDLE, + Result, +}; + +const RECONNECT_INTERVAL_SEC: u64 = 1; +const MAX_REQUEST_COUNT: usize = 3; +const LEADER_CHANGE_RETRY: usize = 10; + +/// Client for communication with a PD cluster. Has the facility to reconnect to the cluster. +pub struct RetryClient { + cluster: RwLock, + connection: Connection, + timeout: Duration, +} + +impl RetryClient { + pub fn connect( + env: Arc, + endpoints: &[String], + security_mgr: Arc, + timeout: Duration, + ) -> Result { + let connection = Connection::new(env, security_mgr); + let cluster = RwLock::new(connection.connect_cluster(endpoints, timeout)?); + Ok(RetryClient { + cluster, + connection, + timeout, + }) + } + + // These get_* functions will try multiple times to make a request, reconnecting as necessary. + pub fn get_region(self: Arc, key: &[u8]) -> BoxFuture<'static, Result> { + let key = key.to_owned(); + let timeout = self.timeout; + Box::pin(retry_request(self, move |cluster| { + cluster.get_region(key.clone(), timeout) + })) + } + + pub fn get_region_by_id(self: Arc, id: RegionId) -> BoxFuture<'static, Result> { + let timeout = self.timeout; + Box::pin(retry_request(self, move |cluster| { + cluster.get_region_by_id(id, timeout) + })) + } + + pub fn get_store(self: Arc, id: StoreId) -> BoxFuture<'static, Result> { + let timeout = self.timeout; + Box::pin(retry_request(self, move |cluster| { + cluster.get_store(id, timeout) + })) + } + + pub fn reconnect(&self, interval: u64) -> Result<()> { + if let Some(cluster) = + self.connection + .reconnect(&self.cluster.read().unwrap(), interval, self.timeout)? + { + *self.cluster.write().unwrap() = cluster; + } + Ok(()) + } + + pub fn with_cluster T>(&self, f: F) -> T { + f(&self.cluster.read().unwrap()) + } + + #[allow(dead_code)] + pub fn get_all_stores(self: Arc) -> BoxFuture<'static, Result>> { + let timeout = self.timeout; + Box::pin(retry_request(self, move |cluster| { + cluster.get_all_stores(timeout) + })) + } + + pub fn get_timestamp(self: Arc) -> BoxFuture<'static, Result> { + // FIXME: retry or reconnect on error + Box::pin(self.cluster.read().unwrap().get_timestamp()) + } +} + +impl fmt::Debug for RetryClient { + fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { + fmt.debug_struct("pd::RetryClient") + .field("cluster_id", &self.cluster.read().unwrap().id) + .field("timeout", &self.timeout) + .finish() + } +} + +fn retry_request( + client: Arc, + func: Func, +) -> RetryRequest>> +where + Resp: Send + 'static, + Func: Fn(&Cluster) -> RespFuture + Send + 'static, + RespFuture: Future> + Send + 'static, +{ + let mut req = Request::new(func, client); + RetryRequest { + reconnect_count: LEADER_CHANGE_RETRY, + future: req + .reconnect_if_needed() + .map_err(|_| internal_err!("failed to reconnect")) + .and_then(move |_| req.send_and_receive()), + } +} + +/// A future which will retry a request up to `reconnect_count` times or until it +/// succeeds. +struct RetryRequest { + reconnect_count: usize, + future: Fut, +} + +struct Request { + // We keep track of requests sent and after `MAX_REQUEST_COUNT` we reconnect. + request_sent: usize, + + client: Arc, + timer: Handle, + + // A function which makes an async request. + func: Func, +} + +impl Future for RetryRequest +where + Resp: Send + 'static, + Fut: Future> + Send + 'static, +{ + type Output = Result; + + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + unsafe { + let this = Pin::get_unchecked_mut(self); + if this.reconnect_count == 0 { + return Poll::Ready(Err(internal_err!("failed to send request"))); + } + + debug!("reconnect remains: {}", this.reconnect_count); + this.reconnect_count -= 1; + let resp = ready!(Pin::new_unchecked(&mut this.future).poll(cx))?; + Poll::Ready(Ok(resp)) + } + } +} + +impl Request +where + Resp: Send + 'static, + Func: Fn(&Cluster) -> RespFuture + Send + 'static, + RespFuture: Future> + Send + 'static, +{ + fn new(func: Func, client: Arc) -> Self { + Request { + request_sent: 0, + client, + timer: GLOBAL_TIMER_HANDLE.clone(), + func, + } + } + + fn reconnect_if_needed(&mut self) -> impl Future> + Send { + if self.request_sent < MAX_REQUEST_COUNT { + return future::Either::Left(future::ok(())); + } + + // FIXME: should not block the core. + match self.client.reconnect(RECONNECT_INTERVAL_SEC) { + Ok(_) => { + self.request_sent = 0; + future::Either::Left(future::ok(())) + } + Err(_) => future::Either::Right( + Compat01As03::new( + self.timer + .delay(Instant::now() + Duration::from_secs(RECONNECT_INTERVAL_SEC)), + ) + .map(|_| Err(())), + ), + } + } + + fn send_and_receive(&mut self) -> impl Future> + Send { + self.request_sent += 1; + debug!("request sent: {}", self.request_sent); + + self.client.with_cluster(&self.func) + } +} diff --git a/src/rpc/pd/timestamp.rs b/src/pd/timestamp.rs similarity index 99% rename from src/rpc/pd/timestamp.rs rename to src/pd/timestamp.rs index ee44499..d1478a7 100644 --- a/src/rpc/pd/timestamp.rs +++ b/src/pd/timestamp.rs @@ -11,8 +11,7 @@ //! single `TsoRequest` to the PD server. The other future receives `TsoResponse`s from the PD //! server and allocates timestamps for the requests. -use super::Timestamp; -use crate::{Error, Result}; +use crate::{transaction::Timestamp, Error, Result}; use futures::{ channel::{mpsc, oneshot}, diff --git a/src/proptests/raw.rs b/src/proptests/raw.rs index a76c83b..de7ce49 100644 --- a/src/proptests/raw.rs +++ b/src/proptests/raw.rs @@ -12,7 +12,7 @@ proptest! { fn point( pair in any::(), ) { - let client = block_on(Client::connect(Config::new(pd_addrs()))).unwrap(); + let client = Client::new(Config::new(pd_addrs())).unwrap(); block_on( client.put(pair.key().clone(), pair.value().clone()) @@ -36,7 +36,7 @@ proptest! { fn batch( kvs in arb_batch(any::(), None), ) { - let client = block_on(Client::connect(Config::new(pd_addrs()))).unwrap(); + let client = Client::new(Config::new(pd_addrs())).unwrap(); let keys = kvs.iter().map(|kv| kv.key()).cloned().collect::>(); block_on( diff --git a/src/raw/client.rs b/src/raw/client.rs index 9dd7e05..345ee4d 100644 --- a/src/raw/client.rs +++ b/src/raw/client.rs @@ -1,128 +1,46 @@ // Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. -use super::ColumnFamily; -use crate::{rpc::RpcClient, BoundRange, Config, Error, Key, KvPair, Result, Value}; +use super::{ + requests::{ + RawBatchDelete, RawBatchGet, RawBatchPut, RawBatchScan, RawDelete, RawDeleteRange, RawGet, + RawPut, RawRequest, RawScan, + }, + ColumnFamily, +}; +use crate::{pd::PdRpcClient, BoundRange, Config, Error, Key, KvPair, Result, Value}; -use derive_new::new; use futures::future::Either; use futures::prelude::*; -use futures::task::{Context, Poll}; -use std::{pin::Pin, sync::Arc, u32}; +use std::{sync::Arc, u32}; const MAX_RAW_KV_SCAN_LIMIT: u32 = 10240; /// The TiKV raw [`Client`](Client) is used to issue requests to the TiKV server and PD cluster. #[derive(Clone)] pub struct Client { - rpc: Arc, + rpc: Arc, cf: Option, key_only: bool, } -// The macros below make writing `impl Client` concise and hopefully easy to -// read. Otherwise, the important bits get lost in boilerplate. - -// `request` and `scan_request` define public functions which return a future for -// a request. When using them, supply the name of the function and the names of -// arguments, the name of the function on the RPC client, and the `Output` type -// of the returned future. -macro_rules! request { - ($fn_name:ident ($($args:ident),*), $rpc_name:ident, $output:ty) => { - pub fn $fn_name( - &self, - $($args: arg_type!($args),)* - ) -> impl Future> { - let this = self.clone(); - this.rpc.$rpc_name($(arg_convert!($args, $args)),*, this.cf) - } - } -} - -macro_rules! scan_request { - ($fn_name:ident ($($args:ident),*), $rpc_name:ident, $output:ty) => { - pub fn $fn_name( - &self, - $($args: arg_type!($args),)* - limit: u32, - ) -> impl Future> { - if limit > MAX_RAW_KV_SCAN_LIMIT { - Either::Right(future::err(Error::max_scan_limit_exceeded( - limit, - MAX_RAW_KV_SCAN_LIMIT, - ))) - } else { - let this = self.clone(); - Either::Left(this.rpc.$rpc_name($(arg_convert!($args, $args)),*, limit, this.key_only, this.cf)) - } - } - } -} - -// The following macros are used by the above macros to understand how arguments -// should be treated. `self` and `limit` (in scan requests) are treated separately. -// Skip to the use of `args!` to see the definitions. -// -// When using arguments in the `request` macros, we need to use their name, type, -// and be able to convert them for the RPC client functions. There are two kinds -// of argument - individual values, and collections of values. In the first case -// we always use `Into`, and in the second we take an iterator which we also -// also transform using `Into::into`. This gives users maximum flexibility. -// -// `arg_type` defines the type for values (`into`) and collections (`iter`). -// Likewise, `arg_convert` rule defines how to convert the argument into the more -// concrete type required by the RPC client. Both macros are created by `args`. - -macro_rules! arg_type_rule { - (into<$ty:ty>) => (impl Into<$ty>); - (iter<$ty:ty>) => (impl IntoIterator>); -} - -macro_rules! arg_convert_rule { - (into $id:ident) => { - $id.into() - }; - (iter $id:ident) => { - $id.into_iter().map(Into::into).collect() - }; -} - -// `$i` is the name of the argument (e.g, `key`) -// `$kind` is either `iter` or `into`. -// `$ty` is the concrete type of the argument. -macro_rules! args { - ($($i:ident: $kind:ident<$ty:ty>;)*) => { - macro_rules! arg_type { - $(($i) => (arg_type_rule!($kind<$ty>));)* - } - macro_rules! arg_convert { - $(($i, $id : ident) => (arg_convert_rule!($kind $id));)* - } - } -} - -args! { - key: into; - keys: iter; - value: into; - pairs: iter; - range: into; - ranges: iter; -} - impl Client { - /// Create a new [`Client`](Client) once the [`Connect`](Connect) resolves. + /// Create a new [`Client`](Client). /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Config, raw::Client}; + /// # use tikv_client::{Config, RawClient}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// let connect = Client::connect(Config::default()); - /// let client = connect.await.unwrap(); + /// let client = RawClient::new(Config::default()).unwrap(); /// # }); /// ``` - pub fn connect(config: Config) -> Connect { - Connect::new(config) + pub fn new(config: Config) -> Result { + let rpc = Arc::new(PdRpcClient::connect(&config)?); + Ok(Client { + rpc, + cf: None, + key_only: false, + }) } /// Set the column family of requests. @@ -132,11 +50,10 @@ impl Client { /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Config, raw::Client}; + /// # use tikv_client::{Config, RawClient}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// let connect = Client::connect(Config::default()); - /// let client = connect.await.unwrap(); + /// let client = RawClient::new(Config::default()).unwrap(); /// let get_request = client.with_cf("write").get("foo".to_owned()); /// # }); /// ``` @@ -157,11 +74,10 @@ impl Client { /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Config, raw::Client, ToOwnedRange}; + /// # use tikv_client::{Config, RawClient, ToOwnedRange}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// let connect = Client::connect(Config::default()); - /// let client = connect.await.unwrap(); + /// let client = RawClient::new(Config::default()).unwrap(); /// let scan_request = client.with_key_only(true).scan(("TiKV"..="TiDB").to_owned(), 2); /// # }); /// ``` @@ -173,205 +89,254 @@ impl Client { } } - /// 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 /// given key. /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Value, Config, raw::Client}; + /// # use tikv_client::{Value, Config, RawClient}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let key = "TiKV"; - /// let req = connected_client.get(key); + /// # let client = RawClient::new(Config::default()).unwrap(); + /// let key = "TiKV".to_owned(); + /// let req = client.get(key); /// let result: Option = req.await.unwrap(); /// # }); /// ``` - request!(get(key), raw_get, Option); + pub fn get(&self, key: impl Into) -> impl Future>> { + RawGet { + key: key.into(), + cf: self.cf.clone(), + } + .execute(self.rpc.clone()) + } - /// Create a new batch get request. + /// Create a new 'batch get' request. /// /// Once resolved this request will result in the fetching of the values associated with the /// given keys. /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{KvPair, Config, raw::Client}; + /// # use tikv_client::{KvPair, Config, RawClient}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let keys = vec!["TiKV", "TiDB"]; - /// let req = connected_client.batch_get(keys); + /// # let client = RawClient::new(Config::default()).unwrap(); + /// let keys = vec!["TiKV".to_owned(), "TiDB".to_owned()]; + /// let req = client.batch_get(keys); /// let result: Vec = req.await.unwrap(); /// # }); /// ``` - request!(batch_get(keys), raw_batch_get, Vec); + pub fn batch_get( + &self, + keys: impl IntoIterator>, + ) -> impl Future>> { + RawBatchGet { + keys: keys.into_iter().map(Into::into).collect(), + cf: self.cf.clone(), + } + .execute(self.rpc.clone()) + } - /// Create a new [`Put`](Put) request. + /// Create a new 'put' request. /// /// Once resolved this request will result in the setting of the value associated with the given key. /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Key, Value, Config, raw::Client}; + /// # use tikv_client::{Key, Value, Config, RawClient}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let key = "TiKV"; - /// let val = "TiKV"; - /// let req = connected_client.put(key, val); + /// # let client = RawClient::new(Config::default()).unwrap(); + /// let key = "TiKV".to_owned(); + /// let val = "TiKV".to_owned(); + /// let req = client.put(key, val); /// let result: () = req.await.unwrap(); /// # }); /// ``` - request!(put(key, value), raw_put, ()); + pub fn put( + &self, + key: impl Into, + value: impl Into, + ) -> impl Future> { + let rpc = self.rpc.clone(); + future::ready(RawPut::new(key, value, &self.cf)).and_then(|put| put.execute(rpc)) + } - /// Create a new [`BatchPut`](BatchPut) request. + /// Create a new 'batch put' request. /// /// Once resolved this request will result in the setting of the value associated with the given key. /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Error, Result, KvPair, Key, Value, Config, raw::Client}; + /// # use tikv_client::{Error, Result, KvPair, Key, Value, Config, RawClient, ToOwnedRange}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let kvpair1 = ("PD", "Go"); - /// let kvpair2 = ("TiKV", "Rust"); + /// # let client = RawClient::new(Config::default()).unwrap(); + /// let kvpair1 = ("PD".to_owned(), "Go".to_owned()); + /// let kvpair2 = ("TiKV".to_owned(), "Rust".to_owned()); /// let iterable = vec![kvpair1, kvpair2]; - /// let req = connected_client.batch_put(iterable); + /// let req = client.batch_put(iterable); /// let result: () = req.await.unwrap(); /// # }); /// ``` - request!(batch_put(pairs), raw_batch_put, ()); + pub fn batch_put( + &self, + pairs: impl IntoIterator>, + ) -> impl Future> { + let rpc = self.rpc.clone(); + future::ready(RawBatchPut::new(pairs, &self.cf)).and_then(|put| put.execute(rpc)) + } - /// Create a new [`Delete`](Delete) request. + /// Create a new 'delete' request. /// /// Once resolved this request will result in the deletion of the given key. /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Key, Config, raw::Client}; + /// # use tikv_client::{Key, Config, RawClient}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let key = "TiKV"; - /// let req = connected_client.delete(key); + /// # let client = RawClient::new(Config::default()).unwrap(); + /// let key = "TiKV".to_owned(); + /// let req = client.delete(key); /// let result: () = req.await.unwrap(); /// # }); /// ``` - request!(delete(key), raw_delete, ()); + pub fn delete(&self, key: impl Into) -> impl Future> { + RawDelete { + key: key.into(), + cf: self.cf.clone(), + } + .execute(self.rpc.clone()) + } - /// Create a new [`BatchDelete`](BatchDelete) request. + /// Create a new 'batch delete' request. /// /// Once resolved this request will result in the deletion of the given keys. /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Config, raw::Client}; + /// # use tikv_client::{Config, RawClient}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let keys = vec!["TiKV", "TiDB"]; - /// let req = connected_client.batch_delete(keys); + /// # let client = RawClient::new(Config::default()).unwrap(); + /// let keys = vec!["TiKV".to_owned(), "TiDB".to_owned()]; + /// let req = client.batch_delete(keys); /// let result: () = req.await.unwrap(); /// # }); /// ``` - request!(batch_delete(keys), raw_batch_delete, ()); + pub fn batch_delete( + &self, + keys: impl IntoIterator>, + ) -> impl Future> { + RawBatchDelete { + keys: keys.into_iter().map(Into::into).collect(), + cf: self.cf.clone(), + } + .execute(self.rpc.clone()) + } - /// Create a new [`Scan`](Scan) request. - /// - /// Once resolved this request will result in a scanner over the given keys. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{KvPair, Config, raw::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let inclusive_range = "TiKV"..="TiDB"; - /// let req = connected_client.scan(inclusive_range, 2); - /// let result: Vec = req.await.unwrap(); - /// # }); - /// ``` - scan_request!(scan(range), raw_scan, Vec); - - /// Create a new [`BatchScan`](BatchScan) request. - /// - /// Once resolved this request will result in a set of scanners over the given keys. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Key, Config, raw::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let inclusive_range1 = "TiDB"..="TiKV"; - /// let inclusive_range2 = "TiKV"..="TiSpark"; - /// let iterable = vec![inclusive_range1, inclusive_range2]; - /// let req = connected_client.batch_scan(iterable, 2); - /// let result = req.await; - /// # }); - /// ``` - scan_request!(batch_scan(ranges), raw_batch_scan, Vec); - - /// Create a new [`DeleteRange`](DeleteRange) request. + /// Create a new 'delete range' request. /// /// Once resolved this request will result in the deletion of all keys over the given range. /// /// ```rust,no_run /// # #![feature(async_await)] - /// # use tikv_client::{Key, Config, raw::Client}; + /// # use tikv_client::{Key, Config, RawClient, ToOwnedRange}; /// # use futures::prelude::*; /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); + /// # let client = RawClient::new(Config::default()).unwrap(); /// let inclusive_range = "TiKV"..="TiDB"; - /// let req = connected_client.delete_range(inclusive_range); + /// let req = client.delete_range(inclusive_range.to_owned()); /// let result: () = req.await.unwrap(); /// # }); /// ``` - request!(delete_range(range), raw_delete_range, ()); -} + pub fn delete_range(&self, range: impl Into) -> impl Future> { + RawDeleteRange { + range: range.into(), + cf: self.cf.clone(), + } + .execute(self.rpc.clone()) + } -/// An unresolved [`Client`](Client) connection to a TiKV cluster. -/// -/// Once resolved it will result in a connected [`Client`](Client). -/// -/// ```rust,no_run -/// # #![feature(async_await)] -/// use tikv_client::{Config, raw::{Client, Connect}}; -/// use futures::prelude::*; -/// -/// # futures::executor::block_on(async { -/// let connect: Connect = Client::connect(Config::default()); -/// let client: Client = connect.await.unwrap(); -/// # }); -/// ``` -#[derive(new)] -pub struct Connect { - config: Config, -} + /// Create a new 'scan' request. + /// + /// Once resolved this request will result in a scanner over the given keys. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{KvPair, Config, RawClient, ToOwnedRange}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let client = RawClient::new(Config::default()).unwrap(); + /// let inclusive_range = "TiKV"..="TiDB"; + /// let req = client.scan(inclusive_range.to_owned(), 2); + /// let result: Vec = req.await.unwrap(); + /// # }); + /// ``` + pub fn scan( + &self, + range: impl Into, + limit: u32, + ) -> impl Future>> { + if limit > MAX_RAW_KV_SCAN_LIMIT { + Either::Right(future::err(Error::max_scan_limit_exceeded( + limit, + MAX_RAW_KV_SCAN_LIMIT, + ))) + } else { + Either::Left( + RawScan { + range: range.into(), + limit, + key_only: self.key_only, + cf: self.cf.clone(), + } + .execute(self.rpc.clone()), + ) + } + } -impl Future for Connect { - type Output = Result; - - fn poll(self: Pin<&mut Self>, _cx: &mut Context) -> Poll { - let config = &self.config; - let rpc = Arc::new(RpcClient::connect(config)?); - Poll::Ready(Ok(Client { - rpc, - cf: None, - key_only: false, - })) + /// Create a new 'batch scan' request. + /// + /// Once resolved this request will result in a set of scanners over the given keys. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Key, Config, RawClient, ToOwnedRange}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let client = RawClient::new(Config::default()).unwrap(); + /// let inclusive_range1 = "TiDB"..="TiKV"; + /// let inclusive_range2 = "TiKV"..="TiSpark"; + /// let iterable = vec![inclusive_range1.to_owned(), inclusive_range2.to_owned()]; + /// let req = client.batch_scan(iterable, 2); + /// let result = req.await; + /// # }); + /// ``` + pub fn batch_scan( + &self, + ranges: impl IntoIterator>, + each_limit: u32, + ) -> impl Future>> { + if each_limit > MAX_RAW_KV_SCAN_LIMIT { + Either::Right(future::err(Error::max_scan_limit_exceeded( + each_limit, + MAX_RAW_KV_SCAN_LIMIT, + ))) + } else { + Either::Left( + RawBatchScan { + ranges: ranges.into_iter().map(Into::into).collect(), + each_limit, + key_only: self.key_only, + cf: self.cf.clone(), + } + .execute(self.rpc.clone()), + ) + } } } diff --git a/src/raw/mod.rs b/src/raw/mod.rs index 0410a42..27e23d2 100644 --- a/src/raw/mod.rs +++ b/src/raw/mod.rs @@ -10,11 +10,15 @@ //! **Warning:** It is not advisable to use both raw and transactional functionality in the same keyspace. //! -pub use self::client::{Client, Connect}; +pub use self::client::Client; +pub(crate) use requests::RawRequest; +#[cfg(test)] +pub use requests::*; use std::fmt; mod client; +mod requests; /// A [`ColumnFamily`](ColumnFamily) is an optional parameter for [`raw::Client`](Client) requests. /// @@ -31,7 +35,7 @@ mod client; /// The best (and only) way to create a [`ColumnFamily`](ColumnFamily) is via the `From` implementation: /// /// ```rust -/// # use tikv_client::raw::ColumnFamily; +/// # use tikv_client::ColumnFamily; /// /// let cf = ColumnFamily::from("write"); /// let cf = ColumnFamily::from(String::from("write")); diff --git a/src/raw/requests.rs b/src/raw/requests.rs new file mode 100644 index 0000000..9d9d6d4 --- /dev/null +++ b/src/raw/requests.rs @@ -0,0 +1,627 @@ +// Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. + +use crate::{ + kv_client::{HasError, KvClient, KvRawRequest, RpcFnType, Store}, + pd::PdClient, + raw::ColumnFamily, + BoundRange, Error, Key, KvPair, Result, Value, +}; + +use futures::future::BoxFuture; +use futures::prelude::*; +use futures::stream::BoxStream; +use grpcio::CallOption; +use kvproto::kvrpcpb; +use kvproto::tikvpb::TikvClient; +use std::mem; +use std::sync::Arc; + +pub trait RawRequest: Sync + Send + 'static + Sized + Clone { + type Result; + type RpcRequest; + type RpcResponse: HasError + Clone + Send + 'static; + type KeyType; + const REQUEST_NAME: &'static str; + const RPC_FN: RpcFnType; + + fn execute( + mut self, + pd_client: Arc, + ) -> BoxFuture<'static, Result> { + let stores = self.store_stream(pd_client); + Self::reduce( + stores + .and_then(move |(key, store)| { + let request = self.clone().into_request(key, &store); + self.mock_dispatch(&request, store.call_options()) + .unwrap_or_else(|| store.dispatch::(&request, store.call_options())) + }) + .map_ok(move |r| Self::map_result(r)) + .boxed(), + ) + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>>; + + fn into_request( + self, + key: Self::KeyType, + store: &Store, + ) -> Self::RpcRequest; + + fn map_result(result: Self::RpcResponse) -> Self::Result; + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result>; +} + +/// Permits easy mocking of rpc calls. +pub trait MockDispatch: RawRequest { + fn mock_dispatch( + &self, + _request: &Self::RpcRequest, + _opt: CallOption, + ) -> Option>> { + None + } +} + +impl MockDispatch for T {} + +#[derive(Clone)] +pub struct RawGet { + pub key: Key, + pub cf: Option, +} + +impl RawRequest for RawGet { + type Result = Option; + type RpcRequest = kvrpcpb::RawGetRequest; + type RpcResponse = kvrpcpb::RawGetResponse; + type KeyType = Key; + const REQUEST_NAME: &'static str = "raw_get"; + const RPC_FN: RpcFnType = TikvClient::raw_get_async_opt; + + fn into_request( + self, + key: Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_key(key.into()); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + let key = self.key.clone(); + pd_client + .store_for_key(&self.key) + .map_ok(move |store| (key, store)) + .into_stream() + .boxed() + } + + fn map_result(mut resp: Self::RpcResponse) -> Self::Result { + let result: Value = resp.take_value().into(); + if result.is_empty() { + None + } else { + Some(result) + } + } + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results + .into_future() + .map(|(f, _)| f.expect("no results should be impossible")) + .boxed() + } +} + +#[derive(Clone)] +pub struct RawBatchGet { + pub keys: Vec, + pub cf: Option, +} + +impl RawRequest for RawBatchGet { + type Result = Vec; + type RpcRequest = kvrpcpb::RawBatchGetRequest; + type RpcResponse = kvrpcpb::RawBatchGetResponse; + type KeyType = Vec; + const REQUEST_NAME: &'static str = "raw_batch_get"; + const RPC_FN: RpcFnType = + TikvClient::raw_batch_get_async_opt; + + fn into_request( + self, + keys: Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_keys(keys.into_iter().map(Into::into).collect()); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + let mut keys = Vec::new(); + mem::swap(&mut keys, &mut self.keys); + + pd_client + .clone() + .group_keys_by_region(keys.into_iter()) + .and_then(move |(region_id, key)| { + pd_client + .clone() + .store_for_id(region_id) + .map_ok(move |store| (key, store)) + }) + .boxed() + } + + fn map_result(mut resp: Self::RpcResponse) -> Self::Result { + resp.take_pairs().into_iter().map(Into::into).collect() + } + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results.try_concat().boxed() + } +} + +#[derive(Clone)] +pub struct RawPut { + pub key: Key, + pub value: Value, + pub cf: Option, +} + +impl RawPut { + pub fn new( + key: impl Into, + value: impl Into, + cf: &Option, + ) -> Result { + let value = value.into(); + if value.is_empty() { + return Err(Error::empty_value()); + } + + let key = key.into(); + Ok(RawPut { + key, + value, + cf: cf.clone(), + }) + } +} + +impl RawRequest for RawPut { + type Result = (); + type RpcRequest = kvrpcpb::RawPutRequest; + type RpcResponse = kvrpcpb::RawPutResponse; + type KeyType = KvPair; + const REQUEST_NAME: &'static str = "raw_put"; + const RPC_FN: RpcFnType = TikvClient::raw_put_async_opt; + + fn into_request( + self, + key: Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_key(key.0.into()); + req.set_value(key.1.into()); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + let kv = (self.key.clone(), self.value.clone()).into(); + pd_client + .store_for_key(&self.key) + .map_ok(move |store| (kv, store)) + .into_stream() + .boxed() + } + + fn map_result(_: Self::RpcResponse) -> Self::Result {} + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results + .into_future() + .map(|(f, _)| f.expect("no results should be impossible")) + .boxed() + } +} + +#[derive(Clone)] +pub struct RawBatchPut { + pub pairs: Vec, + pub cf: Option, +} + +impl RawBatchPut { + pub fn new( + pairs: impl IntoIterator>, + cf: &Option, + ) -> Result { + let pairs: Vec = pairs.into_iter().map(Into::into).collect(); + if pairs.iter().any(|pair| pair.value().is_empty()) { + return Err(Error::empty_value()); + } + + Ok(RawBatchPut { + pairs, + cf: cf.clone(), + }) + } +} + +impl RawRequest for RawBatchPut { + type Result = (); + type RpcRequest = kvrpcpb::RawBatchPutRequest; + type RpcResponse = kvrpcpb::RawBatchPutResponse; + type KeyType = Vec; + const REQUEST_NAME: &'static str = "raw_batch_put"; + const RPC_FN: RpcFnType = + TikvClient::raw_batch_put_async_opt; + + fn into_request( + self, + pairs: Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_pairs(pairs.into_iter().map(Into::into).collect()); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + let mut pairs = Vec::new(); + mem::swap(&mut pairs, &mut self.pairs); + + pd_client + .clone() + .group_keys_by_region(pairs.into_iter()) + .and_then(move |(region_id, pair)| { + pd_client + .clone() + .store_for_id(region_id) + .map_ok(move |store| (pair, store)) + }) + .boxed() + } + + fn map_result(_: Self::RpcResponse) -> Self::Result {} + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results.try_collect().boxed() + } +} + +#[derive(Clone)] +pub struct RawDelete { + pub key: Key, + pub cf: Option, +} + +impl RawRequest for RawDelete { + type Result = (); + type RpcRequest = kvrpcpb::RawDeleteRequest; + type RpcResponse = kvrpcpb::RawDeleteResponse; + type KeyType = Key; + const REQUEST_NAME: &'static str = "raw_delete"; + const RPC_FN: RpcFnType = TikvClient::raw_delete_async_opt; + + fn into_request( + self, + key: Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_key(key.into()); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + let key = self.key.clone(); + pd_client + .store_for_key(&self.key) + .map_ok(move |store| (key, store)) + .into_stream() + .boxed() + } + + fn map_result(_: Self::RpcResponse) -> Self::Result {} + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results + .into_future() + .map(|(f, _)| f.expect("no results should be impossible")) + .boxed() + } +} + +#[derive(Clone)] +pub struct RawBatchDelete { + pub keys: Vec, + pub cf: Option, +} + +impl RawRequest for RawBatchDelete { + type Result = (); + type RpcRequest = kvrpcpb::RawBatchDeleteRequest; + type RpcResponse = kvrpcpb::RawBatchDeleteResponse; + type KeyType = Vec; + const REQUEST_NAME: &'static str = "raw_batch_delete"; + const RPC_FN: RpcFnType = + TikvClient::raw_batch_delete_async_opt; + + fn into_request( + self, + keys: Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_keys(keys.into_iter().map(Into::into).collect()); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + let mut keys = Vec::new(); + mem::swap(&mut keys, &mut self.keys); + + pd_client + .clone() + .group_keys_by_region(keys.into_iter()) + .and_then(move |(region_id, key)| { + pd_client + .clone() + .store_for_id(region_id) + .map_ok(move |store| (key, store)) + }) + .boxed() + } + + fn map_result(_: Self::RpcResponse) -> Self::Result {} + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results.try_collect().boxed() + } +} + +#[derive(Clone)] +pub struct RawDeleteRange { + pub range: BoundRange, + pub cf: Option, +} + +impl RawRequest for RawDeleteRange { + type Result = (); + type RpcRequest = kvrpcpb::RawDeleteRangeRequest; + type RpcResponse = kvrpcpb::RawDeleteRangeResponse; + type KeyType = (Key, Key); + const REQUEST_NAME: &'static str = "raw_delete_range"; + const RPC_FN: RpcFnType = + TikvClient::raw_delete_range_async_opt; + + fn into_request( + self, + (start_key, end_key): Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_start_key(start_key.into()); + req.set_end_key(end_key.into()); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + let range = self.range.clone(); + pd_client + .stores_for_range(range) + .map_ok(move |store| { + // TODO should be bounded by self.range + let range = store.region.range(); + (range, store) + }) + .into_stream() + .boxed() + } + + fn map_result(_: Self::RpcResponse) -> Self::Result {} + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results + .into_future() + .map(|(f, _)| f.expect("no results should be impossible")) + .boxed() + } +} + +#[derive(Clone)] +pub struct RawScan { + pub range: BoundRange, + // TODO this limit is currently treated as a per-region limit, not a total + // limit. + pub limit: u32, + pub key_only: bool, + pub cf: Option, +} + +impl RawRequest for RawScan { + type Result = Vec; + type RpcRequest = kvrpcpb::RawScanRequest; + type RpcResponse = kvrpcpb::RawScanResponse; + type KeyType = (Key, Key); + const REQUEST_NAME: &'static str = "raw_scan"; + const RPC_FN: RpcFnType = TikvClient::raw_scan_async_opt; + + fn into_request( + self, + (start_key, end_key): Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_start_key(start_key.into()); + req.set_end_key(end_key.into()); + req.set_limit(self.limit); + req.set_key_only(self.key_only); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + let range = self.range.clone(); + pd_client + .stores_for_range(range) + .map_ok(move |store| { + // TODO seems like these should be bounded by self.range + let range = store.region.range(); + (range, store) + }) + .into_stream() + .boxed() + } + + fn map_result(mut resp: Self::RpcResponse) -> Self::Result { + resp.take_kvs().into_iter().map(Into::into).collect() + } + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results.try_concat().boxed() + } +} + +#[derive(Clone)] +pub struct RawBatchScan { + pub ranges: Vec, + pub each_limit: u32, + pub key_only: bool, + pub cf: Option, +} + +impl RawRequest for RawBatchScan { + type Result = Vec; + type RpcRequest = kvrpcpb::RawBatchScanRequest; + type RpcResponse = kvrpcpb::RawBatchScanResponse; + type KeyType = Vec; + const REQUEST_NAME: &'static str = "raw_batch_scan"; + const RPC_FN: RpcFnType = + TikvClient::raw_batch_scan_async_opt; + + fn into_request( + self, + ranges: Self::KeyType, + store: &Store, + ) -> Self::RpcRequest { + let mut req = store.request::(); + req.set_ranges(ranges.into_iter().map(Into::into).collect()); + req.set_each_limit(self.each_limit); + req.set_key_only(self.key_only); + req.maybe_set_cf(self.cf); + + req + } + + fn store_stream( + &mut self, + _pd_client: Arc, + ) -> BoxStream<'static, Result<(Self::KeyType, Store)>> { + future::err(Error::unimplemented()).into_stream().boxed() + } + + fn map_result(mut resp: Self::RpcResponse) -> Self::Result { + resp.take_kvs().into_iter().map(Into::into).collect() + } + + fn reduce( + results: BoxStream<'static, Result>, + ) -> BoxFuture<'static, Result> { + results.try_concat().boxed() + } +} + +#[cfg(test)] +mod test { + use super::*; + use crate::pd::MockPdClient; + use futures::executor; + + #[test] + #[ignore] + fn test_raw_scan() { + let client = Arc::new(MockPdClient); + + let start: Key = vec![1].into(); + let end: Key = vec![50].into(); + let scan = RawScan { + range: (start, end).into(), + limit: 10, + key_only: true, + cf: None, + }; + let scan = executor::block_on(scan.execute(client)).unwrap(); + + assert_eq!(scan.len(), 10); + // TODO test the keys returned. + } +} diff --git a/src/rpc/client.rs b/src/rpc/client.rs deleted file mode 100644 index 6b2df52..0000000 --- a/src/rpc/client.rs +++ /dev/null @@ -1,495 +0,0 @@ -// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. - -// TODO: Remove this when txn is done. -#![allow(dead_code)] - -use std::{ - collections::{HashMap, VecDeque}, - fmt, - sync::{Arc, RwLock}, - time::Duration, -}; - -use futures::future::{ready, Either}; -use futures::prelude::*; -use grpcio::{EnvBuilder, Environment}; -use kvproto::metapb; - -use crate::{ - compat::{stream_fn, ClientFutureExt}, - kv::BoundRange, - raw::ColumnFamily, - rpc::{ - pd::{PdClient, Region, RegionId, RetryClient, StoreId, Timestamp}, - security::SecurityManager, - tikv::KvClient, - Address, RawContext, Store, TxnContext, - }, - Config, Error, Key, KvPair, Result, Value, -}; - -const CQ_COUNT: usize = 1; -const CLIENT_PREFIX: &str = "tikv-client"; - -pub struct RpcClient { - pd: Arc, - tikv: Arc>>>, - env: Arc, - security_mgr: Arc, - timeout: Duration, -} - -impl RpcClient { - pub fn connect(config: &Config) -> Result> { - let env = Arc::new( - EnvBuilder::new() - .cq_count(CQ_COUNT) - .name_prefix(thread_name!(CLIENT_PREFIX)) - .build(), - ); - let security_mgr = Arc::new( - if let (Some(ca_path), Some(cert_path), Some(key_path)) = - (&config.ca_path, &config.cert_path, &config.key_path) - { - SecurityManager::load(ca_path, cert_path, key_path)? - } else { - SecurityManager::default() - }, - ); - - let pd = Arc::new(RetryClient::connect( - env.clone(), - &config.pd_endpoints, - security_mgr.clone(), - config.timeout, - )?); - let tikv = Default::default(); - Ok(RpcClient { - pd, - tikv, - env, - security_mgr, - timeout: config.timeout, - }) - } -} - -impl RpcClient { - pub fn raw_get( - self: Arc, - key: Key, - cf: Option, - ) -> impl Future>> { - self.clone() - .raw(&key) - .and_then(|context| context.client.raw_get(context.store, cf, key)) - .map_ok(|value| if value.is_empty() { None } else { Some(value) }) - } - - pub fn raw_batch_get( - self: Arc, - keys: Vec, - cf: Option, - ) -> impl Future>> { - self.clone().group_tasks_by_region(keys).try_fold( - Vec::new(), - move |mut result, (region_id, keys)| { - let cf = cf.clone(); - self.clone() - .raw_from_id(region_id) - .and_then(|context| { - context - .client - .raw_batch_get(context.store, cf, keys.into_iter()) - }) - .map_ok(move |mut pairs| { - result.append(&mut pairs); - result - }) - }, - ) - } - - pub fn raw_put( - self: Arc, - key: Key, - value: Value, - cf: Option, - ) -> impl Future> { - if value.is_empty() { - future::Either::Left(future::err(Error::empty_value())) - } else { - future::Either::Right( - self.raw(&key) - .and_then(|context| context.client.raw_put(context.store, cf, key, value)), - ) - } - } - - pub fn raw_batch_put( - self: Arc, - pairs: Vec, - cf: Option, - ) -> impl Future> { - if pairs.iter().any(|p| p.value().is_empty()) { - future::Either::Left(future::err(Error::empty_value())) - } else { - future::Either::Right(self.clone().group_tasks_by_region(pairs).try_for_each( - move |(region_id, keys)| { - let cf = cf.clone(); - self.clone() - .raw_from_id(region_id) - .and_then(|context| context.client.raw_batch_put(context.store, cf, keys)) - }, - )) - } - } - - pub fn raw_delete( - self: Arc, - key: Key, - cf: Option, - ) -> impl Future> { - self.raw(&key) - .and_then(|context| context.client.raw_delete(context.store, cf, key)) - } - - pub fn raw_batch_delete( - self: Arc, - keys: Vec, - cf: Option, - ) -> impl Future> { - self.clone() - .group_tasks_by_region(keys) - .try_for_each(move |(region_id, keys)| { - let cf = cf.clone(); - self.clone() - .raw_from_id(region_id) - .and_then(|context| context.client.raw_batch_delete(context.store, cf, keys)) - }) - } - - pub fn raw_scan( - self: Arc, - range: BoundRange, - limit: u32, - key_only: bool, - cf: Option, - ) -> impl Future>> { - self.regions_for_range(range) - .try_fold(Vec::new(), move |mut result, context| { - if result.len() as u32 >= limit { - // Skip any more scans if we've hit the limit already. - return Either::Left(ready(Ok(result))); - } - let (start_key, end_key) = context.store.range(); - Either::Right( - context - .client - .raw_scan( - context.store, - cf.clone(), - start_key, - Some(end_key), - limit, - key_only, - ) - .map_ok(move |mut pairs| { - result.append(&mut pairs); - result - }), - ) - }) - } - - pub fn raw_delete_range( - self: Arc, - range: BoundRange, - cf: Option, - ) -> impl Future> { - self.regions_for_range(range).try_for_each(move |context| { - let (start_key, end_key) = context.store.range(); - context - .client - .raw_delete_range(context.store, cf.clone(), start_key, end_key) - }) - } - - pub fn raw_batch_scan( - self: Arc, - _ranges: Vec, - _each_limit: u32, - _key_only: bool, - _cf: Option, - ) -> impl Future>> { - future::err(Error::unimplemented()) - } - - pub fn get_timestamp(self: Arc) -> impl Future> { - self.pd.clone().get_timestamp() - } - - // Returns a Steam which iterates over the contexts for each region covered by range. - fn regions_for_range( - self: Arc, - range: BoundRange, - ) -> impl Stream> { - let (start_key, end_key) = range.into_keys(); - stream_fn(Some(start_key), move |start_key| { - let start_key = match start_key { - None => return Either::Right(ready(Ok(None))), - Some(sk) => sk, - }; - let end_key = end_key.clone(); - let this = self.clone(); - Either::Left(self.get_region(&start_key).and_then(move |location| { - this.raw_from_id(location.id()).map_ok(move |context| { - let region_end = context.store.end_key(); - if end_key.map(|x| x < region_end).unwrap_or(false) || region_end.is_empty() { - return Some((None, context)); - } - Some((Some(region_end), context)) - }) - })) - }) - } - - fn group_tasks_by_region( - self: Arc, - tasks: Vec, - ) -> impl Stream)>> - where - Task: GroupingTask, - { - let tasks: VecDeque = tasks.into(); - - stream_fn(tasks, move |mut tasks| { - if tasks.is_empty() { - Either::Right(ready(Ok(None))) - } else { - Either::Left(self.get_region(tasks[0].key()).map_ok(move |region| { - let id = region.id(); - let mut grouped = Vec::new(); - while let Some(task) = tasks.pop_front() { - if !region.contains(task.key()) { - tasks.push_front(task); - break; - } - grouped.push(task); - } - Some((tasks, (id, grouped))) - })) - } - }) - } - - fn load_store(&self, id: StoreId) -> impl Future> { - info!("reload info for store {}", id); - self.pd.clone().get_store(id) - } - - fn load_region_by_id(&self, id: RegionId) -> impl Future> { - self.pd.clone().get_region_by_id(id) - } - - fn get_region(&self, key: &Key) -> impl Future> { - self.pd.clone().get_region(key.into()) - } - - fn kv_client(&self, a: &impl Address) -> Result> { - let address = a.address(); - if let Some(client) = self.tikv.read().unwrap().get(address) { - return Ok(client.clone()); - }; - info!("connect to tikv endpoint: {:?}", address); - let tikv = self.tikv.clone(); - KvClient::connect(self.env.clone(), address, &self.security_mgr, self.timeout) - .map(Arc::new) - .map(|c| { - tikv.write().unwrap().insert(address.to_owned(), c.clone()); - c - }) - } - - fn store_for_key(self: Arc, key: &Key) -> impl Future> { - let region = self.get_region(key); - self.map_region_to_store(region) - } - - fn map_store_to_raw_context( - self: Arc, - region_ctx: impl Future>, - ) -> impl Future> { - region_ctx.ok_and_then(move |region_ctx| { - self.kv_client(®ion_ctx) - .map(|client| RawContext::new(region_ctx, client)) - }) - } - - fn map_region_to_store( - self: Arc, - region: impl Future>, - ) -> impl Future> { - region.and_then(move |region| { - let peer = region.peer().expect("leader must exist"); - let store_id = peer.get_store_id(); - self.load_store(store_id) - .map_ok(|store| Store { region, store }) - }) - } - - fn raw_from_id(self: Arc, id: RegionId) -> impl Future> { - let region = self.clone().load_region_by_id(id); - let store = self.clone().map_region_to_store(region); - self.map_store_to_raw_context(store) - } - - fn raw(self: Arc, key: &Key) -> impl Future> { - let store = self.clone().store_for_key(key); - self.map_store_to_raw_context(store) - } - - fn txn(self: Arc, key: &Key) -> impl Future> { - self.store_for_key(key) - .map_ok(|region_ctx| TxnContext::new(region_ctx)) - } -} - -impl fmt::Debug for RpcClient { - fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { - fmt.debug_struct("tikv-client") - .field("pd", &self.pd) - .finish() - } -} - -trait GroupingTask: Clone + Default + Sized { - fn key(&self) -> &Key; -} - -impl GroupingTask for Key { - fn key(&self) -> &Key { - self - } -} - -impl GroupingTask for KvPair { - fn key(&self) -> &Key { - self.key() - } -} - -impl GroupingTask for (Key, Option) { - fn key(&self) -> &Key { - &self.0 - } -} - -#[cfg(test)] -mod test { - use super::*; - use crate::rpc::pd::Timestamp; - use futures::executor; - use futures::future::{ready, BoxFuture}; - - struct MockPdClient {} - - impl PdClient for MockPdClient { - fn connect( - _env: Arc, - _endpoints: &[String], - _security_mgr: Arc, - _timeout: Duration, - ) -> Result { - Ok(MockPdClient {}) - } - - fn get_region(self: Arc, key: &[u8]) -> BoxFuture<'static, Result> { - let region = if key.len() <= 1 { - let mut region = Region::default(); - region.region.set_start_key(vec![0]); - region.region.set_end_key(vec![4]); - region - } else { - let mut region = Region::default(); - region.region.set_start_key(vec![4]); - region.region.set_end_key(vec![8]); - region - }; - - Box::pin(ready(Ok(region))) - } - - fn get_region_by_id(self: Arc, _id: RegionId) -> BoxFuture<'static, Result> { - unimplemented!(); - } - - fn get_store(self: Arc, _id: StoreId) -> BoxFuture<'static, Result> { - unimplemented!(); - } - - fn get_all_stores(self: Arc) -> BoxFuture<'static, Result>> { - unimplemented!(); - } - - fn get_timestamp(self: Arc) -> BoxFuture<'static, Result> { - unimplemented!(); - } - } - - fn mock_rpc_client() -> RpcClient { - let config = Config::default(); - let env = Arc::new( - EnvBuilder::new() - .cq_count(CQ_COUNT) - .name_prefix(thread_name!(CLIENT_PREFIX)) - .build(), - ); - RpcClient { - pd: Arc::new(MockPdClient {}), - tikv: Default::default(), - env, - security_mgr: Arc::new(SecurityManager::default()), - timeout: config.timeout, - } - } - - #[test] - fn test_kv_client() { - let client = mock_rpc_client(); - let addr1 = "foo"; - let addr2 = "bar"; - - let kv1 = client.kv_client(&addr1).unwrap(); - let kv2 = client.kv_client(&addr2).unwrap(); - let kv3 = client.kv_client(&addr2).unwrap(); - assert!(&*kv1 as *const _ != &*kv2 as *const _); - assert_eq!(&*kv2 as *const _, &*kv3 as *const _); - } - - #[test] - fn test_group_tasks_by_region() { - let client = mock_rpc_client(); - - let tasks: Vec = vec![ - vec![1].into(), - vec![2].into(), - vec![3].into(), - vec![5, 1].into(), - vec![5, 2].into(), - ]; - - let stream = Arc::new(client).group_tasks_by_region(tasks); - let mut stream = executor::block_on_stream(stream); - - assert_eq!( - stream.next().unwrap().unwrap().1, - vec![vec![1].into(), vec![2].into(), vec![3].into()] - ); - assert_eq!( - stream.next().unwrap().unwrap().1, - vec![vec![5, 1].into(), vec![5, 2].into()] - ); - assert!(stream.next().is_none()); - } -} diff --git a/src/rpc/context.rs b/src/rpc/context.rs deleted file mode 100644 index 7b1c0a0..0000000 --- a/src/rpc/context.rs +++ /dev/null @@ -1,48 +0,0 @@ -// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. - -use std::time::Instant; - -use prometheus::{HistogramVec, IntCounterVec}; - -use crate::{rpc::util::duration_to_sec, Result}; - -pub struct RequestContext { - start: Instant, - cmd: &'static str, - duration: &'static HistogramVec, - failed_duration: &'static HistogramVec, - failed_counter: &'static IntCounterVec, -} - -impl RequestContext { - pub fn new( - cmd: &'static str, - duration: &'static HistogramVec, - counter: &'static IntCounterVec, - failed_duration: &'static HistogramVec, - failed_counter: &'static IntCounterVec, - ) -> Self { - counter.with_label_values(&[cmd]).inc(); - RequestContext { - start: Instant::now(), - cmd, - duration, - failed_duration, - failed_counter, - } - } - - pub fn done(&self, r: Result) -> Result { - if r.is_ok() { - self.duration - .with_label_values(&[self.cmd]) - .observe(duration_to_sec(self.start.elapsed())); - } else { - self.failed_duration - .with_label_values(&[self.cmd]) - .observe(duration_to_sec(self.start.elapsed())); - self.failed_counter.with_label_values(&[self.cmd]).inc(); - } - r - } -} diff --git a/src/rpc/mod.rs b/src/rpc/mod.rs deleted file mode 100644 index 92eb8e7..0000000 --- a/src/rpc/mod.rs +++ /dev/null @@ -1,80 +0,0 @@ -// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. - -#[macro_use] -mod util; -mod client; -mod context; -mod pd; -mod security; -mod tikv; - -pub(crate) use crate::rpc::client::RpcClient; -pub use crate::rpc::pd::Timestamp; - -use derive_new::new; -use kvproto::{kvrpcpb, metapb}; -use std::sync::Arc; - -use crate::{ - rpc::{pd::Region, tikv::KvClient}, - Key, -}; - -/// A single KV store. -struct Store { - region: Region, - store: metapb::Store, -} - -impl Store { - fn start_key(&self) -> Key { - self.region.start_key().to_vec().into() - } - - fn end_key(&self) -> Key { - self.region.end_key().to_vec().into() - } - - fn range(&self) -> (Key, Key) { - (self.start_key(), self.end_key()) - } -} - -/// An object which can be identified by an IP address. -trait Address { - fn address(&self) -> &str; -} - -impl Address for Store { - fn address(&self) -> &str { - self.store.get_address() - } -} - -#[cfg(test)] -impl Address for &'static str { - fn address(&self) -> &str { - self - } -} - -impl From for kvrpcpb::Context { - fn from(mut store: Store) -> kvrpcpb::Context { - let mut kvctx = kvrpcpb::Context::default(); - kvctx.set_region_id(store.region.id()); - kvctx.set_region_epoch(store.region.region.take_region_epoch()); - kvctx.set_peer(store.region.peer().expect("leader must exist")); - kvctx - } -} - -#[derive(new)] -struct RawContext { - store: Store, - client: Arc, -} - -#[derive(new)] -struct TxnContext { - store: Store, -} diff --git a/src/rpc/pd/context.rs b/src/rpc/pd/context.rs deleted file mode 100644 index e125035..0000000 --- a/src/rpc/pd/context.rs +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. - -use prometheus::{Histogram, HistogramVec, IntCounterVec}; - -use crate::rpc::context::RequestContext; - -pub fn request_context(cmd: &'static str) -> RequestContext { - RequestContext::new( - cmd, - &PD_REQUEST_DURATION_HISTOGRAM_VEC, - &PD_REQUEST_COUNTER_VEC, - &PD_FAILED_REQUEST_DURATION_HISTOGRAM_VEC, - &PD_FAILED_REQUEST_COUNTER_VEC, - ) -} - -pub fn observe_tso_batch(batch_size: usize) { - PD_TSO_BATCH_SIZE_HISTOGRAM.observe(batch_size as f64); -} - -lazy_static! { - static ref PD_REQUEST_DURATION_HISTOGRAM_VEC: HistogramVec = register_histogram_vec!( - "pd_request_duration_seconds", - "Bucketed histogram of PD requests duration", - &["type"] - ) - .unwrap(); - static ref PD_REQUEST_COUNTER_VEC: IntCounterVec = register_int_counter_vec!( - "pd_request_total", - "Total number of requests sent to PD", - &["type"] - ) - .unwrap(); - static ref PD_FAILED_REQUEST_DURATION_HISTOGRAM_VEC: HistogramVec = register_histogram_vec!( - "pd_failed_request_duration_seconds", - "Bucketed histogram of failed PD requests duration", - &["type"] - ) - .unwrap(); - static ref PD_FAILED_REQUEST_COUNTER_VEC: IntCounterVec = register_int_counter_vec!( - "pd_failed_request_total", - "Total number of failed requests sent to PD", - &["type"] - ) - .unwrap(); - static ref PD_TSO_BATCH_SIZE_HISTOGRAM: Histogram = register_histogram!( - "pd_tso_batch_size", - "Bucketed histogram of TSO request batch size" - ) - .unwrap(); -} diff --git a/src/rpc/tikv/client.rs b/src/rpc/tikv/client.rs deleted file mode 100644 index 2164e8b..0000000 --- a/src/rpc/tikv/client.rs +++ /dev/null @@ -1,702 +0,0 @@ -// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. - -// TODO: Remove this when txn is done. -#![allow(dead_code)] - -use std::{fmt, sync::Arc, time::Duration}; - -use futures::compat::Compat01As03; -use futures::prelude::*; -use grpcio::{CallOption, Environment}; -use kvproto::{errorpb, kvrpcpb, tikvpb::TikvClient}; - -use crate::{ - raw::ColumnFamily, - rpc::{ - security::SecurityManager, - tikv::context::{request_context, RequestContext}, - Store, TxnContext, - }, - transaction::TxnInfo, - Error, ErrorKind, Key, KvPair, Result, Value, -}; - -trait HasRegionError { - fn region_error(&mut self) -> Option; -} - -trait HasError { - fn error(&mut self) -> Option; -} - -impl From for Error { - fn from(mut e: errorpb::Error) -> Error { - let message = e.take_message(); - if e.has_not_leader() { - let e = e.get_not_leader(); - let message = format!("{}. Leader: {:?}", message, e.get_leader()); - Error::not_leader(e.get_region_id(), Some(message)) - } else if e.has_region_not_found() { - Error::region_not_found(e.get_region_not_found().get_region_id(), Some(message)) - } else if e.has_key_not_in_region() { - let e = e.take_key_not_in_region(); - Error::key_not_in_region(e) - } else if e.has_epoch_not_match() { - Error::stale_epoch(Some(format!( - "{}. New epoch: {:?}", - message, - e.get_epoch_not_match().get_current_regions() - ))) - } else if e.has_server_is_busy() { - Error::server_is_busy(e.take_server_is_busy()) - } else if e.has_stale_command() { - Error::stale_command(message) - } else if e.has_store_not_match() { - Error::store_not_match(e.take_store_not_match(), message) - } else if e.has_raft_entry_too_large() { - Error::raft_entry_too_large(e.take_raft_entry_too_large(), message) - } else { - Error::internal_error(message) - } - } -} - -macro_rules! has_region_error { - ($type:ty) => { - impl HasRegionError for $type { - fn region_error(&mut self) -> Option { - if self.has_region_error() { - Some(self.take_region_error().into()) - } else { - None - } - } - } - }; -} - -has_region_error!(kvrpcpb::GetResponse); -has_region_error!(kvrpcpb::ScanResponse); -has_region_error!(kvrpcpb::PrewriteResponse); -has_region_error!(kvrpcpb::CommitResponse); -has_region_error!(kvrpcpb::ImportResponse); -has_region_error!(kvrpcpb::BatchRollbackResponse); -has_region_error!(kvrpcpb::CleanupResponse); -has_region_error!(kvrpcpb::BatchGetResponse); -has_region_error!(kvrpcpb::ScanLockResponse); -has_region_error!(kvrpcpb::ResolveLockResponse); -has_region_error!(kvrpcpb::GcResponse); -has_region_error!(kvrpcpb::RawGetResponse); -has_region_error!(kvrpcpb::RawBatchGetResponse); -has_region_error!(kvrpcpb::RawPutResponse); -has_region_error!(kvrpcpb::RawBatchPutResponse); -has_region_error!(kvrpcpb::RawDeleteResponse); -has_region_error!(kvrpcpb::RawBatchDeleteResponse); -has_region_error!(kvrpcpb::DeleteRangeResponse); -has_region_error!(kvrpcpb::RawDeleteRangeResponse); -has_region_error!(kvrpcpb::RawScanResponse); -has_region_error!(kvrpcpb::RawBatchScanResponse); - -macro_rules! has_key_error { - ($type:ty) => { - impl HasError for $type { - fn error(&mut self) -> Option { - if self.has_error() { - Some(self.take_error().into()) - } else { - None - } - } - } - }; -} - -has_key_error!(kvrpcpb::GetResponse); -has_key_error!(kvrpcpb::CommitResponse); -has_key_error!(kvrpcpb::BatchRollbackResponse); -has_key_error!(kvrpcpb::CleanupResponse); -has_key_error!(kvrpcpb::ScanLockResponse); -has_key_error!(kvrpcpb::ResolveLockResponse); -has_key_error!(kvrpcpb::GcResponse); - -macro_rules! has_str_error { - ($type:ty) => { - impl HasError for $type { - fn error(&mut self) -> Option { - if self.get_error().is_empty() { - None - } else { - Some(Error::kv_error(self.take_error())) - } - } - } - }; -} - -has_str_error!(kvrpcpb::RawGetResponse); -has_str_error!(kvrpcpb::RawPutResponse); -has_str_error!(kvrpcpb::RawBatchPutResponse); -has_str_error!(kvrpcpb::RawDeleteResponse); -has_str_error!(kvrpcpb::RawBatchDeleteResponse); -has_str_error!(kvrpcpb::RawDeleteRangeResponse); -has_str_error!(kvrpcpb::ImportResponse); -has_str_error!(kvrpcpb::DeleteRangeResponse); - -macro_rules! has_no_error { - ($type:ty) => { - impl HasError for $type { - fn error(&mut self) -> Option { - None - } - } - }; -} - -has_no_error!(kvrpcpb::ScanResponse); -has_no_error!(kvrpcpb::PrewriteResponse); -has_no_error!(kvrpcpb::BatchGetResponse); -has_no_error!(kvrpcpb::RawBatchGetResponse); -has_no_error!(kvrpcpb::RawScanResponse); -has_no_error!(kvrpcpb::RawBatchScanResponse); - -macro_rules! raw_request { - ($region:expr, $cf:expr, $type:ty) => {{ - let mut req = <$type>::default(); - req.set_context($region.into()); - if let Some(cf) = $cf { - req.set_cf(cf.to_string()); - } - req - }}; -} - -macro_rules! txn_request { - ($context:expr, $type:ty) => {{ - let mut req = <$type>::default(); - req.set_context($context.store.into()); - req - }}; -} - -impl From for kvrpcpb::TxnInfo { - fn from(txn_info: TxnInfo) -> kvrpcpb::TxnInfo { - let mut pb = kvrpcpb::TxnInfo::default(); - pb.set_txn(txn_info.txn); - pb.set_status(txn_info.status); - pb - } -} - -pub struct KvClient { - client: Arc, - timeout: Duration, -} - -impl KvClient { - pub(in crate::rpc) fn connect( - env: Arc, - addr: &str, - security_mgr: &Arc, - timeout: Duration, - ) -> Result { - let client = Arc::new(security_mgr.connect(env, addr, TikvClient::new)?); - Ok(KvClient { client, timeout }) - } - - pub(in crate::rpc) fn kv_get( - &self, - context: TxnContext, - version: u64, - key: Key, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::GetRequest); - req.set_key(key.into()); - req.set_version(version); - - self.execute( - request_context("kv_get"), - move |cli: Arc, opt: _| { - cli.kv_get_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_scan( - &self, - context: TxnContext, - version: u64, - start_key: Key, - end_key: Key, - limit: u32, - key_only: bool, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::ScanRequest); - req.set_start_key(start_key.into()); - req.set_end_key(end_key.into()); - req.set_version(version); - req.set_limit(limit); - req.set_key_only(key_only); - - self.execute( - request_context("kv_scan"), - move |cli: Arc, opt: _| { - cli.kv_scan_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_prewrite( - &self, - context: TxnContext, - mutations: impl Iterator, - primary_lock: Key, - start_version: u64, - lock_ttl: u64, - skip_constraint_check: bool, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::PrewriteRequest); - req.set_mutations(mutations.collect()); - req.set_primary_lock(primary_lock.into()); - req.set_start_version(start_version); - req.set_lock_ttl(lock_ttl); - req.set_skip_constraint_check(skip_constraint_check); - - self.execute( - request_context("kv_prewrite"), - move |cli: Arc, opt: _| { - cli.kv_prewrite_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_commit( - &self, - context: TxnContext, - keys: impl Iterator, - start_version: u64, - commit_version: u64, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::CommitRequest); - req.set_keys(keys.map(|x| x.into()).collect()); - req.set_start_version(start_version); - req.set_commit_version(commit_version); - - self.execute( - request_context("kv_commit"), - move |cli: Arc, opt: _| { - cli.kv_commit_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_import( - &self, - mutations: impl Iterator, - commit_version: u64, - ) -> impl Future> { - let mut req = kvrpcpb::ImportRequest::default(); - req.set_mutations(mutations.collect()); - req.set_commit_version(commit_version); - - self.execute( - request_context("kv_import"), - move |cli: Arc, opt: _| { - cli.kv_import_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_cleanup( - &self, - context: TxnContext, - key: Key, - start_version: u64, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::CleanupRequest); - req.set_key(key.into()); - req.set_start_version(start_version); - - self.execute( - request_context("kv_cleanup"), - move |cli: Arc, opt: _| { - cli.kv_cleanup_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_batch_get( - &self, - context: TxnContext, - keys: impl Iterator, - version: u64, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::BatchGetRequest); - req.set_keys(keys.map(|x| x.into()).collect()); - req.set_version(version); - - self.execute( - request_context("kv_batch_get"), - move |cli: Arc, opt: _| { - cli.kv_batch_get_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_batch_rollback( - &self, - context: TxnContext, - keys: impl Iterator, - start_version: u64, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::BatchRollbackRequest); - req.set_keys(keys.map(|x| x.into()).collect()); - req.set_start_version(start_version); - - self.execute( - request_context("kv_batch_rollback"), - move |cli: Arc, opt: _| { - cli.kv_batch_rollback_async_opt(&req, opt) - .map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_scan_lock( - &self, - context: TxnContext, - start_key: Key, - max_version: u64, - limit: u32, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::ScanLockRequest); - req.set_start_key(start_key.into()); - req.set_max_version(max_version); - req.set_limit(limit); - - self.execute( - request_context("kv_scan_lock"), - move |cli: Arc, opt: _| { - cli.kv_scan_lock_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_resolve_lock( - &self, - context: TxnContext, - txn_infos: impl Iterator, - start_version: u64, - commit_version: u64, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::ResolveLockRequest); - req.set_start_version(start_version); - req.set_commit_version(commit_version); - req.set_txn_infos(txn_infos.map(Into::into).collect()); - - self.execute( - request_context("kv_resolve_lock"), - move |cli: Arc, opt: _| { - cli.kv_resolve_lock_async_opt(&req, opt) - .map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_gc( - &self, - context: TxnContext, - safe_point: u64, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::GcRequest); - req.set_safe_point(safe_point); - - self.execute( - request_context("kv_gc"), - move |cli: Arc, opt: _| { - cli.kv_gc_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn kv_delete_range( - &self, - context: TxnContext, - start_key: Key, - end_key: Key, - ) -> impl Future> { - let mut req = txn_request!(context, kvrpcpb::DeleteRangeRequest); - req.set_start_key(start_key.into()); - req.set_end_key(end_key.into()); - - self.execute( - request_context("kv_delete_range"), - move |cli: Arc, opt: _| { - cli.kv_delete_range_async_opt(&req, opt) - .map(Compat01As03::new) - }, - ) - } - - pub(in crate::rpc) fn raw_get( - &self, - store: Store, - cf: Option, - key: Key, - ) -> impl Future> { - let mut req = raw_request!(store, cf, kvrpcpb::RawGetRequest); - req.set_key(key.into()); - - self.execute( - request_context("raw_get"), - move |cli: Arc, opt: _| { - cli.raw_get_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - .map_ok(|mut resp| resp.take_value().into()) - } - - pub(in crate::rpc) fn raw_batch_get( - &self, - store: Store, - cf: Option, - keys: impl Iterator, - ) -> impl Future>> { - let mut req = raw_request!(store, cf, kvrpcpb::RawBatchGetRequest); - req.set_keys(keys.map(|x| x.into()).collect()); - - self.execute( - request_context("raw_batch_get"), - move |cli: Arc, opt: _| { - cli.raw_batch_get_async_opt(&req, opt) - .map(Compat01As03::new) - }, - ) - .map_ok(|mut resp| Self::convert_from_grpc_pairs(resp.take_pairs())) - } - - pub(in crate::rpc) fn raw_put( - &self, - store: Store, - cf: Option, - key: Key, - value: Value, - ) -> impl Future> { - let mut req = raw_request!(store, cf, kvrpcpb::RawPutRequest); - req.set_key(key.into()); - req.set_value(value.into()); - - self.execute( - request_context("raw_put"), - move |cli: Arc, opt: _| { - cli.raw_put_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - .map_ok(|_| ()) - } - - pub(in crate::rpc) fn raw_batch_put( - &self, - store: Store, - cf: Option, - pairs: Vec, - ) -> impl Future> { - let mut req = raw_request!(store, cf, kvrpcpb::RawBatchPutRequest); - req.set_pairs(Self::convert_to_grpc_pairs(pairs)); - - self.execute( - request_context("raw_batch_put"), - move |cli: Arc, opt: _| { - cli.raw_batch_put_async_opt(&req, opt) - .map(Compat01As03::new) - }, - ) - .map_ok(|_| ()) - } - - pub(in crate::rpc) fn raw_delete( - &self, - store: Store, - cf: Option, - key: Key, - ) -> impl Future> { - let mut req = raw_request!(store, cf, kvrpcpb::RawDeleteRequest); - req.set_key(key.into()); - - self.execute( - request_context("raw_delete"), - move |cli: Arc, opt: _| { - cli.raw_delete_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - .map_ok(|_| ()) - } - - pub(in crate::rpc) fn raw_batch_delete( - &self, - store: Store, - cf: Option, - keys: Vec, - ) -> impl Future> { - let mut req = raw_request!(store, cf, kvrpcpb::RawBatchDeleteRequest); - req.set_keys(keys.into_iter().map(|x| x.into()).collect()); - - self.execute( - request_context("raw_batch_delete"), - move |cli: Arc, opt: _| { - cli.raw_batch_delete_async_opt(&req, opt) - .map(Compat01As03::new) - }, - ) - .map_ok(|_| ()) - } - - pub(in crate::rpc) fn raw_scan( - &self, - store: Store, - cf: Option, - start_key: Key, - end_key: Option, - limit: u32, - key_only: bool, - ) -> impl Future>> { - let mut req = raw_request!(store, cf, kvrpcpb::RawScanRequest); - req.set_start_key(start_key.into()); - // FIXME we shouldn't panic when there is no end_key - end_key.map(|k| req.set_end_key(k.into())).unwrap(); - req.set_limit(limit); - req.set_key_only(key_only); - - self.execute( - request_context("raw_scan"), - move |cli: Arc, opt: _| { - cli.raw_scan_async_opt(&req, opt).map(Compat01As03::new) - }, - ) - .map_ok(|mut resp| Self::convert_from_grpc_pairs(resp.take_kvs())) - } - - pub(in crate::rpc) fn raw_batch_scan( - &self, - store: Store, - cf: Option, - ranges: impl Iterator, Option)>, - each_limit: u32, - key_only: bool, - ) -> impl Future>> { - let mut req = raw_request!(store, cf, kvrpcpb::RawBatchScanRequest); - req.set_ranges(Self::convert_to_grpc_ranges(ranges)); - req.set_each_limit(each_limit); - req.set_key_only(key_only); - - self.execute( - request_context("raw_batch_scan"), - move |cli: Arc, opt: _| { - cli.raw_batch_scan_async_opt(&req, opt) - .map(Compat01As03::new) - }, - ) - .map_ok(|mut resp| Self::convert_from_grpc_pairs(resp.take_kvs())) - } - - pub(in crate::rpc) fn raw_delete_range( - &self, - store: Store, - cf: Option, - start_key: Key, - end_key: Key, - ) -> impl Future> { - let mut req = raw_request!(store, cf, kvrpcpb::RawDeleteRangeRequest); - req.set_start_key(start_key.into()); - req.set_end_key(end_key.into()); - - self.execute( - request_context("raw_delete_range"), - move |cli: Arc, opt: _| { - cli.raw_delete_range_async_opt(&req, opt) - .map(Compat01As03::new) - }, - ) - .map_ok(|_| ()) - } - - fn execute( - &self, - context: RequestContext, - executor: Executor, - ) -> impl Future> - where - Executor: FnOnce(Arc, CallOption) -> ::grpcio::Result, - RpcFuture: Future>, - Resp: HasRegionError + HasError + Sized + Clone, - { - executor( - Arc::clone(&self.client), - CallOption::default().timeout(self.timeout), - ) - .unwrap() - .map(|r| match r { - Err(e) => Err(ErrorKind::Grpc(e).into()), - Ok(mut r) => { - if let Some(e) = r.region_error() { - Err(e) - } else if let Some(e) = r.error() { - Err(e) - } else { - Ok(r) - } - } - }) - .map(move |r| context.done(r)) - } - - #[inline] - fn convert_to_grpc_pair(pair: KvPair) -> kvrpcpb::KvPair { - let mut result = kvrpcpb::KvPair::default(); - let (key, value) = pair.into(); - result.set_key(key.into()); - result.set_value(value.into()); - result - } - - #[inline] - fn convert_to_grpc_pairs(pairs: Vec) -> Vec { - pairs.into_iter().map(Self::convert_to_grpc_pair).collect() - } - - #[inline] - fn convert_from_grpc_pair(mut pair: kvrpcpb::KvPair) -> KvPair { - KvPair::new(Key::from(pair.take_key()), Value::from(pair.take_value())) - } - - #[inline] - fn convert_from_grpc_pairs(pairs: Vec) -> Vec { - pairs - .into_iter() - .map(Self::convert_from_grpc_pair) - .collect() - } - - #[inline] - fn convert_to_grpc_range(range: (Option, Option)) -> kvrpcpb::KeyRange { - let (start, end) = range; - let mut range = kvrpcpb::KeyRange::default(); - start.map(|k| range.set_start_key(k.into())).unwrap(); - end.map(|k| range.set_end_key(k.into())).unwrap(); - range - } - - #[inline] - fn convert_to_grpc_ranges( - ranges: impl Iterator, Option)>, - ) -> Vec { - ranges.map(Self::convert_to_grpc_range).collect() - } -} - -impl fmt::Debug for KvClient { - fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { - fmt.debug_struct("KvClient") - .field("timeout", &self.timeout) - .finish() - } -} diff --git a/src/rpc/tikv/context.rs b/src/rpc/tikv/context.rs deleted file mode 100644 index 5a501bb..0000000 --- a/src/rpc/tikv/context.rs +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. - -use prometheus::{HistogramVec, IntCounterVec}; - -pub use crate::rpc::context::RequestContext; - -pub fn request_context(cmd: &'static str) -> RequestContext { - RequestContext::new( - cmd, - &TIKV_REQUEST_DURATION_HISTOGRAM_VEC, - &TIKV_REQUEST_COUNTER_VEC, - &TIKV_FAILED_REQUEST_DURATION_HISTOGRAM_VEC, - &TIKV_FAILED_REQUEST_COUNTER_VEC, - ) -} - -lazy_static! { - static ref TIKV_REQUEST_DURATION_HISTOGRAM_VEC: HistogramVec = register_histogram_vec!( - "tikv_request_duration_seconds", - "Bucketed histogram of TiKV requests duration", - &["type"] - ) - .unwrap(); - static ref TIKV_REQUEST_COUNTER_VEC: IntCounterVec = register_int_counter_vec!( - "tikv_request_total", - "Total number of requests sent to TiKV", - &["type"] - ) - .unwrap(); - static ref TIKV_FAILED_REQUEST_DURATION_HISTOGRAM_VEC: HistogramVec = register_histogram_vec!( - "tikv_failed_request_duration_seconds", - "Bucketed histogram of failed TiKV requests duration", - &["type"] - ) - .unwrap(); - static ref TIKV_FAILED_REQUEST_COUNTER_VEC: IntCounterVec = register_int_counter_vec!( - "tikv_failed_request_total", - "Total number of failed requests sent to TiKV", - &["type"] - ) - .unwrap(); -} diff --git a/src/rpc/tikv/mod.rs b/src/rpc/tikv/mod.rs deleted file mode 100644 index 0830821..0000000 --- a/src/rpc/tikv/mod.rs +++ /dev/null @@ -1,6 +0,0 @@ -// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. - -pub mod client; -pub mod context; - -pub use self::client::KvClient; diff --git a/src/rpc/security.rs b/src/security.rs similarity index 100% rename from src/rpc/security.rs rename to src/security.rs diff --git a/src/stats.rs b/src/stats.rs new file mode 100644 index 0000000..c32c14c --- /dev/null +++ b/src/stats.rs @@ -0,0 +1,128 @@ +// Copyright 2018 TiKV Project Authors. Licensed under Apache-2.0. + +use prometheus::{Histogram, HistogramVec, IntCounterVec}; +use std::time::Instant; + +use crate::{util::duration_to_sec, Result}; + +pub struct RequestStats { + start: Instant, + cmd: &'static str, + duration: &'static HistogramVec, + failed_duration: &'static HistogramVec, + failed_counter: &'static IntCounterVec, +} + +impl RequestStats { + pub fn new( + cmd: &'static str, + duration: &'static HistogramVec, + counter: &'static IntCounterVec, + failed_duration: &'static HistogramVec, + failed_counter: &'static IntCounterVec, + ) -> Self { + counter.with_label_values(&[cmd]).inc(); + RequestStats { + start: Instant::now(), + cmd, + duration, + failed_duration, + failed_counter, + } + } + + pub fn done(&self, r: Result) -> Result { + if r.is_ok() { + self.duration + .with_label_values(&[self.cmd]) + .observe(duration_to_sec(self.start.elapsed())); + } else { + self.failed_duration + .with_label_values(&[self.cmd]) + .observe(duration_to_sec(self.start.elapsed())); + self.failed_counter.with_label_values(&[self.cmd]).inc(); + } + r + } +} + +pub fn tikv_stats(cmd: &'static str) -> RequestStats { + RequestStats::new( + cmd, + &TIKV_REQUEST_DURATION_HISTOGRAM_VEC, + &TIKV_REQUEST_COUNTER_VEC, + &TIKV_FAILED_REQUEST_DURATION_HISTOGRAM_VEC, + &TIKV_FAILED_REQUEST_COUNTER_VEC, + ) +} + +pub fn pd_stats(cmd: &'static str) -> RequestStats { + RequestStats::new( + cmd, + &PD_REQUEST_DURATION_HISTOGRAM_VEC, + &PD_REQUEST_COUNTER_VEC, + &PD_FAILED_REQUEST_DURATION_HISTOGRAM_VEC, + &PD_FAILED_REQUEST_COUNTER_VEC, + ) +} + +#[allow(dead_code)] +pub fn observe_tso_batch(batch_size: usize) { + PD_TSO_BATCH_SIZE_HISTOGRAM.observe(batch_size as f64); +} + +lazy_static! { + static ref TIKV_REQUEST_DURATION_HISTOGRAM_VEC: HistogramVec = register_histogram_vec!( + "tikv_request_duration_seconds", + "Bucketed histogram of TiKV requests duration", + &["type"] + ) + .unwrap(); + static ref TIKV_REQUEST_COUNTER_VEC: IntCounterVec = register_int_counter_vec!( + "tikv_request_total", + "Total number of requests sent to TiKV", + &["type"] + ) + .unwrap(); + static ref TIKV_FAILED_REQUEST_DURATION_HISTOGRAM_VEC: HistogramVec = register_histogram_vec!( + "tikv_failed_request_duration_seconds", + "Bucketed histogram of failed TiKV requests duration", + &["type"] + ) + .unwrap(); + static ref TIKV_FAILED_REQUEST_COUNTER_VEC: IntCounterVec = register_int_counter_vec!( + "tikv_failed_request_total", + "Total number of failed requests sent to TiKV", + &["type"] + ) + .unwrap(); + static ref PD_REQUEST_DURATION_HISTOGRAM_VEC: HistogramVec = register_histogram_vec!( + "pd_request_duration_seconds", + "Bucketed histogram of PD requests duration", + &["type"] + ) + .unwrap(); + static ref PD_REQUEST_COUNTER_VEC: IntCounterVec = register_int_counter_vec!( + "pd_request_total", + "Total number of requests sent to PD", + &["type"] + ) + .unwrap(); + static ref PD_FAILED_REQUEST_DURATION_HISTOGRAM_VEC: HistogramVec = register_histogram_vec!( + "pd_failed_request_duration_seconds", + "Bucketed histogram of failed PD requests duration", + &["type"] + ) + .unwrap(); + static ref PD_FAILED_REQUEST_COUNTER_VEC: IntCounterVec = register_int_counter_vec!( + "pd_failed_request_total", + "Total number of failed requests sent to PD", + &["type"] + ) + .unwrap(); + static ref PD_TSO_BATCH_SIZE_HISTOGRAM: Histogram = register_histogram!( + "pd_tso_batch_size", + "Bucketed histogram of TSO request batch size" + ) + .unwrap(); +} diff --git a/src/transaction/client.rs b/src/transaction/client.rs index 25162bd..85fe357 100644 --- a/src/transaction/client.rs +++ b/src/transaction/client.rs @@ -1,7 +1,7 @@ // Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. use super::{Snapshot, Timestamp, Transaction}; -use crate::rpc::RpcClient; +use crate::pd::{PdClient, PdRpcClient}; use crate::{Config, Result}; use derive_new::new; @@ -12,7 +12,7 @@ use std::sync::Arc; /// The TiKV transactional `Client` is used to issue requests to the TiKV server and PD cluster. pub struct Client { - rpc: Arc, + pd: Arc, } impl Client { @@ -20,10 +20,10 @@ impl Client { /// /// ```rust,no_run /// # #![feature(async_await)] - /// use tikv_client::{Config, transaction::Client}; + /// use tikv_client::{Config, TransactionClient}; /// use futures::prelude::*; /// # futures::executor::block_on(async { - /// let connect = Client::connect(Config::default()); + /// let connect = TransactionClient::connect(Config::default()); /// let client = connect.await.unwrap(); /// # }); /// ``` @@ -37,10 +37,10 @@ impl Client { /// /// ```rust,no_run /// # #![feature(async_await)] - /// use tikv_client::{Config, transaction::Client}; + /// use tikv_client::{Config, TransactionClient}; /// use futures::prelude::*; /// # futures::executor::block_on(async { - /// let connect = Client::connect(Config::default()); + /// let connect = TransactionClient::connect(Config::default()); /// let client = connect.await.unwrap(); /// let mut transaction = client.begin().await.unwrap(); /// // ... Issue some commands. @@ -57,10 +57,10 @@ impl Client { /// /// ```rust,no_run /// # #![feature(async_await)] - /// use tikv_client::{Config, transaction::Client}; + /// use tikv_client::{Config, TransactionClient}; /// use futures::prelude::*; /// # futures::executor::block_on(async { - /// let connect = Client::connect(Config::default()); + /// let connect = TransactionClient::connect(Config::default()); /// let client = connect.await.unwrap(); /// let snapshot = client.snapshot().await.unwrap(); /// // ... Issue some commands. @@ -75,10 +75,10 @@ impl Client { /// /// ```rust,no_run /// # #![feature(async_await)] - /// use tikv_client::{Config, transaction::{Client, Timestamp}}; + /// use tikv_client::{Config, TransactionClient, Timestamp}; /// use futures::prelude::*; /// # futures::executor::block_on(async { - /// let connect = Client::connect(Config::default()); + /// let connect = TransactionClient::connect(Config::default()); /// let client = connect.await.unwrap(); /// let timestamp = Timestamp { physical: 1564481750172, logical: 1 }; /// let snapshot = client.snapshot_at(timestamp); @@ -93,16 +93,16 @@ impl Client { /// /// ```rust,no_run /// # #![feature(async_await)] - /// use tikv_client::{Config, transaction::Client}; + /// use tikv_client::{Config, TransactionClient}; /// use futures::prelude::*; /// # futures::executor::block_on(async { - /// let connect = Client::connect(Config::default()); + /// let connect = TransactionClient::connect(Config::default()); /// let client = connect.await.unwrap(); /// let timestamp = client.current_timestamp().await.unwrap(); /// # }); /// ``` pub async fn current_timestamp(&self) -> Result { - self.rpc.clone().get_timestamp().await + self.pd.clone().get_timestamp().await } } @@ -112,12 +112,12 @@ impl Client { /// /// ```rust,no_run /// # #![feature(async_await)] -/// use tikv_client::{Config, transaction::{Client, Connect}}; +/// use tikv_client::{Config, TransactionClient, Connect}; /// use futures::prelude::*; /// /// # futures::executor::block_on(async { -/// let connect: Connect = Client::connect(Config::default()); -/// let client: Client = connect.await.unwrap(); +/// let connect: Connect = TransactionClient::connect(Config::default()); +/// let client: TransactionClient = connect.await.unwrap(); /// # }); /// ``` #[derive(new)] @@ -130,9 +130,9 @@ impl Future for Connect { fn poll(self: Pin<&mut Self>, _cx: &mut Context) -> Poll { let config = &self.config; - // TODO: RpcClient::connect currently uses a blocking implementation. + // TODO: PdRpcClient::connect currently uses a blocking implementation. // Make it asynchronous later. - let rpc = Arc::new(RpcClient::connect(config)?); - Poll::Ready(Ok(Client { rpc })) + let pd = Arc::new(PdRpcClient::connect(config)?); + Poll::Ready(Ok(Client { pd })) } } diff --git a/src/transaction/mod.rs b/src/transaction/mod.rs index 568aa00..2d250c9 100644 --- a/src/transaction/mod.rs +++ b/src/transaction/mod.rs @@ -2,7 +2,7 @@ //! Transactional related functionality. //! -//! Using the [`transaction::Client`](transaction::Client) you can utilize TiKV's transactional interface. +//! Using the [`TransactionClient`](TransactionClient) you can utilize TiKV's transactional interface. //! //! This interface offers SQL-like transactions on top of the raw interface. //! @@ -11,17 +11,20 @@ pub use self::client::{Client, Connect}; pub use self::requests::Scanner; -pub use self::transaction::{Snapshot, Transaction, TxnInfo}; -pub use super::rpc::Timestamp; +use crate::{Key, KvPair, Result, Value}; +use derive_new::new; use kvproto::kvrpcpb; - -use crate::{Key, Value}; +use std::{collections::BTreeMap, ops::RangeBounds}; mod client; -mod requests; -#[allow(clippy::module_inception)] -mod transaction; +pub(crate) mod requests; + +#[derive(Eq, PartialEq, Debug, Clone, Copy)] +pub struct Timestamp { + pub physical: i64, + pub logical: i64, +} #[derive(Debug, Clone)] pub enum Mutation { @@ -56,3 +59,330 @@ impl Mutation { } } } + +/// A undo-able set of actions on the dataset. +/// +/// Using a transaction you can prepare a set of actions (such as `get`, or `set`) on data at a +/// particular timestamp obtained from the placement driver. +/// +/// Once a transaction is commited, a new commit timestamp is obtained from the placement driver. +/// +/// Create a new transaction from a snapshot using `new`. +/// +/// ```rust,no_run +/// # #![feature(async_await)] +/// use tikv_client::{Config, TransactionClient}; +/// use futures::prelude::*; +/// # futures::executor::block_on(async { +/// let connect = TransactionClient::connect(Config::default()); +/// let client = connect.await.unwrap(); +/// let txn = client.begin().await.unwrap(); +/// # }); +/// ``` +#[derive(new)] +pub struct Transaction { + snapshot: Snapshot, + #[new(default)] + mutations: BTreeMap, +} + +impl Transaction { + /// Gets the value associated with the given key. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Value, Config, transaction::Client}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.await.unwrap(); + /// let mut txn = connected_client.begin().await.unwrap(); + /// let key = "TiKV".to_owned(); + /// let req = txn.get(key); + /// let result: Value = req.await.unwrap(); + /// // Finish the transaction... + /// txn.commit().await.unwrap(); + /// # }); + /// ``` + pub async fn get(&self, key: impl Into) -> Result { + let key = key.into(); + if let Some(value) = self.get_from_mutations(&key) { + Ok(value) + } else { + self.snapshot.get(key).await + } + } + + /// Gets the values associated with the given keys. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{KvPair, Config, transaction::Client}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.await.unwrap(); + /// let mut txn = connected_client.begin().await.unwrap(); + /// let keys = vec!["TiKV".to_owned(), "TiDB".to_owned()]; + /// let req = txn.batch_get(keys); + /// let result: Vec = req.await.unwrap(); + /// // Finish the transaction... + /// txn.commit().await.unwrap(); + /// # }); + /// ``` + pub async fn batch_get( + &self, + keys: impl IntoIterator>, + ) -> Result> { + let mut result = Vec::new(); + let mut keys_from_snapshot = Vec::new(); + let mut result_indices_from_snapshot = Vec::new(); + + // Try to fill the result vector from mutations + for key in keys { + let key = key.into(); + if let Some(value) = self.get_from_mutations(&key) { + result.push((key, value).into()); + } else { + keys_from_snapshot.push(key); + result_indices_from_snapshot.push(result.len()); + // Push a placeholder + result.push(KvPair::default()); + } + } + + // Get others from snapshot + let kv_pairs_from_snapshot = self + .snapshot + .batch_get(keys_from_snapshot.into_iter()) + .await?; + for (kv_pair, index) in kv_pairs_from_snapshot + .into_iter() + .zip(result_indices_from_snapshot) + { + result[index] = kv_pair; + } + + Ok(result) + } + + pub fn scan(&self, _range: impl RangeBounds) -> Scanner { + unimplemented!() + } + + pub fn scan_reverse(&self, _range: impl RangeBounds) -> Scanner { + unimplemented!() + } + + /// Sets the value associated with the given key. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Key, Value, Config, transaction::Client}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.await.unwrap(); + /// let mut txn = connected_client.begin().await.unwrap(); + /// let key = "TiKV".to_owned(); + /// let val = "TiKV".to_owned(); + /// txn.set(key, val); + /// // Finish the transaction... + /// txn.commit().await.unwrap(); + /// # }); + /// ``` + pub fn set(&mut self, key: impl Into, value: impl Into) { + self.mutations + .insert(key.into(), Mutation::Put(value.into())); + } + + /// Deletes the given key. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Key, Config, transaction::Client}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.await.unwrap(); + /// let mut txn = connected_client.begin().await.unwrap(); + /// let key = "TiKV".to_owned(); + /// txn.delete(key); + /// // Finish the transaction... + /// txn.commit().await.unwrap(); + /// # }); + /// ``` + pub fn delete(&mut self, key: impl Into) { + self.mutations.insert(key.into(), Mutation::Del); + } + + /// Locks the given keys. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Config, transaction::Client}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connect = Client::connect(Config::default()); + /// # let connected_client = connect.await.unwrap(); + /// let mut txn = connected_client.begin().await.unwrap(); + /// txn.lock_keys(vec!["TiKV".to_owned(), "Rust".to_owned()]); + /// // ... Do some actions. + /// txn.commit().await.unwrap(); + /// # }); + /// ``` + pub fn lock_keys(&mut self, keys: impl IntoIterator>) { + for key in keys { + let key = key.into(); + // Mutated keys don't need a lock. + self.mutations.entry(key).or_insert(Mutation::Lock); + } + } + + /// Commits the actions of the transaction. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Config, transaction::Client}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connect = Client::connect(Config::default()); + /// # let connected_client = connect.await.unwrap(); + /// let mut txn = connected_client.begin().await.unwrap(); + /// // ... Do some actions. + /// let req = txn.commit(); + /// let result: () = req.await.unwrap(); + /// # }); + /// ``` + pub async fn commit(&mut self) -> Result<()> { + self.prewrite().await?; + self.commit_primary().await?; + // FIXME: return from this method once the primary key is committed + let _ = self.commit_secondary().await; + Ok(()) + } + + /// Returns the timestamp which the transaction started at. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Config, transaction::{Client, Timestamp}}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connect = Client::connect(Config::default()); + /// # let connected_client = connect.await.unwrap(); + /// let txn = connected_client.begin().await.unwrap(); + /// // ... Do some actions. + /// let ts: Timestamp = txn.start_ts(); + /// # }); + /// ``` + pub fn start_ts(&self) -> Timestamp { + self.snapshot().timestamp + } + + /// Gets the `Snapshot` the transaction is operating on. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Config, transaction::{Client, Snapshot}}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connect = Client::connect(Config::default()); + /// # let connected_client = connect.await.unwrap(); + /// let txn = connected_client.begin().await.unwrap(); + /// // ... Do some actions. + /// let snap: &Snapshot = txn.snapshot(); + /// # }); + /// ``` + pub fn snapshot(&self) -> &Snapshot { + &self.snapshot + } + + async fn prewrite(&mut self) -> Result<()> { + // TODO: Too many clones. Consider using bytes::Byte. + let _rpc_mutations: Vec<_> = self + .mutations + .iter() + .map(|(k, v)| v.clone().with_key(k.clone())) + .collect(); + unimplemented!() + } + + async fn commit_primary(&mut self) -> Result<()> { + unimplemented!() + } + + async fn commit_secondary(&mut self) -> Result<()> { + unimplemented!() + } + + fn get_from_mutations(&self, key: &Key) -> Option { + self.mutations.get(key).and_then(Mutation::get_value) + } +} + +pub struct TxnInfo { + pub txn: u64, + pub status: u64, +} + +/// A snapshot of dataset at a particular point in time. +#[derive(new)] +pub struct Snapshot { + timestamp: Timestamp, +} + +impl Snapshot { + /// Gets the value associated with the given key. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{Value, Config, TransactionClient}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connecting_client = TransactionClient::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.await.unwrap(); + /// let snapshot = connected_client.snapshot().await.unwrap(); + /// let key = "TiKV".to_owned(); + /// let req = snapshot.get(key); + /// let result: Value = req.await.unwrap(); + /// # }); + /// ``` + pub async fn get(&self, _key: impl Into) -> Result { + unimplemented!() + } + + /// Gets the values associated with the given keys. + /// + /// ```rust,no_run + /// # #![feature(async_await)] + /// # use tikv_client::{KvPair, Config, TransactionClient}; + /// # use futures::prelude::*; + /// # futures::executor::block_on(async { + /// # let connecting_client = TransactionClient::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.await.unwrap(); + /// let mut txn = connected_client.begin().await.unwrap(); + /// let keys = vec!["TiKV".to_owned(), "TiDB".to_owned()]; + /// let req = txn.batch_get(keys); + /// let result: Vec = req.await.unwrap(); + /// // Finish the transaction... + /// txn.commit().await.unwrap(); + /// # }); + /// ``` + pub async fn batch_get( + &self, + _keys: impl IntoIterator>, + ) -> Result> { + unimplemented!() + } + + pub fn scan(&self, range: impl RangeBounds) -> Scanner { + drop(range); + unimplemented!() + } + + pub fn scan_reverse(&self, range: impl RangeBounds) -> Scanner { + drop(range); + unimplemented!() + } +} diff --git a/src/transaction/transaction.rs b/src/transaction/transaction.rs deleted file mode 100644 index 0e93f86..0000000 --- a/src/transaction/transaction.rs +++ /dev/null @@ -1,335 +0,0 @@ -// Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0. - -use super::{Mutation, Scanner, Timestamp}; -use crate::{Key, KvPair, Result, Value}; - -use derive_new::new; -use std::collections::BTreeMap; -use std::ops::RangeBounds; - -/// A undo-able set of actions on the dataset. -/// -/// Using a transaction you can prepare a set of actions (such as `get`, or `set`) on data at a -/// particular timestamp obtained from the placement driver. -/// -/// Once a transaction is commited, a new commit timestamp is obtained from the placement driver. -/// -/// Create a new transaction from a snapshot using `new`. -/// -/// ```rust,no_run -/// # #![feature(async_await)] -/// use tikv_client::{Config, transaction::Client}; -/// use futures::prelude::*; -/// # futures::executor::block_on(async { -/// let connect = Client::connect(Config::default()); -/// let client = connect.await.unwrap(); -/// let txn = client.begin().await.unwrap(); -/// # }); -/// ``` -#[derive(new)] -pub struct Transaction { - snapshot: Snapshot, - #[new(default)] - mutations: BTreeMap, -} - -impl Transaction { - /// Gets the value associated with the given key. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Value, Config, transaction::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let mut txn = connected_client.begin().await.unwrap(); - /// let key = "TiKV".to_owned(); - /// let req = txn.get(key); - /// let result: Value = req.await.unwrap(); - /// // Finish the transaction... - /// txn.commit().await.unwrap(); - /// # }); - /// ``` - pub async fn get(&self, key: impl Into) -> Result { - let key = key.into(); - if let Some(value) = self.get_from_mutations(&key) { - Ok(value) - } else { - self.snapshot.get(key).await - } - } - - /// Gets the values associated with the given keys. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{KvPair, Config, transaction::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let mut txn = connected_client.begin().await.unwrap(); - /// let keys = vec!["TiKV".to_owned(), "TiDB".to_owned()]; - /// let req = txn.batch_get(keys); - /// let result: Vec = req.await.unwrap(); - /// // Finish the transaction... - /// txn.commit().await.unwrap(); - /// # }); - /// ``` - pub async fn batch_get( - &self, - keys: impl IntoIterator>, - ) -> Result> { - let mut result = Vec::new(); - let mut keys_from_snapshot = Vec::new(); - let mut result_indices_from_snapshot = Vec::new(); - - // Try to fill the result vector from mutations - for key in keys { - let key = key.into(); - if let Some(value) = self.get_from_mutations(&key) { - result.push((key, value).into()); - } else { - keys_from_snapshot.push(key); - result_indices_from_snapshot.push(result.len()); - // Push a placeholder - result.push(KvPair::default()); - } - } - - // Get others from snapshot - let kv_pairs_from_snapshot = self - .snapshot - .batch_get(keys_from_snapshot.into_iter()) - .await?; - for (kv_pair, index) in kv_pairs_from_snapshot - .into_iter() - .zip(result_indices_from_snapshot) - { - result[index] = kv_pair; - } - - Ok(result) - } - - pub fn scan(&self, _range: impl RangeBounds) -> Scanner { - unimplemented!() - } - - pub fn scan_reverse(&self, _range: impl RangeBounds) -> Scanner { - unimplemented!() - } - - /// Sets the value associated with the given key. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Key, Value, Config, transaction::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let mut txn = connected_client.begin().await.unwrap(); - /// let key = "TiKV".to_owned(); - /// let val = "TiKV".to_owned(); - /// txn.set(key, val); - /// // Finish the transaction... - /// txn.commit().await.unwrap(); - /// # }); - /// ``` - pub fn set(&mut self, key: impl Into, value: impl Into) { - self.mutations - .insert(key.into(), Mutation::Put(value.into())); - } - - /// Deletes the given key. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Key, Config, transaction::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let mut txn = connected_client.begin().await.unwrap(); - /// let key = "TiKV".to_owned(); - /// txn.delete(key); - /// // Finish the transaction... - /// txn.commit().await.unwrap(); - /// # }); - /// ``` - pub fn delete(&mut self, key: impl Into) { - self.mutations.insert(key.into(), Mutation::Del); - } - - /// Locks the given keys. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Config, transaction::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connect = Client::connect(Config::default()); - /// # let connected_client = connect.await.unwrap(); - /// let mut txn = connected_client.begin().await.unwrap(); - /// txn.lock_keys(vec!["TiKV".to_owned(), "Rust".to_owned()]); - /// // ... Do some actions. - /// txn.commit().await.unwrap(); - /// # }); - /// ``` - pub fn lock_keys(&mut self, keys: impl IntoIterator>) { - for key in keys { - let key = key.into(); - // Mutated keys don't need a lock. - self.mutations.entry(key).or_insert(Mutation::Lock); - } - } - - /// Commits the actions of the transaction. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Config, transaction::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connect = Client::connect(Config::default()); - /// # let connected_client = connect.await.unwrap(); - /// let mut txn = connected_client.begin().await.unwrap(); - /// // ... Do some actions. - /// let req = txn.commit(); - /// let result: () = req.await.unwrap(); - /// # }); - /// ``` - pub async fn commit(&mut self) -> Result<()> { - self.prewrite().await?; - self.commit_primary().await?; - // FIXME: return from this method once the primary key is committed - let _ = self.commit_secondary().await; - Ok(()) - } - - /// Returns the timestamp which the transaction started at. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Config, transaction::{Client, Timestamp}}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connect = Client::connect(Config::default()); - /// # let connected_client = connect.await.unwrap(); - /// let txn = connected_client.begin().await.unwrap(); - /// // ... Do some actions. - /// let ts: Timestamp = txn.start_ts(); - /// # }); - /// ``` - pub fn start_ts(&self) -> Timestamp { - self.snapshot().timestamp - } - - /// Gets the `Snapshot` the transaction is operating on. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Config, transaction::{Client, Snapshot}}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connect = Client::connect(Config::default()); - /// # let connected_client = connect.await.unwrap(); - /// let txn = connected_client.begin().await.unwrap(); - /// // ... Do some actions. - /// let snap: &Snapshot = txn.snapshot(); - /// # }); - /// ``` - pub fn snapshot(&self) -> &Snapshot { - &self.snapshot - } - - async fn prewrite(&mut self) -> Result<()> { - // TODO: Too many clones. Consider using bytes::Byte. - let _rpc_mutations: Vec<_> = self - .mutations - .iter() - .map(|(k, v)| v.clone().with_key(k.clone())) - .collect(); - unimplemented!() - } - - async fn commit_primary(&mut self) -> Result<()> { - unimplemented!() - } - - async fn commit_secondary(&mut self) -> Result<()> { - unimplemented!() - } - - fn get_from_mutations(&self, key: &Key) -> Option { - self.mutations.get(key).and_then(Mutation::get_value) - } -} - -pub struct TxnInfo { - pub txn: u64, - pub status: u64, -} - -/// A snapshot of dataset at a particular point in time. -#[derive(new)] -pub struct Snapshot { - timestamp: Timestamp, -} - -impl Snapshot { - /// Gets the value associated with the given key. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{Value, Config, transaction::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let snapshot = connected_client.snapshot().await.unwrap(); - /// let key = "TiKV".to_owned(); - /// let req = snapshot.get(key); - /// let result: Value = req.await.unwrap(); - /// # }); - /// ``` - pub async fn get(&self, _key: impl Into) -> Result { - unimplemented!() - } - - /// Gets the values associated with the given keys. - /// - /// ```rust,no_run - /// # #![feature(async_await)] - /// # use tikv_client::{KvPair, Config, transaction::Client}; - /// # use futures::prelude::*; - /// # futures::executor::block_on(async { - /// # let connecting_client = Client::connect(Config::new(vec!["192.168.0.100", "192.168.0.101"])); - /// # let connected_client = connecting_client.await.unwrap(); - /// let mut txn = connected_client.begin().await.unwrap(); - /// let keys = vec!["TiKV".to_owned(), "TiDB".to_owned()]; - /// let req = txn.batch_get(keys); - /// let result: Vec = req.await.unwrap(); - /// // Finish the transaction... - /// txn.commit().await.unwrap(); - /// # }); - /// ``` - pub async fn batch_get( - &self, - _keys: impl IntoIterator>, - ) -> Result> { - unimplemented!() - } - - pub fn scan(&self, range: impl RangeBounds) -> Scanner { - drop(range); - unimplemented!() - } - - pub fn scan_reverse(&self, range: impl RangeBounds) -> Scanner { - drop(range); - unimplemented!() - } -} diff --git a/src/rpc/util.rs b/src/util.rs similarity index 97% rename from src/rpc/util.rs rename to src/util.rs index 6cf1064..9a0298e 100644 --- a/src/rpc/util.rs +++ b/src/util.rs @@ -18,7 +18,7 @@ macro_rules! internal_err { /// make a thread name with additional tag inheriting from current thread. macro_rules! thread_name { ($name:expr) => {{ - $crate::rpc::util::get_tag_from_thread_name() + $crate::util::get_tag_from_thread_name() .map(|tag| format!("{}::{}", $name, tag)) .unwrap_or_else(|| $name.to_owned()) }}; diff --git a/tests/integration_tests.rs b/tests/integration_tests.rs index 88a6b15..c489ea3 100644 --- a/tests/integration_tests.rs +++ b/tests/integration_tests.rs @@ -5,7 +5,7 @@ use failure::Fallible; use futures::executor::ThreadPool; use futures::prelude::*; use std::env; -use tikv_client::{transaction::*, Config, Result}; +use tikv_client::{Config, Result, TransactionClient}; #[test] fn get_timestamp() -> Fallible<()> { @@ -13,7 +13,7 @@ fn get_timestamp() -> Fallible<()> { let mut pool = ThreadPool::new()?; let config = Config::new(pd_addrs()); let fut = async { - let client = Client::connect(config).await?; + let client = TransactionClient::connect(config).await?; Result::Ok(future::join_all((0..COUNT).map(|_| client.current_timestamp())).await) }; // Calculate each version of retrieved timestamp