From 212d2c4277775b6d59b9eb82383d65f7fe7c8d7d Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Fri, 28 Dec 2018 12:58:59 +0800 Subject: [PATCH 01/20] Initial version of Raw Kv client Note: raw::Client::batch_scan is not implemented yet. Signed-off-by: Xiaoguang Sun --- Cargo.toml | 17 + examples/raw.rs | 2 +- examples/transaction.rs | 2 +- src/errors.rs | 63 +++- src/lib.rs | 102 ++++-- src/raw.rs | 505 +++++++++++++++------------- src/rpc/client.rs | 706 ++++++++++++++++++++++++++++++++++++++++ src/rpc/context.rs | 68 ++++ src/rpc/mod.rs | 22 ++ src/rpc/pd/client.rs | 218 +++++++++++++ src/rpc/pd/context.rs | 90 +++++ src/rpc/pd/leader.rs | 432 ++++++++++++++++++++++++ src/rpc/pd/mod.rs | 180 ++++++++++ src/rpc/pd/request.rs | 136 ++++++++ src/rpc/security.rs | 170 ++++++++++ src/rpc/tikv/client.rs | 700 +++++++++++++++++++++++++++++++++++++++ src/rpc/tikv/context.rs | 58 ++++ src/rpc/tikv/mod.rs | 17 + src/rpc/util.rs | 144 ++++++++ src/transaction.rs | 27 +- tests/raw.rs | 165 ++++++++++ 21 files changed, 3553 insertions(+), 271 deletions(-) create mode 100644 src/rpc/client.rs create mode 100644 src/rpc/context.rs create mode 100644 src/rpc/mod.rs create mode 100644 src/rpc/pd/client.rs create mode 100644 src/rpc/pd/context.rs create mode 100644 src/rpc/pd/leader.rs create mode 100644 src/rpc/pd/mod.rs create mode 100644 src/rpc/pd/request.rs create mode 100644 src/rpc/security.rs create mode 100644 src/rpc/tikv/client.rs create mode 100644 src/rpc/tikv/context.rs create mode 100644 src/rpc/tikv/mod.rs create mode 100644 src/rpc/util.rs create mode 100644 tests/raw.rs diff --git a/Cargo.toml b/Cargo.toml index 344150b4..078b6a0b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,3 +17,20 @@ serde = "1.0" serde_derive = "1.0" quick-error = "1.2" grpcio = { version = "0.4", features = [ "secure" ] } +protobuf = "~2.0" +tokio-core = "0.1" +tokio-timer = "0.2" +fxhash = "0.2" +lazy_static = "0.2.1" +log = "0.3.9" + +[dependencies.kvproto] +git = "https://github.com/pingcap/kvproto.git" + +[dependencies.prometheus] +version = "0.4.2" +default-features = false +features = ["push", "process"] + +[dev-dependencies] +tempdir = "0.3" diff --git a/examples/raw.rs b/examples/raw.rs index 403c1e79..2f8b4d21 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -16,7 +16,7 @@ use std::path::PathBuf; use tikv_client::*; fn main() { - let config = Config::new(vec!["127.0.0.1:3379"]).with_security( + let config = Config::new(vec!["127.0.0.1:2379"]).with_security( PathBuf::from("/path/to/ca.pem"), PathBuf::from("/path/to/client.pem"), PathBuf::from("/path/to/client-key.pem"), diff --git a/examples/transaction.rs b/examples/transaction.rs index 23a1e285..7904ca8c 100644 --- a/examples/transaction.rs +++ b/examples/transaction.rs @@ -70,7 +70,7 @@ fn dels(client: &Client, keys: impl IntoIterator) { } fn main() { - let config = Config::new(vec!["127.0.0.1:3379"]).with_security( + let config = Config::new(vec!["127.0.0.1:2379"]).with_security( PathBuf::from("/path/to/ca.pem"), PathBuf::from("/path/to/client.pem"), PathBuf::from("/path/to/client-key.pem"), diff --git a/src/errors.rs b/src/errors.rs index 66a9a20b..95a8d542 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -11,9 +11,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::{error, result}; + use grpcio; use quick_error::quick_error; -use std::{error, result}; quick_error! { #[derive(Debug)] @@ -43,33 +44,69 @@ quick_error! { description("region is not found") display("region is not found for key {:?}", key) } - RegionNotFound(id: u64) { + RegionNotFound(region_id: u64, message: Option) { description("region is not found") - display("region {:?} is not found", id) + display("region {:?} is not found. {}", region_id, message.as_ref().unwrap_or(&"".to_owned())) } - NotLeader(region_id: u64) { + NotLeader(region_id: u64, message: Option) { description("peer is not leader") - display("peer is not leader for region {:?}.", region_id) + display("peer is not leader for region {}. {}", region_id, message.as_ref().unwrap_or(&"".to_owned())) } - StoreNotMatch { + StoreNotMatch(request_store_id: u64, actual_store_id: u64, message: String) { description("store not match") - display("store not match") + display("requesting store '{}' when actual store is '{}'. {}", request_store_id, actual_store_id, message) } KeyNotInRegion(key: Vec, region_id: u64, start_key: Vec, end_key: Vec) { description("region is not found") display("key {:?} is not in region {:?}: [{:?}, {:?})", key, region_id, start_key, end_key) } - StaleEpoch { + StaleEpoch(message: Option) { description("stale epoch") - display("stale epoch") + display("{}", message.as_ref().unwrap_or(&"".to_owned())) } - ServerIsBusy(reason: String) { + StaleCommand(message: String) { + description("stale command") + display("{}", message) + } + ServerIsBusy(reason: String, backoff: u64) { description("server is busy") - display("server is busy: {:?}", reason) + display("server is busy: {:?}. Backoff {} ms", reason, backoff) } - RaftEntryTooLarge(region_id: u64, entry_size: u64) { + RaftEntryTooLarge(region_id: u64, entry_size: u64, message: String) { description("raft entry too large") - display("{:?} bytes raft entry of region {:?} is too large", entry_size, region_id) + display("{:?} bytes raft entry of region {:?} is too large. {}", entry_size, region_id, message) + } + KeyError(message: String) { + description("key error") + display("{}", message) + } + KVError(message: String) { + description("kv error") + display("{}", message) + } + InternalError(message: String) { + description("internal error") + display("{}", message) + } + InvalidKeyRange { + description("invalid key range") + display("Only left closed intervals are supported") + } + Unimplemented { + description("unimplemented feature") + display("Unimplemented feature") + } + EmptyValue { + description("can not set empty value") + display("Can not set empty value") + } + NoSuchKey { + description("key does not exist") + display("Key doest not exist") + } + InvalidOverlappingRanges { + description("ranges can not be overlapping") + display("Ranges can not be overlapping") } } } diff --git a/src/lib.rs b/src/lib.rs index 3589dc89..48cd651f 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -11,17 +11,20 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![recursion_limit = "128"] +#![type_length_limit = "2097152"] +use std::{convert::AsRef, ops::Deref, path::PathBuf, time::Duration}; + +use futures::Future; use serde_derive::*; -use std::ops::Deref; -use std::path::PathBuf; + +pub use crate::errors::{Error, Result}; pub mod errors; pub mod raw; +mod rpc; pub mod transaction; -pub use crate::errors::Error; -pub use crate::errors::Result; - #[derive(Default, Clone, Eq, PartialEq, Ord, PartialOrd, Hash, Debug)] pub struct Key(Vec); #[derive(Default, Clone, Eq, PartialEq, Hash, Debug)] @@ -29,9 +32,15 @@ pub struct Value(Vec); #[derive(Default, Clone, Eq, PartialEq, Debug)] pub struct KvPair(Key, Value); -impl Into for Vec { - fn into(self) -> Key { - Key(self) +impl From> for Key { + fn from(vec: Vec) -> Key { + Key(vec) + } +} + +impl<'a> From<&'a [u8]> for Key { + fn from(s: &'a [u8]) -> Key { + Key(s.to_vec()) } } @@ -41,28 +50,64 @@ impl AsRef for Key { } } +impl AsRef<[u8]> for Key { + fn as_ref(&self) -> &[u8] { + &self.0 + } +} + impl Deref for Key { - type Target = Vec; + type Target = [u8]; - fn deref(&self) -> &Self::Target { + fn deref<'a>(&'a self) -> &'a Self::Target { &self.0 } } -impl Into for Vec { - fn into(self) -> Value { - Value(self) +impl Key { + fn into_inner(self) -> Vec { + self.0 + } +} + +impl From> for Value { + fn from(vec: Vec) -> Value { + Value(vec) + } +} + +impl<'a> From<&'a [u8]> for Value { + fn from(s: &'a [u8]) -> Value { + Value(s.to_vec()) + } +} + +impl AsRef for Value { + fn as_ref(&self) -> &Self { + self + } +} + +impl AsRef<[u8]> for Value { + fn as_ref(&self) -> &[u8] { + &self.0 } } impl Deref for Value { - type Target = Vec; + type Target = [u8]; - fn deref(&self) -> &Self::Target { + fn deref<'a>(&'a self) -> &'a Self::Target { &self.0 } } +impl Value { + fn into_inner(self) -> Vec { + self.0 + } +} + impl KvPair { pub fn new(key: Key, value: Value) -> Self { KvPair(key, value) @@ -75,11 +120,15 @@ impl KvPair { pub fn value(&self) -> &Value { &self.1 } + + pub fn into_inner(self) -> (Key, Value) { + (self.0, self.1) + } } -impl Into for (Key, Value) { - fn into(self) -> KvPair { - KvPair(self.0, self.1) +impl From<(Key, Value)> for KvPair { + fn from(pair: (Key, Value)) -> KvPair { + KvPair(pair.0, pair.1) } } @@ -87,10 +136,11 @@ impl Into for (Key, Value) { #[serde(default)] #[serde(rename_all = "kebab-case")] pub struct Config { - pub pd_endpoints: Vec, - pub ca_path: Option, - pub cert_path: Option, - pub key_path: Option, + pd_endpoints: Vec, + ca_path: Option, + cert_path: Option, + key_path: Option, + timeout: Duration, } impl Config { @@ -100,6 +150,7 @@ impl Config { ca_path: None, cert_path: None, key_path: None, + timeout: Duration::from_secs(2), } } @@ -114,4 +165,11 @@ impl Config { self.key_path = Some(key_path.into()); self } + + pub fn timeout(mut self, timeout: Duration) -> Self { + self.timeout = timeout; + self + } } + +pub type KvFuture = Box>; diff --git a/src/raw.rs b/src/raw.rs index 5e97d17b..d6a060fb 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -11,9 +11,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::{Config, Error, Key, KvPair, Value}; -use futures::{Future, Poll}; -use std::ops::RangeBounds; +use std::{ + ops::{Bound, Deref, DerefMut, RangeBounds}, + sync::Arc, +}; + +use futures::{future, Async, Future, Poll}; + +use crate::{rpc::RpcClient, Config, Error, Key, KvFuture, KvPair, Result, Value}; #[derive(Default, Clone, Eq, PartialEq, Ord, PartialOrd, Hash, Debug)] pub struct ColumnFamily(String); @@ -27,17 +32,35 @@ where } } -pub struct Get<'a> { - client: &'a Client, - key: Key, +impl ColumnFamily { + pub fn into_inner(self) -> String { + self.0 + } +} + +pub trait RequestInner: Sized { + type Resp; + + fn execute(self, client: Arc, cf: Option) -> KvFuture; +} + +pub struct Request +where + Inner: RequestInner, +{ + inner: Option<(Arc, Inner)>, + future: Option>, cf: Option, } -impl<'a> Get<'a> { - fn new(client: &'a Client, key: Key) -> Self { - Get { - client, - key, +impl Request +where + Inner: RequestInner, +{ + fn new(client: Arc, inner: Inner) -> Self { + Request { + inner: Some((client, inner)), + future: None, cf: None, } } @@ -48,321 +71,320 @@ impl<'a> Get<'a> { } } -impl<'a> Future for Get<'a> { - type Item = Value; +impl Deref for Request +where + Inner: RequestInner, +{ + type Target = Inner; + + fn deref(&self) -> &Self::Target { + &self.inner.as_ref().unwrap().1 + } +} + +impl DerefMut for Request +where + Inner: RequestInner, +{ + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.inner.as_mut().unwrap().1 + } +} + +impl Future for Request +where + Inner: RequestInner, +{ + type Item = Inner::Resp; type Error = Error; fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.key; - let _ = &self.cf; - unimplemented!() + loop { + if self.inner.is_some() { + let (client, inner) = self.inner.take().unwrap(); + self.future = Some(inner.execute(client, self.cf.take())); + } else { + break self.future.as_mut().map(|x| x.poll()).unwrap(); + } + } } } -pub struct BatchGet<'a> { - client: &'a Client, - keys: Vec, - cf: Option, +pub struct GetInner { + key: Key, } -impl<'a> BatchGet<'a> { - fn new(client: &'a Client, keys: Vec) -> Self { - BatchGet { - client, - keys, - cf: None, - } +impl GetInner { + fn new(key: Key) -> Self { + GetInner { key } } +} - pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); - self +impl RequestInner for GetInner { + type Resp = Value; + + fn execute(self, client: Arc, cf: Option) -> KvFuture { + Box::new(client.raw_get(self.key, cf)) } } -impl<'a> Future for BatchGet<'a> { - type Item = Vec; - type Error = (); +pub type Get = Request; - fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.keys; - let _ = &self.cf; - unimplemented!() +pub struct BatchGetInner { + keys: Vec, +} + +impl RequestInner for BatchGetInner { + type Resp = Vec; + + fn execute(self, client: Arc, cf: Option) -> KvFuture { + Box::new(client.raw_batch_get(self.keys, cf)) } } -pub struct Put<'a> { - client: &'a Client, +impl BatchGetInner { + fn new(keys: Vec) -> Self { + BatchGetInner { keys } + } +} + +pub type BatchGet = Request; + +pub struct PutInner { key: Key, value: Value, - cf: Option, } -impl<'a> Put<'a> { - fn new(client: &'a Client, key: Key, value: Value) -> Self { - Put { - client, - key, - value, - cf: None, - } - } - - pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); - self +impl PutInner { + fn new(key: Key, value: Value) -> Self { + PutInner { key, value } } } -impl<'a> Future for Put<'a> { - type Item = (); - type Error = (); +impl RequestInner for PutInner { + type Resp = (); - fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.key; - let _ = &self.value; - let _ = &self.cf; - unimplemented!() + fn execute(self, client: Arc, cf: Option) -> KvFuture { + let (key, value) = (self.key, self.value); + Box::new(client.raw_put(key, value, cf)) } } -pub struct BatchPut<'a> { - client: &'a Client, +pub type Put = Request; + +pub struct BatchPutInner { pairs: Vec, - cf: Option, } -impl<'a> BatchPut<'a> { - fn new(client: &'a Client, pairs: Vec) -> Self { - BatchPut { - client, - pairs, - cf: None, - } - } - - pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); - self +impl BatchPutInner { + fn new(pairs: Vec) -> Self { + BatchPutInner { pairs } } } -impl<'a> Future for BatchPut<'a> { - type Item = (); - type Error = (); +impl RequestInner for BatchPutInner { + type Resp = (); - fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.pairs; - let _ = &self.cf; - unimplemented!() + fn execute(self, client: Arc, cf: Option) -> KvFuture { + Box::new(client.raw_batch_put(self.pairs, cf)) } } -pub struct Delete<'a> { - client: &'a Client, +pub type BatchPut = Request; + +pub struct DeleteInner { key: Key, - cf: Option, } -impl<'a> Delete<'a> { - fn new(client: &'a Client, key: Key) -> Self { - Delete { - client, - key, - cf: None, - } - } - - pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); - self +impl DeleteInner { + fn new(key: Key) -> Self { + DeleteInner { key } } } -impl<'a> Future for Delete<'a> { - type Item = (); - type Error = (); +impl RequestInner for DeleteInner { + type Resp = (); - fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.key; - let _ = &self.cf; - unimplemented!() + fn execute(self, client: Arc, cf: Option) -> KvFuture { + Box::new(client.raw_delete(self.key, cf)) } } -pub struct BatchDelete<'a> { - client: &'a Client, +pub type Delete = Request; + +pub struct BatchDeleteInner { keys: Vec, - cf: Option, } -impl<'a> BatchDelete<'a> { - fn new(client: &'a Client, keys: Vec) -> Self { - BatchDelete { - client, - keys, - cf: None, - } - } - - pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); - self +impl BatchDeleteInner { + fn new(keys: Vec) -> Self { + BatchDeleteInner { keys } } } -impl<'a> Future for BatchDelete<'a> { - type Item = (); - type Error = (); +impl RequestInner for BatchDeleteInner { + type Resp = (); - fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.keys; - let _ = &self.cf; - unimplemented!() + fn execute(self, client: Arc, cf: Option) -> KvFuture { + Box::new(client.raw_batch_delete(self.keys, cf)) } } -pub struct Scan<'a> { - client: &'a Client, - range: (Key, Key), +pub type BatchDelete = Request; + +pub struct ScanInner { + range: Result<(Key, Option)>, limit: u32, key_only: bool, - cf: Option, - reverse: bool, } -impl<'a> Scan<'a> { - fn new(client: &'a Client, range: (Key, Key), limit: u32) -> Self { - Scan { - client, +impl ScanInner { + fn new(range: Result<(Key, Option)>, limit: u32) -> Self { + ScanInner { range, limit, key_only: false, - cf: None, - reverse: false, } } +} - pub fn key_only(mut self) -> Self { - self.key_only = true; - self +impl RequestInner for ScanInner { + type Resp = Vec; + + fn execute(self, client: Arc, cf: Option) -> KvFuture { + match self.range { + Ok(range) => Box::new(client.raw_scan(range, self.limit, self.key_only, cf)), + Err(e) => Box::new(future::err(e)), + } + } +} + +pub struct Scan { + request: Request, +} + +impl Scan { + fn new(client: Arc, inner: ScanInner) -> Self { + Scan { + request: Request::new(client, inner), + } } pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); + self.request = self.request.cf(cf); self } - pub fn reverse(mut self) -> Self { - self.reverse = true; + pub fn key_only(mut self) -> Self { + self.request.inner = self.request.inner.map(|mut x| { + x.1.key_only = true; + x + }); self } } -impl<'a> Future for Scan<'a> { +impl Future for Scan { type Item = Vec; - type Error = (); + type Error = Error; fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.range; - let _ = &self.limit; - let _ = &self.key_only; - let _ = &self.cf; - unimplemented!() + self.request.poll() } } -pub struct BatchScan<'a> { - client: &'a Client, - ranges: Vec<(Key, Key)>, +pub struct BatchScanInner { + ranges: Vec)>>, each_limit: u32, key_only: bool, - cf: Option, - reverse: bool, } -impl<'a> BatchScan<'a> { - fn new(client: &'a Client, ranges: Vec<(Key, Key)>, each_limit: u32) -> Self { - BatchScan { - client, +impl BatchScanInner { + fn new(ranges: Vec)>>, each_limit: u32) -> Self { + BatchScanInner { ranges, each_limit, key_only: false, - cf: None, - reverse: false, } } +} - pub fn key_only(mut self) -> Self { - self.key_only = true; - self +impl RequestInner for BatchScanInner { + type Resp = Vec; + + fn execute(self, client: Arc, cf: Option) -> KvFuture { + let (mut errors, ranges): (Vec<_>, Vec<_>) = + self.ranges.into_iter().partition(Result::is_err); + if !errors.is_empty() { + Box::new(future::err(errors.pop().unwrap().unwrap_err())) + } else { + Box::new(client.raw_batch_scan( + ranges.into_iter().map(Result::unwrap).collect(), + self.each_limit, + self.key_only, + cf, + )) + } + } +} + +pub struct BatchScan { + request: Request, +} + +impl BatchScan { + fn new(client: Arc, inner: BatchScanInner) -> Self { + BatchScan { + request: Request::new(client, inner), + } } pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); + self.request = self.request.cf(cf); self } - pub fn reverse(mut self) -> Self { - self.reverse = true; + pub fn key_only(mut self) -> Self { + self.request.inner = self.request.inner.map(|mut x| { + x.1.key_only = true; + x + }); self } } -impl<'a> Future for BatchScan<'a> { +impl Future for BatchScan { type Item = Vec; - type Error = (); + type Error = Error; fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.ranges; - let _ = &self.each_limit; - let _ = &self.key_only; - let _ = &self.cf; - unimplemented!() + self.request.poll() } } -pub struct DeleteRange<'a> { - client: &'a Client, - range: (Key, Key), - cf: Option, +pub struct DeleteRangeInner { + range: Result<(Key, Option)>, } -impl<'a> DeleteRange<'a> { - fn new(client: &'a Client, range: (Key, Key)) -> Self { - DeleteRange { - client, - range, - cf: None, - } - } - - pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); - self +impl DeleteRangeInner { + fn new(range: Result<(Key, Option)>) -> Self { + DeleteRangeInner { range } } } -impl<'a> Future for DeleteRange<'a> { - type Item = (); - type Error = (); +impl RequestInner for DeleteRangeInner { + type Resp = (); - fn poll(&mut self) -> Poll { - let _ = &self.client; - let _ = &self.range; - let _ = &self.cf; - unimplemented!() + fn execute(self, client: Arc, cf: Option) -> KvFuture { + match self.range { + Ok(range) => Box::new(client.raw_delete_range(range, cf)), + Err(e) => Box::new(future::err(e)), + } } } +pub type DeleteRange = Request; + pub struct Connect { config: Config, } @@ -378,12 +400,15 @@ impl Future for Connect { type Error = Error; fn poll(&mut self) -> Poll { - let _config = &self.config; - unimplemented!() + let config = &self.config; + let rpc = Arc::new(RpcClient::connect(config)?); + Ok(Async::Ready(Client { rpc })) } } -pub struct Client; +pub struct Client { + rpc: Arc, +} impl Client { #![allow(clippy::new_ret_no_self)] @@ -391,32 +416,43 @@ impl Client { Connect::new(config.clone()) } + #[inline] + fn rpc(&self) -> Arc { + Arc::clone(&self.rpc) + } + pub fn get(&self, key: impl AsRef) -> Get { - Get::new(self, key.as_ref().clone()) + Get::new(self.rpc(), GetInner::new(key.as_ref().clone())) } pub fn batch_get(&self, keys: impl AsRef<[Key]>) -> BatchGet { - BatchGet::new(self, keys.as_ref().to_vec()) + BatchGet::new(self.rpc(), BatchGetInner::new(keys.as_ref().to_vec())) } pub fn put(&self, key: impl Into, value: impl Into) -> Put { - Put::new(self, key.into(), value.into()) + Put::new(self.rpc(), PutInner::new(key.into(), value.into())) } pub fn batch_put(&self, pairs: impl IntoIterator>) -> BatchPut { - BatchPut::new(self, pairs.into_iter().map(Into::into).collect()) + BatchPut::new( + self.rpc(), + BatchPutInner::new(pairs.into_iter().map(Into::into).collect()), + ) } pub fn delete(&self, key: impl AsRef) -> Delete { - Delete::new(self, key.as_ref().clone()) + Delete::new(self.rpc(), DeleteInner::new(key.as_ref().clone())) } pub fn batch_delete(&self, keys: impl AsRef<[Key]>) -> BatchDelete { - BatchDelete::new(self, keys.as_ref().to_vec()) + BatchDelete::new(self.rpc(), BatchDeleteInner::new(keys.as_ref().to_vec())) } pub fn scan(&self, range: impl RangeBounds, limit: u32) -> Scan { - Scan::new(self, Self::extract_range(&range), limit) + Scan::new( + self.rpc(), + ScanInner::new(Self::range_bounds(&range), limit), + ) } pub fn batch_scan(&self, ranges: Ranges, each_limit: u32) -> BatchScan @@ -425,17 +461,40 @@ impl Client { Bounds: RangeBounds, { BatchScan::new( - self, - ranges.as_ref().iter().map(Self::extract_range).collect(), - each_limit, + self.rpc(), + BatchScanInner::new( + ranges.as_ref().iter().map(Self::range_bounds).collect(), + each_limit, + ), ) } pub fn delete_range(&self, range: impl RangeBounds) -> DeleteRange { - DeleteRange::new(self, Self::extract_range(&range)) + DeleteRange::new( + self.rpc(), + DeleteRangeInner::new(Self::range_bounds(&range)), + ) } - fn extract_range(_range: &impl RangeBounds) -> (Key, Key) { - unimplemented!() + fn bound(bound: Bound<&Key>) -> Option { + match bound { + Bound::Included(k) => Some(k.clone()), + Bound::Excluded(k) => Some(k.clone()), + Bound::Unbounded => None, + } + } + + fn range_bounds(range: &impl RangeBounds) -> Result<(Key, Option)> { + if let Bound::Included(_) = range.end_bound() { + return Err(Error::InvalidKeyRange); + } + if let Bound::Excluded(_) = range.start_bound() { + return Err(Error::InvalidKeyRange); + } + + Ok(( + Self::bound(range.start_bound()).unwrap_or_else(Key::default), + Self::bound(range.end_bound()), + )) } } diff --git a/src/rpc/client.rs b/src/rpc/client.rs new file mode 100644 index 00000000..3680a0dc --- /dev/null +++ b/src/rpc/client.rs @@ -0,0 +1,706 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + collections::HashMap, + fmt, + ops::Deref, + sync::{Arc, RwLock}, + time::Duration, +}; + +use futures::{ + future::{self, loop_fn, Either, Loop}, + Future, +}; +use grpcio::{EnvBuilder, Environment}; +use kvproto::kvrpcpb; +use log::*; + +use crate::{ + raw::ColumnFamily, + rpc::{ + pd::{PdClient, PdTimestamp, Region, RegionId, RegionVerId, Store, StoreId}, + security::SecurityManager, + tikv::KvClient, + util::HandyRwLock, + }, + Config, Error, Key, KvPair, Result, Value, +}; + +const CQ_COUNT: usize = 1; +const CLIENT_PREFIX: &str = "tikv-client"; + +struct RpcClientInner { + pd: Arc, + tikv: Arc>>>, + env: Arc, + security_mgr: Arc, + timeout: Duration, +} + +impl RpcClientInner { + fn connect(config: &Config) -> Result { + let env = Arc::new( + EnvBuilder::new() + .cq_count(CQ_COUNT) + .name_prefix(thd_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(PdClient::connect( + Arc::clone(&env), + &config.pd_endpoints, + Arc::clone(&security_mgr), + config.timeout, + )?); + let tikv = Default::default(); + Ok(RpcClientInner { + pd, + tikv, + env, + security_mgr, + timeout: config.timeout, + }) + } + + fn get_all_stores(&self) -> impl Future, Error = Error> { + self.pd.get_all_stores() + } + + fn get_store_by_id(&self, id: StoreId) -> impl Future { + self.pd.get_store(id) + } + + fn get_region(&self, key: &[u8]) -> impl Future { + self.pd.get_region(key) + } + + fn get_region_by_id(&self, id: RegionId) -> impl Future { + self.pd.get_region_by_id(id) + } + + fn get_ts(&self) -> impl Future { + self.pd.get_ts() + } + + fn load_store(&self, id: StoreId) -> impl Future { + info!("reload info for store {}", id); + self.pd.get_store(id).map(Into::into) + } + + fn load_region(&self, key: &Key) -> impl Future { + self.pd.get_region(key.as_ref()) + } + + fn load_region_by_id(&self, id: RegionId) -> impl Future { + self.pd.get_region_by_id(id) + } + + fn locate_key(&self, key: &Key) -> impl Future { + self.load_region(key).map(KeyLocation::new) + } + + fn kv_client(&self, context: RegionContext) -> Result<(RegionContext, Arc)> { + if let Some(conn) = self.tikv.rl().get(context.address()) { + return Ok((context, Arc::clone(conn))); + }; + info!("connect to tikv endpoint: {:?}", context.address()); + let tikv = Arc::clone(&self.tikv); + KvClient::connect( + Arc::clone(&self.env), + context.address(), + &self.security_mgr, + self.timeout, + ) + .map(Arc::new) + .map(|c| { + tikv.wl() + .insert(context.address().to_owned(), Arc::clone(&c)); + (context, c) + }) + } +} + +pub struct RpcClient { + inner: Arc, +} + +impl RpcClient { + pub fn connect(config: &Config) -> Result { + Ok(RpcClient { + inner: Arc::new(RpcClientInner::connect(config)?), + }) + } + + fn group_tasks_by_region( + &self, + tasks: Vec, + ) -> impl Future, Error = Error> + where + Task: GroupingTask, + { + let result: Option> = None; + let inner = self.inner(); + loop_fn((0, tasks, result), move |(mut index, tasks, mut result)| { + if index == tasks.len() { + Either::A(future::ok(Loop::Break(result))) + } else { + let inner = Arc::clone(&inner); + Either::B(inner.locate_key(tasks[index].key()).map(move |location| { + while let Some(item) = tasks.get(index) { + if !location.contains(item.key()) { + break; + } + let ver_id = location.ver_id(); + let item = item.clone(); + if let Some(ref mut grouped) = result { + grouped.add(ver_id, item); + } else { + result = Some(GroupedTasks::new(ver_id, item)); + } + index += 1; + } + if index == tasks.len() { + Loop::Break(result) + } else { + Loop::Continue((index, tasks, result)) + } + })) + } + }) + .map(|r| r.unwrap_or_default()) + } + + fn region_context( + inner: Arc, + key: &Key, + ) -> impl Future), Error = Error> { + let inner2 = Arc::clone(&inner); + inner + .locate_key(key) + .and_then(move |location| { + let peer = location.peer().expect("leader must exist"); + let store_id = peer.get_store_id(); + inner.load_store(store_id).map(|store| RegionContext { + region: location.into_inner(), + store, + }) + }) + .and_then(move |region| inner2.kv_client(region)) + } + + fn region_context_by_id( + inner: Arc, + id: RegionId, + ) -> impl Future), Error = Error> { + let inner2 = Arc::clone(&inner); + inner + .load_region_by_id(id) + .and_then(move |region| { + let peer = region.peer().expect("leader must exist"); + let store_id = peer.get_store_id(); + inner + .load_store(store_id) + .map(|store| RegionContext { region, store }) + }) + .and_then(move |region| inner2.kv_client(region)) + } + + fn raw( + inner: Arc, + key: &Key, + cf: Option, + ) -> impl Future { + Self::region_context(inner, key) + .map(move |(region, client)| RawContext::new(region, client, cf)) + } + + fn txn(inner: Arc, key: &Key) -> impl Future { + Self::region_context(inner, key).map(move |(region, _client)| TxnContext::new(region)) + } + + #[inline] + fn inner(&self) -> Arc { + Arc::clone(&self.inner) + } + + pub fn raw_get( + &self, + key: Key, + cf: Option, + ) -> impl Future { + Self::raw(self.inner(), &key, cf) + .and_then(move |context| context.client().raw_get(context, key)) + .and_then(move |value| { + if value.is_empty() { + Err(Error::NoSuchKey) + } else { + Ok(value) + } + }) + } + + pub fn raw_batch_get( + &self, + keys: Vec, + cf: Option, + ) -> impl Future, Error = Error> { + let inner = self.inner(); + self.group_tasks_by_region(keys) + .and_then(move |gk| { + let gk = gk.into_inner(); + let mut tasks = Vec::with_capacity(gk.len()); + for (region, keys) in gk.into_iter() { + let inner = Arc::clone(&inner); + let cf = cf.clone(); + let task = Self::region_context_by_id(inner, region.id) + .map(move |(region, client)| RawContext::new(region, client, cf)) + .and_then(move |context| { + context.client().raw_batch_get(context, keys.into_iter()) + }); + tasks.push(task); + } + future::join_all(tasks) + }) + .map(|r| r.into_iter().flat_map(|a| a.into_iter()).collect()) + } + + pub fn raw_put( + &self, + key: Key, + value: Value, + cf: Option, + ) -> impl Future { + if value.is_empty() { + Either::A(future::err(Error::EmptyValue)) + } else { + Either::B( + Self::raw(self.inner(), &key, cf) + .and_then(move |context| context.client().raw_put(context, key, value)), + ) + } + } + + pub fn raw_batch_put( + &self, + pairs: Vec, + cf: Option, + ) -> impl Future { + if pairs.iter().any(|p| p.value().is_empty()) { + Either::A(future::err(Error::EmptyValue)) + } else { + let inner = self.inner(); + Either::B( + self.group_tasks_by_region(pairs) + .and_then(move |gk| { + let gk = gk.into_inner(); + let mut tasks = Vec::with_capacity(gk.len()); + for (region, pairs) in gk.into_iter() { + let inner = Arc::clone(&inner); + let cf = cf.clone(); + let task = Self::region_context_by_id(inner, region.id) + .map(move |(region, client)| RawContext::new(region, client, cf)) + .and_then(move |context| { + context.client().raw_batch_put(context, pairs) + }); + tasks.push(task); + } + future::join_all(tasks) + }) + .map(|_| ()), + ) + } + } + + pub fn raw_delete( + &self, + key: Key, + cf: Option, + ) -> impl Future { + Self::raw(self.inner(), &key, cf) + .and_then(move |context| context.client().raw_delete(context, key)) + } + + pub fn raw_batch_delete( + &self, + keys: Vec, + cf: Option, + ) -> impl Future { + let inner = self.inner(); + self.group_tasks_by_region(keys) + .and_then(move |gk| { + let gk = gk.into_inner(); + let mut tasks = Vec::with_capacity(gk.len()); + for (region, keys) in gk.into_iter() { + let inner = Arc::clone(&inner); + let cf = cf.clone(); + let task = Self::region_context_by_id(inner, region.id) + .map(move |(region, client)| RawContext::new(region, client, cf)) + .and_then(move |context| context.client().raw_batch_delete(context, keys)); + tasks.push(task); + } + future::join_all(tasks) + }) + .map(|_| ()) + } + + pub fn raw_scan( + &self, + range: (Key, Option), + limit: u32, + key_only: bool, + cf: Option, + ) -> impl Future, Error = Error> { + struct State { + limit: u32, + key_only: bool, + cf: Option, + }; + let scan: ScanRegionsContext, State> = ScanRegionsContext::new( + range, + State { + limit, + key_only, + cf, + }, + ); + let inner = Arc::clone(&self.inner); + loop_fn((inner, scan), |(inner, scan)| { + inner + .locate_key(scan.start_key()) + .and_then(move |location| { + let region = location.into_inner(); + let cf = scan.cf.clone(); + Self::region_context_by_id(Arc::clone(&inner), region.id) + .map(move |(region, client)| { + (scan, region.range(), RawContext::new(region, client, cf)) + }) + .and_then(move |(mut scan, region_range, context)| { + let (start_key, end_key) = scan.range(); + context + .client() + .raw_scan(context, start_key, end_key, scan.limit, scan.key_only) + .map(move |pairs| (scan, region_range, pairs)) + }) + .map(move |(mut scan, region_range, mut pairs)| { + let limit = scan.limit; + scan.result_mut().append(&mut pairs); + if scan.result().len() as u32 >= limit { + Loop::Break(scan.into_inner()) + } else { + match scan.next(region_range) { + ScanRegionsStatus::Continue => Loop::Continue((inner, scan)), + ScanRegionsStatus::Break => Loop::Break(scan.into_inner()), + } + } + }) + }) + }) + } + + pub fn raw_batch_scan( + &self, + ranges: Vec<(Key, Option)>, + _each_limit: u32, + _key_only: bool, + cf: Option, + ) -> impl Future, Error = Error> { + drop(ranges); + drop(cf); + future::err(Error::Unimplemented) + } + + pub fn raw_delete_range( + &self, + range: (Key, Option), + cf: Option, + ) -> impl Future { + let scan: ScanRegionsContext<(), Option> = ScanRegionsContext::new(range, cf); + let inner = Arc::clone(&self.inner); + loop_fn((inner, scan), |(inner, scan)| { + inner + .locate_key(scan.start_key()) + .and_then(move |location| { + let region = location.into_inner(); + let cf = scan.clone(); + Self::region_context_by_id(Arc::clone(&inner), region.id) + .map(move |(region, client)| { + (scan, region.range(), RawContext::new(region, client, cf)) + }) + .and_then(move |(mut scan, region_range, context)| { + let (start_key, end_key) = scan.range(); + let start_key = start_key.expect("start key must be specified"); + let end_key = end_key.expect("end key must be specified"); + context + .client() + .raw_delete_range(context, start_key, end_key) + .map(move |_| (scan, region_range)) + }) + .map( + move |(mut scan, region_range)| match scan.next(region_range) { + ScanRegionsStatus::Continue => Loop::Continue((inner, scan)), + ScanRegionsStatus::Break => Loop::Break(()), + }, + ) + }) + }) + } +} + +impl fmt::Debug for RpcClient { + fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { + fmt.debug_struct("tikv-client") + .field("pd", &self.inner.pd) + .finish() + } +} + +pub struct RegionContext { + region: Region, + store: Store, +} + +impl RegionContext { + fn address(&self) -> &str { + self.store.get_address() + } + + fn start_key(&self) -> Key { + self.region.start_key().into() + } + + fn end_key(&self) -> Key { + self.region.end_key().into() + } + + fn range(&self) -> (Key, Key) { + (self.start_key(), self.end_key()) + } +} + +impl From for kvrpcpb::Context { + fn from(mut ctx: RegionContext) -> kvrpcpb::Context { + let mut kvctx = kvrpcpb::Context::new(); + kvctx.set_region_id(ctx.region.id); + kvctx.set_region_epoch(ctx.region.take_region_epoch()); + kvctx.set_peer(ctx.region.peer().expect("leader must exist").into_inner()); + kvctx + } +} + +pub struct RawContext { + region: RegionContext, + client: Arc, + cf: Option, +} + +impl RawContext { + fn new(region: RegionContext, client: Arc, cf: Option) -> Self { + RawContext { region, client, cf } + } + + fn client(&self) -> Arc { + Arc::clone(&self.client) + } + + pub fn into_inner(self) -> (RegionContext, Option) { + (self.region, self.cf) + } +} + +pub struct TxnContext { + region: RegionContext, +} + +impl TxnContext { + fn new(region: RegionContext) -> Self { + TxnContext { region } + } + + pub fn into_inner(self) -> RegionContext { + self.region + } +} + +struct KeyLocation(Region); + +impl KeyLocation { + fn new(region: Region) -> Self { + KeyLocation(region) + } + + fn contains(&self, key: &Key) -> bool { + let start = self.start_key(); + let end = self.end_key(); + let key: &[u8] = key.as_ref(); + start <= key && (end < key || end.is_empty()) + } + + fn into_inner(self) -> Region { + self.0 + } +} + +impl Deref for KeyLocation { + type Target = Region; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +trait GroupingTask: Clone + Default + Sized { + fn key(&self) -> &Key; +} + +#[derive(Default)] +struct GroupedTasks(HashMap>, RegionVerId); + +impl GroupedTasks +where + Task: GroupingTask, +{ + fn new(ver_id: RegionVerId, task: Task) -> Self { + let mut map = HashMap::with_capacity(1); + map.insert(ver_id.clone(), vec![task]); + GroupedTasks(map, ver_id) + } + + fn add(&mut self, ver_id: RegionVerId, task: Task) { + self.0 + .entry(ver_id) + .or_insert_with(|| Vec::with_capacity(1)) + .push(task) + } + + fn into_inner(self) -> HashMap> { + self.0 + } +} + +impl Deref for GroupedTasks +where + Task: GroupingTask, +{ + type Target = HashMap>; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +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 + } +} + +enum ScanRegionsStatus { + Continue, + Break, +} + +struct ScanRegionsContext +where + Res: Default, + State: Sized, +{ + start_key: Option, + end_key: Option, + result: Res, + state: State, +} + +impl ScanRegionsContext +where + Res: Default, + State: Sized, +{ + fn new(range: (Key, Option), state: State) -> Self { + ScanRegionsContext { + start_key: Some(range.0), + end_key: range.1, + result: Res::default(), + state, + } + } + + fn range(&mut self) -> (Option, Option) { + (self.start_key.take(), self.end_key.clone()) + } + + fn start_key(&self) -> &Key { + self.start_key.as_ref().unwrap() + } + + fn end_key(&self) -> Option<&Key> { + self.end_key.as_ref() + } + + fn next(&mut self, region_range: (Key, Key)) -> ScanRegionsStatus { + { + let region_end = ®ion_range.1; + if self.end_key().map(|x| x < region_end).unwrap_or(false) || region_end.is_empty() { + return ScanRegionsStatus::Break; + } + } + self.start_key = Some(region_range.1); + ScanRegionsStatus::Continue + } + + fn into_inner(self) -> Res { + self.result + } + + fn result_mut(&mut self) -> &mut Res { + &mut self.result + } + + fn result(&self) -> &Res { + &self.result + } +} + +impl Deref for ScanRegionsContext +where + Res: Default, + State: Sized, +{ + type Target = State; + + fn deref(&self) -> &Self::Target { + &self.state + } +} diff --git a/src/rpc/context.rs b/src/rpc/context.rs new file mode 100644 index 00000000..fccd3e5e --- /dev/null +++ b/src/rpc/context.rs @@ -0,0 +1,68 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +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, + executor: Option, +} + +impl RequestContext { + pub fn new( + cmd: &'static str, + duration: &'static HistogramVec, + counter: &'static IntCounterVec, + failed_duration: &'static HistogramVec, + failed_counter: &'static IntCounterVec, + executor: Executor, + ) -> Self { + counter.with_label_values(&[cmd]).inc(); + RequestContext { + start: Instant::now(), + cmd, + duration, + failed_duration, + failed_counter, + executor: Some(executor), + } + } + + pub fn executor(&mut self) -> Executor { + self.executor + .take() + .expect("executor can only be take once") + } + + 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 new file mode 100644 index 00000000..f9963950 --- /dev/null +++ b/src/rpc/mod.rs @@ -0,0 +1,22 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +#[macro_use] +mod util; +mod client; +mod context; +mod pd; +mod security; +mod tikv; + +pub(crate) use crate::rpc::client::RpcClient; diff --git a/src/rpc/pd/client.rs b/src/rpc/pd/client.rs new file mode 100644 index 00000000..fd99c6b9 --- /dev/null +++ b/src/rpc/pd/client.rs @@ -0,0 +1,218 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + fmt, + sync::{Arc, RwLock}, + time::Duration, +}; + +use futures::Future; +use grpcio::{CallOption, Environment}; +use kvproto::{metapb, pdpb, pdpb_grpc::PdClient as RpcClient}; + +use crate::{ + rpc::{ + pd::{ + context::{request_context, PdRequestContext}, + leader::LeaderClient, + request::Request, + PdTimestamp, Region, RegionId, Store, StoreId, + }, + security::SecurityManager, + util::HandyRwLock, + }, + Error, Result, +}; + +const LEADER_CHANGE_RETRY: usize = 10; + +trait PdResponse { + fn header(&self) -> &pdpb::ResponseHeader; +} + +macro_rules! pd_response { + ($type:ty) => { + impl PdResponse for $type { + fn header(&self) -> &pdpb::ResponseHeader { + self.get_header() + } + } + }; +} + +pd_response!(pdpb::GetStoreResponse); +pd_response!(pdpb::GetRegionResponse); +pd_response!(pdpb::GetAllStoresResponse); + +pub struct PdClient { + cluster_id: u64, + leader: Arc>, + timeout: Duration, +} + +impl PdClient { + pub fn connect( + env: Arc, + endpoints: &[String], + security_mgr: Arc, + timeout: Duration, + ) -> Result { + let leader = LeaderClient::connect(env, endpoints, security_mgr, timeout)?; + let cluster_id = leader.rl().cluster_id(); + + Ok(PdClient { + cluster_id, + leader, + timeout, + }) + } + + fn get_leader(&self) -> pdpb::Member { + self.leader.rl().members.get_leader().clone() + } + + fn get_region_and_leader( + &self, + key: &[u8], + ) -> impl Future), Error = Error> { + let mut req = pd_request!(self.cluster_id, pdpb::GetRegionRequest); + req.set_region_key(key.to_owned()); + let key = req.get_region_key().to_owned(); + + self.execute(request_context( + "get_region", + move |cli: &RpcClient, opt: _| cli.get_region_async_opt(&req, opt), + )) + .and_then(move |mut resp| { + let region = if resp.has_region() { + resp.take_region() + } else { + return Err(Error::RegionForKeyNotFound(key)); + }; + let leader = if resp.has_leader() { + Some(resp.take_leader()) + } else { + None + }; + Ok((region, leader)) + }) + } + + fn get_region_and_leader_by_id( + &self, + region_id: u64, + ) -> impl Future), Error = Error> { + let mut req = pd_request!(self.cluster_id, pdpb::GetRegionByIDRequest); + req.set_region_id(region_id); + + self.execute(request_context( + "get_region_by_id", + move |cli: &RpcClient, opt: _| cli.get_region_by_id_async_opt(&req, opt), + )) + .and_then(move |mut resp| { + let region = if resp.has_region() { + resp.take_region() + } else { + return Err(Error::RegionNotFound(region_id, None)); + }; + let leader = if resp.has_leader() { + Some(resp.take_leader()) + } else { + None + }; + Ok((region, leader)) + }) + } + + fn execute( + &self, + mut context: PdRequestContext, + ) -> impl Future + where + Resp: PdResponse + Send + fmt::Debug + 'static, + RpcFuture: Future + Send + 'static, + Executor: FnMut(&RpcClient, CallOption) -> ::grpcio::Result + Send + 'static, + { + let timeout = self.timeout; + let mut executor = context.executor(); + let wrapper = move |cli: &RwLock| { + let option = CallOption::default().timeout(timeout); + let cli = &cli.rl().client; + executor(cli, option) + .unwrap() + .map_err(Error::Grpc) + .and_then(|r| { + { + let header = r.header(); + if header.has_error() { + return Err(internal_err!(header.get_error().get_message())); + } + } + Ok(r) + }) + }; + Request::new( + wrapper, + Arc::clone(&self.leader), + LeaderClient::reconnect, + LEADER_CHANGE_RETRY, + ) + .execute() + .then(move |r| context.done(r)) + } + + pub fn get_all_stores(&self) -> impl Future, Error = Error> { + let req = pd_request!(self.cluster_id, pdpb::GetAllStoresRequest); + + self.execute(request_context( + "get_all_stores", + move |cli: &RpcClient, opt: _| cli.get_all_stores_async_opt(&req, opt), + )) + .map(|mut resp| resp.take_stores().into_iter().map(Into::into).collect()) + } + + pub fn get_store(&self, store_id: StoreId) -> impl Future { + let mut req = pd_request!(self.cluster_id, pdpb::GetStoreRequest); + req.set_store_id(store_id); + + self.execute(request_context( + "get_store", + move |cli: &RpcClient, opt: _| cli.get_store_async_opt(&req, opt), + )) + .map(|mut resp| resp.take_store().into()) + } + + pub fn get_region(&self, key: &[u8]) -> impl Future { + self.get_region_and_leader(key) + .map(|x| Region::new(x.0, x.1)) + } + + pub fn get_region_by_id(&self, id: RegionId) -> impl Future { + self.get_region_and_leader_by_id(id) + .map(|x| Region::new(x.0, x.1)) + } + + pub fn get_ts(&self) -> impl Future { + self.leader.wl().get_ts() + } +} + +impl fmt::Debug for PdClient { + fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { + fmt.debug_struct("PdClient") + .field("cluster_id", &self.cluster_id) + .field("leader", &self.get_leader()) + .finish() + } +} diff --git a/src/rpc/pd/context.rs b/src/rpc/pd/context.rs new file mode 100644 index 00000000..29a79b34 --- /dev/null +++ b/src/rpc/pd/context.rs @@ -0,0 +1,90 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::ops::{Deref, DerefMut}; + +use lazy_static::*; +use prometheus::*; + +use crate::rpc::context::RequestContext; + +pub struct PdRequestContext { + target: RequestContext, +} + +impl Deref for PdRequestContext { + type Target = RequestContext; + + fn deref(&self) -> &Self::Target { + &self.target + } +} + +impl DerefMut for PdRequestContext { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.target + } +} + +pub fn request_context( + cmd: &'static str, + executor: Executor, +) -> PdRequestContext { + PdRequestContext { + target: RequestContext::new( + cmd, + &PD_REQUEST_DURATION_HISTOGRAM_VEC, + &PD_REQUEST_COUNTER_VEC, + &PD_FAILED_REQUEST_DURATION_HISTOGRAM_VEC, + &PD_FAILED_REQUEST_COUNTER_VEC, + executor, + ), + } +} + +pub fn observe_tso_batch(batch_size: usize) -> u32 { + PD_TSO_BATCH_SIZE_HISTOGRAM.observe(batch_size as f64); + batch_size as u32 +} + +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/pd/leader.rs b/src/rpc/pd/leader.rs new file mode 100644 index 00000000..5fbc388c --- /dev/null +++ b/src/rpc/pd/leader.rs @@ -0,0 +1,432 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + sync::{Arc, RwLock}, + thread::{self, JoinHandle}, + time::{Duration, Instant}, +}; + +use futures::{ + sync::{ + mpsc::{unbounded, UnboundedReceiver, UnboundedSender}, + oneshot, + }, + Future, Sink, Stream, +}; +use fxhash::FxHashSet as HashSet; +use grpcio::{CallOption, Environment, WriteFlags}; +use kvproto::{pdpb, pdpb_grpc}; +use log::*; +use tokio_core::reactor::{Core, Handle as OtherHandle}; + +use crate::{ + rpc::{ + pd::{ + context::{observe_tso_batch, request_context}, + PdTimestamp, + }, + security::SecurityManager, + util::HandyRwLock, + }, + Error, Result, +}; + +macro_rules! pd_request { + ($cluster_id:expr, $type:ty) => {{ + let mut request = <$type>::new(); + let mut header = ::kvproto::pdpb::RequestHeader::new(); + header.set_cluster_id($cluster_id); + request.set_header(header); + request + }}; +} + +type TsoChannel = oneshot::Sender; + +enum PdTask { + Init, + Request, + Response(Vec>, pdpb::TsoResponse), +} + +struct PdReactor { + task_tx: Option>>, + tso_tx: UnboundedSender, + tso_rx: Option>, + + handle: Option>, + tso_pending: Option>, + tso_buffer: Option>, + tso_batch: Vec, +} + +impl Drop for PdReactor { + fn drop(&mut self) { + if let Some(handle) = self.handle.take() { + handle.join().unwrap(); + } + } +} + +impl PdReactor { + fn new() -> Self { + let (tso_tx, tso_rx) = unbounded(); + PdReactor { + task_tx: None, + tso_tx, + tso_rx: Some(tso_rx), + handle: None, + tso_buffer: Some(Vec::with_capacity(8)), + tso_batch: Vec::with_capacity(8), + tso_pending: None, + } + } + + fn start(&mut self, client: Arc>) { + if self.handle.is_none() { + info!("starting pd reactor thread"); + let (task_tx, task_rx) = unbounded(); + task_tx.unbounded_send(Some(PdTask::Init)).unwrap(); + self.task_tx = Some(task_tx); + self.handle = Some( + thread::Builder::new() + .name("dispatcher thread".to_owned()) + .spawn(move || Self::poll(&client, task_rx)) + .unwrap(), + ) + } else { + warn!("tso sender and receiver are stale, refreshing.."); + let (tso_tx, tso_rx) = unbounded(); + self.tso_tx = tso_tx; + self.tso_rx = Some(tso_rx); + self.schedule(PdTask::Init); + } + } + + fn schedule(&self, task: PdTask) { + self.task_tx + .as_ref() + .unwrap() + .unbounded_send(Some(task)) + .expect("unbounded send should never fail"); + } + + fn poll(client: &Arc>, rx: UnboundedReceiver>) { + let mut core = Core::new().unwrap(); + let handle = core.handle(); + { + let f = rx.take_while(|t| Ok(t.is_some())).for_each(|t| { + Self::dispatch(&client, t.unwrap(), &handle); + Ok(()) + }); + core.run(f).unwrap(); + } + } + + fn init(client: &Arc>, handle: &OtherHandle) { + let client = Arc::clone(client); + let (tx, rx) = client.wl().client.tso().unwrap(); + let tso_rx = client.wl().reactor.tso_rx.take().unwrap(); + handle.spawn( + tx.sink_map_err(Error::Grpc) + .send_all(tso_rx.then(|r| match r { + Ok(r) => Ok((r, WriteFlags::default())), + Err(()) => Err(internal_err!("failed to recv tso requests")), + })) + .then(|r| match r { + Ok((mut sender, _)) => { + sender.get_mut().cancel(); + Ok(()) + } + Err(e) => { + error!("failed to send tso requests: {:?}", e); + Err(()) + } + }), + ); + handle.spawn( + rx.for_each(move |resp| { + let mut client = client.wl(); + let reactor = &mut client.reactor; + let tso_pending = reactor.tso_pending.take().unwrap(); + reactor.schedule(PdTask::Response(tso_pending, resp)); + if !reactor.tso_batch.is_empty() { + /* schedule another tso_batch of request */ + reactor.schedule(PdTask::Request); + } + Ok(()) + }) + .map_err(|e| panic!("unexpected error: {:?}", e)), + ); + } + + fn tso_request(client: &Arc>) { + let mut client = client.wl(); + let cluster_id = client.cluster_id; + let reactor = &mut client.reactor; + let mut tso_batch = reactor.tso_buffer.take().unwrap(); + tso_batch.extend(reactor.tso_batch.drain(..)); + let mut request = pd_request!(cluster_id, pdpb::TsoRequest); + let batch_size = observe_tso_batch(tso_batch.len()); + request.set_count(batch_size); + reactor.tso_pending = Some(tso_batch); + reactor.tso_tx.unbounded_send(request).unwrap(); + } + + fn tso_response( + client: &Arc>, + mut requests: Vec, + response: &pdpb::TsoResponse, + ) { + let timestamp = response.get_timestamp(); + for (offset, request) in requests.drain(..).enumerate() { + request + .send(PdTimestamp { + physical: timestamp.physical, + logical: timestamp.logical + offset as i64, + }) + .unwrap(); + } + client.wl().reactor.tso_buffer = Some(requests); + } + + fn dispatch(client: &Arc>, task: PdTask, handle: &OtherHandle) { + match task { + PdTask::Request => Self::tso_request(client), + PdTask::Response(requests, response) => Self::tso_response(client, requests, &response), + PdTask::Init => Self::init(client, handle), + } + } + + fn get_ts(&mut self) -> impl Future { + let context = request_context("get_ts", ()); + let (tx, rx) = oneshot::channel::(); + self.tso_batch.push(tx); + if self.tso_pending.is_none() { + /* schedule tso request to run */ + self.schedule(PdTask::Request); + } + rx.map_err(Error::Canceled).then(move |r| context.done(r)) + } +} + +pub struct LeaderClient { + pub client: pdpb_grpc::PdClient, + pub members: pdpb::GetMembersResponse, + + env: Arc, + cluster_id: u64, + security_mgr: Arc, + last_update: Instant, + reactor: PdReactor, + timeout: Duration, +} + +impl LeaderClient { + pub fn cluster_id(&self) -> u64 { + self.cluster_id + } + + pub fn connect( + env: Arc, + endpoints: &[String], + security_mgr: Arc, + timeout: Duration, + ) -> Result>> { + let (client, members) = validate_endpoints(&env, endpoints, &security_mgr, timeout)?; + let cluster_id = members.get_header().get_cluster_id(); + let client = Arc::new(RwLock::new(LeaderClient { + env, + client, + members, + security_mgr, + last_update: Instant::now(), + reactor: PdReactor::new(), + cluster_id, + timeout, + })); + + client.wl().reactor.start(Arc::clone(&client)); + Ok(client) + } + + pub fn get_ts(&mut self) -> impl Future { + self.reactor.get_ts() + } + + // Re-establish connection with PD leader in synchronized fashion. + pub fn reconnect(leader: &Arc>, interval: u64) -> Result<()> { + warn!("updating pd client, block the tokio core"); + let ((client, members), start) = { + let leader = leader.rl(); + if leader.last_update.elapsed() < Duration::from_secs(interval) { + // Avoid unnecessary updating. + return Ok(()); + } + + let start = Instant::now(); + let timeout = leader.timeout; + ( + try_connect_leader(&leader.env, &leader.security_mgr, &leader.members, timeout)?, + start, + ) + }; + + { + let leader_clone = Arc::clone(leader); + let mut leader = leader.wl(); + leader.client = client; + leader.members = members; + leader.last_update = Instant::now(); + leader.reactor.start(leader_clone); + } + warn!("updating PD client done, spent {:?}", start.elapsed()); + Ok(()) + } +} + +pub fn validate_endpoints( + env: &Arc, + endpoints: &[String], + security_mgr: &SecurityManager, + timeout: Duration, +) -> Result<(pdpb_grpc::PdClient, pdpb::GetMembersResponse)> { + let len = endpoints.len(); + let mut endpoints_set = HashSet::with_capacity_and_hasher(len, Default::default()); + + let mut members = None; + let mut cluster_id = None; + for ep in endpoints { + if !endpoints_set.insert(ep) { + return Err(internal_err!("duplicated PD endpoint {}", ep)); + } + + let (_, resp) = match connect(Arc::clone(&env), security_mgr, ep, timeout) { + Ok(resp) => resp, + // Ignore failed PD node. + Err(e) => { + error!("PD endpoint {} failed to respond: {:?}", ep, e); + continue; + } + }; + + // Check cluster ID. + let cid = resp.get_header().get_cluster_id(); + if let Some(sample) = cluster_id { + if sample != cid { + return Err(internal_err!( + "PD response cluster_id mismatch, want {}, got {}", + sample, + cid + )); + } + } else { + cluster_id = Some(cid); + } + // TODO: check all fields later? + + if members.is_none() { + members = Some(resp); + } + } + + match members { + Some(members) => { + let (client, members) = try_connect_leader(&env, security_mgr, &members, timeout)?; + info!("All PD endpoints are consistent: {:?}", endpoints); + Ok((client, members)) + } + _ => Err(internal_err!("PD cluster failed to respond")), + } +} + +fn connect( + env: Arc, + security_mgr: &SecurityManager, + addr: &str, + timeout: Duration, +) -> Result<(pdpb_grpc::PdClient, pdpb::GetMembersResponse)> { + let client = security_mgr.connect(env, addr, pdpb_grpc::PdClient::new)?; + let option = CallOption::default().timeout(timeout); + match client.get_members_opt(&pdpb::GetMembersRequest::new(), option) { + Ok(resp) => Ok((client, resp)), + Err(e) => Err(Error::Grpc(e)), + } +} + +fn try_connect( + env: &Arc, + security_mgr: &SecurityManager, + addr: &str, + cluster_id: u64, + timeout: Duration, +) -> Result<(pdpb_grpc::PdClient, pdpb::GetMembersResponse)> { + let (client, r) = connect(Arc::clone(&env), security_mgr, addr, timeout)?; + let new_cluster_id = r.get_header().get_cluster_id(); + if new_cluster_id != cluster_id { + Err(internal_err!( + "{} no longer belongs to cluster {}, it is in {}", + addr, + cluster_id, + new_cluster_id + )) + } else { + Ok((client, r)) + } +} + +pub fn try_connect_leader( + env: &Arc, + security_mgr: &SecurityManager, + previous: &pdpb::GetMembersResponse, + timeout: Duration, +) -> Result<(pdpb_grpc::PdClient, pdpb::GetMembersResponse)> { + let previous_leader = previous.get_leader(); + let members = previous.get_members(); + let cluster_id = previous.get_header().get_cluster_id(); + let mut resp = None; + // Try to connect to other members, then the previous leader. + 'outer: for m in members + .into_iter() + .filter(|m| *m != previous_leader) + .chain(&[previous_leader.clone()]) + { + for ep in m.get_client_urls() { + match try_connect(&env, security_mgr, ep.as_str(), cluster_id, timeout) { + Ok((_, r)) => { + resp = Some(r); + break 'outer; + } + Err(e) => { + error!("failed to connect to {}, {:?}", ep, e); + continue; + } + } + } + } + + // Then try to connect the PD cluster leader. + if let Some(resp) = resp { + let leader = resp.get_leader().clone(); + for ep in leader.get_client_urls() { + if let Ok((client, r)) = + try_connect(&env, security_mgr, ep.as_str(), cluster_id, timeout) + { + return Ok((client, r)); + } + } + } + + Err(internal_err!("failed to connect to {:?}", members)) +} diff --git a/src/rpc/pd/mod.rs b/src/rpc/pd/mod.rs new file mode 100644 index 00000000..f1e95f27 --- /dev/null +++ b/src/rpc/pd/mod.rs @@ -0,0 +1,180 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::ops::{Deref, DerefMut}; + +use kvproto::{kvrpcpb, metapb}; + +pub use crate::rpc::pd::client::PdClient; +use crate::{Error, Key, Result}; + +#[macro_use] +mod leader; +mod client; +mod context; +mod request; + +pub type RegionId = u64; +pub type StoreId = u64; + +#[derive(Eq, PartialEq, Hash, Clone, Default, Debug)] +pub struct RegionVerId { + pub id: RegionId, + pub conf_ver: u64, + pub ver: u64, +} + +#[derive(Clone, Default, Debug, PartialEq)] +pub struct Region { + pub region: metapb::Region, + pub leader: Option, +} + +impl Deref for Region { + type Target = metapb::Region; + + fn deref(&self) -> &Self::Target { + &self.region + } +} + +impl DerefMut for Region { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.region + } +} + +impl Region { + pub fn new(region: metapb::Region, leader: Option) -> Self { + Region { + region, + leader: leader.map(Peer), + } + } + + pub fn switch_peer(&mut self, _to: StoreId) -> Result<()> { + unimplemented!() + } + + pub fn contains(&self, key: &Key) -> bool { + let key: &[u8] = key.as_ref(); + let start_key = self.region.get_start_key(); + let end_key = self.region.get_end_key(); + start_key <= key && (end_key > key || end_key.is_empty()) + } + + pub fn context(&self) -> Result { + self.leader + .as_ref() + .ok_or_else(|| Error::NotLeader(self.region.get_id(), None)) + .map(|l| { + let mut ctx = kvrpcpb::Context::default(); + ctx.set_region_id(self.region.get_id()); + ctx.set_region_epoch(Clone::clone(self.region.get_region_epoch())); + ctx.set_peer(Clone::clone(l)); + ctx + }) + } + + pub fn start_key(&self) -> &[u8] { + self.region.get_start_key() + } + + pub fn end_key(&self) -> &[u8] { + self.region.get_end_key() + } + + pub fn ver_id(&self) -> RegionVerId { + let region = &self.region; + let epoch = region.get_region_epoch(); + RegionVerId { + id: region.get_id(), + conf_ver: epoch.get_conf_ver(), + ver: epoch.get_version(), + } + } + + pub fn id(&self) -> RegionId { + self.region.get_id() + } + + pub fn peer(&self) -> Result { + self.leader + .as_ref() + .map(Clone::clone) + .map(Into::into) + .ok_or_else(|| Error::StaleEpoch(None)) + } + + pub fn meta(&self) -> metapb::Region { + Clone::clone(&self.region) + } +} + +#[derive(Clone, Debug, PartialEq)] +pub struct Store(metapb::Store); + +impl From for Store { + fn from(store: metapb::Store) -> Store { + Store(store) + } +} + +impl Deref for Store { + type Target = metapb::Store; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl DerefMut for Store { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.0 + } +} + +#[derive(Clone, Default, Debug, PartialEq)] +pub struct Peer(metapb::Peer); + +impl From for Peer { + fn from(peer: metapb::Peer) -> Peer { + Peer(peer) + } +} + +impl Deref for Peer { + type Target = metapb::Peer; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl DerefMut for Peer { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.0 + } +} + +impl Peer { + pub fn into_inner(self) -> metapb::Peer { + self.0 + } +} + +#[derive(Eq, PartialEq, Debug)] +pub struct PdTimestamp { + pub physical: i64, + pub logical: i64, +} diff --git a/src/rpc/pd/request.rs b/src/rpc/pd/request.rs new file mode 100644 index 00000000..fded054c --- /dev/null +++ b/src/rpc/pd/request.rs @@ -0,0 +1,136 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + result, + sync::{Arc, RwLock}, + time::{Duration, Instant}, +}; + +use futures::future::{loop_fn, ok, Either, Future, Loop}; +use log::*; +use tokio_timer::timer::Handle; + +use crate::{rpc::util::GLOBAL_TIMER_HANDLE, Error, Result}; + +pub const RECONNECT_INTERVAL_SEC: u64 = 1; // 1s + +/// The context of sending requets. +pub struct Request { + reconnect_count: usize, + request_sent: usize, + + client: Arc>, + timer: Handle, + + resp: Option>, + func: Func, + reconnect: Reconnect, +} + +const MAX_REQUEST_COUNT: usize = 3; + +impl Request +where + Resp: Send + 'static, + Func: FnMut(&RwLock) -> RespFuture + Send + 'static, + Cli: Send + Sync + 'static, + Reconnect: FnMut(&Arc>, u64) -> Result<()> + Send + 'static, + RespFuture: Future + Send + 'static, +{ + pub fn new(func: Func, client: Arc>, reconnect: Reconnect, retry: usize) -> Self { + Request { + reconnect_count: retry, + request_sent: 0, + client, + timer: GLOBAL_TIMER_HANDLE.clone(), + resp: None, + func, + reconnect, + } + } + + fn reconnect_if_needed(mut self) -> impl Future + Send { + debug!("reconnect remains: {}", self.reconnect_count); + + if self.request_sent < MAX_REQUEST_COUNT { + return Either::A(ok(self)); + } + + // Updating client. + self.reconnect_count -= 1; + + // FIXME: should not block the core. + match (self.reconnect)(&self.client, RECONNECT_INTERVAL_SEC) { + Ok(_) => { + self.request_sent = 0; + Either::A(ok(self)) + } + Err(_) => Either::B( + self.timer + .delay(Instant::now() + Duration::from_secs(RECONNECT_INTERVAL_SEC)) + .then(|_| Err(self)), + ), + } + } + + fn send_and_receive(mut self) -> impl Future + Send { + self.request_sent += 1; + debug!("request sent: {}", self.request_sent); + + ok(self).and_then(|mut ctx| { + let req = (ctx.func)(&ctx.client); + req.then(|resp| match resp { + Ok(resp) => { + ctx.resp = Some(Ok(resp)); + Ok(ctx) + } + Err(err) => { + error!("request failed: {:?}", err); + Err(ctx) + } + }) + }) + } + + fn break_or_continue(ctx: result::Result) -> Result> { + let ctx = match ctx { + Ok(ctx) | Err(ctx) => ctx, + }; + let done = ctx.reconnect_count == 0 || ctx.resp.is_some(); + if done { + Ok(Loop::Break(ctx)) + } else { + Ok(Loop::Continue(ctx)) + } + } + + fn post_loop(ctx: Result) -> Result { + let ctx = ctx.expect("end loop with Ok(_)"); + ctx.resp + .unwrap_or_else(|| Err(internal_err!("fail to request"))) + } + + /// Returns a Future, it is resolves once a future returned by the closure + /// is resolved successfully, otherwise it repeats `retry` times. + pub fn execute(self) -> impl Future { + let ctx = self; + loop_fn(ctx, |ctx| { + ctx.reconnect_if_needed() + .and_then(Self::send_and_receive) + .then(Self::break_or_continue) + }) + .then(Self::post_loop) + .map_err(|e| e) + } +} diff --git a/src/rpc/security.rs b/src/rpc/security.rs new file mode 100644 index 00000000..52503831 --- /dev/null +++ b/src/rpc/security.rs @@ -0,0 +1,170 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + fs::File, + io::Read, + path::{Path, PathBuf}, + ptr, + sync::Arc, + time::Duration, +}; + +use grpcio::{Channel, ChannelBuilder, ChannelCredentialsBuilder, Environment}; +use log::*; + +use crate::Result; + +fn check_pem_file(tag: &str, path: &Path) -> Result> { + match File::open(path) { + Err(e) => Err(internal_err!( + "failed to open {} to load {}: {:?}", + path.display(), + tag, + e + )), + Ok(f) => Ok(Some(f)), + } +} + +fn load_pem(tag: &str, path: &Path) -> Result> { + let mut key = vec![]; + let f = check_pem_file(tag, path)?; + match f { + None => return Ok(vec![]), + Some(mut f) => { + if let Err(e) = f.read_to_end(&mut key) { + return Err(internal_err!( + "failed to load {} from path {}: {:?}", + tag, + path.display(), + e + )); + } + } + } + Ok(key) +} + +struct PrivateKey(Vec); + +impl PrivateKey { + fn load(path: &Path) -> Result { + let key = load_pem("private key", path)?; + Ok(PrivateKey(key)) + } +} + +impl Drop for PrivateKey { + fn drop(&mut self) { + unsafe { + for b in &mut self.0 { + ptr::write_volatile(b, 0); + } + } + } +} + +impl PrivateKey { + fn get(&self) -> Vec { + self.0.clone() + } +} + +#[derive(Default)] +pub struct SecurityManager { + ca: Vec, + cert: Vec, + key: PathBuf, +} + +impl SecurityManager { + pub fn load( + ca_path: impl AsRef, + cert_path: impl AsRef, + key_path: impl Into, + ) -> Result { + let key_path = key_path.into(); + let _ = PrivateKey::load(&key_path)?; + Ok(SecurityManager { + ca: load_pem("ca", ca_path.as_ref())?, + cert: load_pem("certificate", cert_path.as_ref())?, + key: key_path, + }) + } + + pub fn connect( + &self, + env: Arc, + addr: &str, + factory: Factory, + ) -> Result + where + Factory: FnOnce(Channel) -> Client, + { + info!("connect to rpc server at endpoint: {:?}", addr); + let addr = addr + .trim_left_matches("http://") + .trim_left_matches("https://"); + let cb = ChannelBuilder::new(env) + .keepalive_time(Duration::from_secs(10)) + .keepalive_timeout(Duration::from_secs(3)); + + let channel = if self.ca.is_empty() { + cb.connect(addr) + } else { + let key = PrivateKey::load(&self.key)?; + let cred = ChannelCredentialsBuilder::new() + .root_cert(self.ca.clone()) + .cert(self.cert.clone(), key.get()) + .build(); + cb.secure_connect(addr, cred) + }; + + Ok(factory(channel)) + } +} + +#[cfg(test)] +mod tests { + extern crate tempdir; + use super::*; + + use std::fs::File; + use std::io::Write; + use std::path::PathBuf; + + use self::tempdir::TempDir; + + #[test] + fn test_security() { + let temp = TempDir::new("test_cred").unwrap(); + let example_ca = temp.path().join("ca"); + let example_cert = temp.path().join("cert"); + let example_pem = temp.path().join("key"); + for (id, f) in (&[&example_ca, &example_cert, &example_pem]) + .into_iter() + .enumerate() + { + File::create(f).unwrap().write_all(&[id as u8]).unwrap(); + } + let cert_path: PathBuf = format!("{}", example_cert.display()).into(); + let key_path: PathBuf = format!("{}", example_pem.display()).into(); + let ca_path: PathBuf = format!("{}", example_ca.display()).into(); + let mgr = SecurityManager::load(&ca_path, &cert_path, &key_path).unwrap(); + assert_eq!(mgr.ca, vec![0]); + assert_eq!(mgr.cert, vec![1]); + let key = PrivateKey::load(&key_path).unwrap(); + assert_eq!(key.get(), vec![2]); + } +} diff --git a/src/rpc/tikv/client.rs b/src/rpc/tikv/client.rs new file mode 100644 index 00000000..0dfc591e --- /dev/null +++ b/src/rpc/tikv/client.rs @@ -0,0 +1,700 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{fmt, sync::Arc, time::Duration}; + +use futures::Future; +use grpcio::{CallOption, Environment}; +use kvproto::{errorpb, kvrpcpb, tikvpb_grpc::TikvClient}; +use protobuf; + +use crate::{ + rpc::{ + client::{RawContext, TxnContext}, + security::SecurityManager, + tikv::context::{request_context, RequestContext}, + }, + transaction::{Mutation, TxnInfo}, + Error, 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(); + Error::NotLeader( + e.get_region_id(), + Some(format!("{}. Leader: {:?}", message, e.get_leader())), + ) + } else if e.has_region_not_found() { + Error::RegionNotFound(e.get_region_not_found().get_region_id(), Some(message)) + } else if e.has_key_not_in_region() { + let mut e = e.take_key_not_in_region(); + Error::KeyNotInRegion( + e.take_key(), + e.get_region_id(), + e.take_start_key(), + e.take_end_key(), + ) + } else if e.has_stale_epoch() { + Error::StaleEpoch(Some(format!( + "{}. New epoch: {:?}", + message, + e.get_stale_epoch().get_new_regions() + ))) + } else if e.has_server_is_busy() { + let mut e = e.take_server_is_busy(); + Error::ServerIsBusy(e.take_reason(), e.get_backoff_ms()) + } else if e.has_stale_command() { + Error::StaleCommand(message) + } else if e.has_store_not_match() { + let e = e.get_store_not_match(); + Error::StoreNotMatch(e.get_request_store_id(), e.get_actual_store_id(), message) + } else if e.has_raft_entry_too_large() { + let e = e.get_raft_entry_too_large(); + Error::RaftEntryTooLarge(e.get_region_id(), e.get_entry_size(), message) + } else { + Error::InternalError(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); + +impl From for Error { + fn from(e: kvrpcpb::KeyError) -> Error { + Error::KeyError(format!("{:?}", e)) + } +} + +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::KVError(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 { + ($context:expr, $type:ty) => {{ + let mut req = <$type>::new(); + let (region, cf) = $context.into_inner(); + req.set_context(region.into()); + if let Some(cf) = cf { + req.set_cf(cf.into_inner()); + } + req + }}; +} + +macro_rules! txn_request { + ($context:expr, $type:ty) => {{ + let mut req = <$type>::new(); + req.set_context($context.into_inner().into()); + req + }}; +} + +impl From for kvrpcpb::Mutation { + fn from(mutation: Mutation) -> kvrpcpb::Mutation { + let mut pb = kvrpcpb::Mutation::new(); + match mutation { + Mutation::Put(k, v) => { + pb.set_op(kvrpcpb::Op::Put); + pb.set_key(k.into_inner()); + pb.set_value(v.into_inner()); + } + Mutation::Del(k) => { + pb.set_op(kvrpcpb::Op::Del); + pb.set_key(k.into_inner()); + } + Mutation::Lock(k) => { + pb.set_op(kvrpcpb::Op::Lock); + pb.set_key(k.into_inner()); + } + Mutation::Rollback(k) => { + pb.set_op(kvrpcpb::Op::Rollback); + pb.set_key(k.into_inner()); + } + }; + pb + } +} + +impl From for kvrpcpb::TxnInfo { + fn from(txn_info: TxnInfo) -> kvrpcpb::TxnInfo { + let mut pb = kvrpcpb::TxnInfo::new(); + pb.set_txn(txn_info.txn); + pb.set_status(txn_info.status); + pb + } +} + +pub struct KvClient { + client: Arc, + timeout: Duration, + address: String, +} + +impl KvClient { + pub 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, + address: addr.to_owned(), + }) + } + + pub 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_inner()); + req.set_version(version); + + self.execute(request_context( + "kv_get", + move |cli: Arc, opt: _| cli.kv_get_async_opt(&req, opt), + )) + } + + pub 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_inner()); + req.set_end_key(end_key.into_inner()); + 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), + )) + } + + pub 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.map(Into::into).collect()); + req.set_primary_lock(primary_lock.into_inner()); + 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), + )) + } + + pub 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_inner()).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), + )) + } + + pub fn kv_import( + &self, + mutations: impl Iterator, + commit_version: u64, + ) -> impl Future { + let mut req = kvrpcpb::ImportRequest::new(); + req.set_mutations(mutations.map(Into::into).collect()); + req.set_commit_version(commit_version); + + self.execute(request_context( + "kv_import", + move |cli: Arc, opt: _| cli.kv_import_async_opt(&req, opt), + )) + } + + pub 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_inner()); + req.set_start_version(start_version); + + self.execute(request_context( + "kv_cleanup", + move |cli: Arc, opt: _| cli.kv_cleanup_async_opt(&req, opt), + )) + } + + pub 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_inner()).collect()); + req.set_version(version); + + self.execute(request_context( + "kv_batch_get", + move |cli: Arc, opt: _| cli.kv_batch_get_async_opt(&req, opt), + )) + } + + pub 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_inner()).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), + )) + } + + pub 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_inner()); + 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), + )) + } + + pub 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), + )) + } + + pub 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), + )) + } + + pub 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_inner()); + req.set_end_key(end_key.into_inner()); + + self.execute(request_context( + "kv_delete_range", + move |cli: Arc, opt: _| cli.kv_delete_range_async_opt(&req, opt), + )) + } + + pub fn raw_get( + &self, + context: RawContext, + key: Key, + ) -> impl Future { + let mut req = raw_request!(context, kvrpcpb::RawGetRequest); + req.set_key(key.into_inner()); + + self.execute(request_context( + "raw_get", + move |cli: Arc, opt: _| cli.raw_get_async_opt(&req, opt), + )) + .map(|mut resp| resp.take_value().into()) + } + + pub fn raw_batch_get( + &self, + context: RawContext, + keys: impl Iterator, + ) -> impl Future, Error = Error> { + let mut req = raw_request!(context, kvrpcpb::RawBatchGetRequest); + req.set_keys(keys.map(|x| x.into_inner()).collect()); + + self.execute(request_context( + "raw_batch_get", + move |cli: Arc, opt: _| cli.raw_batch_get_async_opt(&req, opt), + )) + .map(|mut resp| Self::convert_from_grpc_pairs(resp.take_pairs())) + } + + pub fn raw_put( + &self, + context: RawContext, + key: Key, + value: Value, + ) -> impl Future { + let mut req = raw_request!(context, kvrpcpb::RawPutRequest); + req.set_key(key.into_inner()); + req.set_value(value.into_inner()); + + self.execute(request_context( + "raw_put", + move |cli: Arc, opt: _| cli.raw_put_async_opt(&req, opt), + )) + .map(|_| ()) + } + + pub fn raw_batch_put( + &self, + context: RawContext, + pairs: Vec, + ) -> impl Future { + let mut req = raw_request!(context, 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(|_| ()) + } + + pub fn raw_delete( + &self, + context: RawContext, + key: Key, + ) -> impl Future { + let mut req = raw_request!(context, kvrpcpb::RawDeleteRequest); + req.set_key(key.into_inner()); + + self.execute(request_context( + "raw_delete", + move |cli: Arc, opt: _| cli.raw_delete_async_opt(&req, opt), + )) + .map(|_| ()) + } + + pub fn raw_batch_delete( + &self, + context: RawContext, + keys: Vec, + ) -> impl Future { + let mut req = raw_request!(context, kvrpcpb::RawBatchDeleteRequest); + req.set_keys(keys.into_iter().map(|x| x.into_inner()).collect()); + + self.execute(request_context( + "raw_batch_delete", + move |cli: Arc, opt: _| cli.raw_batch_delete_async_opt(&req, opt), + )) + .map(|_| ()) + } + + pub fn raw_scan( + &self, + context: RawContext, + start_key: Option, + end_key: Option, + limit: u32, + key_only: bool, + ) -> impl Future, Error = Error> { + let mut req = raw_request!(context, kvrpcpb::RawScanRequest); + start_key + .map(|k| req.set_start_key(k.into_inner())) + .unwrap(); + end_key.map(|k| req.set_end_key(k.into_inner())).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(|mut resp| Self::convert_from_grpc_pairs(resp.take_kvs())) + } + + pub fn raw_batch_scan( + &self, + context: RawContext, + ranges: impl Iterator, Option)>, + each_limit: u32, + key_only: bool, + ) -> impl Future, Error = Error> { + let mut req = raw_request!(context, 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(|mut resp| Self::convert_from_grpc_pairs(resp.take_kvs())) + } + + pub fn raw_delete_range( + &self, + context: RawContext, + start_key: Key, + end_key: Key, + ) -> impl Future { + let mut req = raw_request!(context, kvrpcpb::RawDeleteRangeRequest); + req.set_start_key(start_key.into_inner()); + req.set_end_key(end_key.into_inner()); + + self.execute(request_context( + "raw_delete_range", + move |cli: Arc, opt: _| cli.raw_delete_range_async_opt(&req, opt), + )) + .map(|_| ()) + } + + fn execute( + &self, + mut context: RequestContext, + ) -> impl Future + where + Executor: FnOnce(Arc, CallOption) -> ::grpcio::Result, + RpcFuture: Future, + Resp: HasRegionError + HasError + Sized + Clone, + { + let executor = context.executor(); + executor( + Arc::clone(&self.client), + CallOption::default().timeout(self.timeout), + ) + .unwrap() + .then(|r| match r { + Err(e) => Err(Error::Grpc(e)), + Ok(mut r) => { + if let Some(e) = r.region_error() { + Err(e) + } else if let Some(e) = r.error() { + Err(e) + } else { + Ok(r) + } + } + }) + .then(move |r| context.done(r)) + } + + #[inline] + fn convert_to_grpc_pair(pair: KvPair) -> kvrpcpb::KvPair { + let mut result = kvrpcpb::KvPair::new(); + let (key, value) = pair.into_inner(); + result.set_key(key.into_inner()); + result.set_value(value.into_inner()); + result + } + + #[inline] + fn convert_to_grpc_pairs(pairs: Vec) -> protobuf::RepeatedField { + pairs.into_iter().map(Self::convert_to_grpc_pair).collect() + } + + #[inline] + fn convert_from_grpc_pair(mut pair: kvrpcpb::KvPair) -> KvPair { + KvPair::new(pair.take_key().into(), pair.take_value().into()) + } + + #[inline] + fn convert_from_grpc_pairs(pairs: protobuf::RepeatedField) -> Vec { + pairs + .into_vec() + .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::new(); + start.map(|k| range.set_start_key(k.into_inner())).unwrap(); + end.map(|k| range.set_end_key(k.into_inner())).unwrap(); + range + } + + #[inline] + fn convert_to_grpc_ranges( + ranges: impl Iterator, Option)>, + ) -> protobuf::RepeatedField { + 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("address", &self.address) + .finish() + } +} diff --git a/src/rpc/tikv/context.rs b/src/rpc/tikv/context.rs new file mode 100644 index 00000000..5b9d5935 --- /dev/null +++ b/src/rpc/tikv/context.rs @@ -0,0 +1,58 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use lazy_static::*; +use prometheus::*; + +pub use crate::rpc::context::RequestContext; + +pub fn request_context( + cmd: &'static str, + executor: Executor, +) -> RequestContext { + RequestContext::new( + cmd, + &TIKV_REQUEST_DURATION_HISTOGRAM_VEC, + &TIKV_REQUEST_COUNTER_VEC, + &TIKV_FAILED_REQUEST_DURATION_HISTOGRAM_VEC, + &TIKV_FAILED_REQUEST_COUNTER_VEC, + executor, + ) +} + +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 new file mode 100644 index 00000000..46adc256 --- /dev/null +++ b/src/rpc/tikv/mod.rs @@ -0,0 +1,17 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +pub mod client; +pub mod context; + +pub use self::client::KvClient; diff --git a/src/rpc/util.rs b/src/rpc/util.rs new file mode 100644 index 00000000..1bf07604 --- /dev/null +++ b/src/rpc/util.rs @@ -0,0 +1,144 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + sync::{mpsc, RwLock, RwLockReadGuard, RwLockWriteGuard}, + thread, + time::Duration, +}; + +use lazy_static::*; +use tokio_timer::{self, timer::Handle}; + +macro_rules! internal_err { + ($e:expr) => ({ + $crate::Error::InternalError(format!("[{}:{}]: {}", file!(), line!(), $e)) + }); + ($f:tt, $($arg:expr),+) => ({ + internal_err!(format!($f, $($arg),+)) + }); +} + +/// make a thread name with additional tag inheriting from current thread. +macro_rules! thd_name { + ($name:expr) => {{ + $crate::rpc::util::get_tag_from_thread_name() + .map(|tag| format!("{}::{}", $name, tag)) + .unwrap_or_else(|| $name.to_owned()) + }}; +} + +/// A handy shortcut to replace `RwLock` write/read().unwrap() pattern to +/// shortcut wl and rl. +pub trait HandyRwLock { + fn wl(&self) -> RwLockWriteGuard; + fn rl(&self) -> RwLockReadGuard; +} + +impl HandyRwLock for RwLock { + fn wl(&self) -> RwLockWriteGuard { + self.write().unwrap() + } + + fn rl(&self) -> RwLockReadGuard { + self.read().unwrap() + } +} + +pub fn get_tag_from_thread_name() -> Option { + thread::current() + .name() + .and_then(|name| name.split("::").skip(1).last()) + .map(From::from) +} + +/// Convert Duration to seconds. +#[inline] +pub fn duration_to_sec(d: Duration) -> f64 { + let nanos = f64::from(d.subsec_nanos()); + // Most of case, we can't have so large Duration, so here just panic if overflow now. + d.as_secs() as f64 + (nanos / 1_000_000_000.0) +} + +lazy_static! { + pub static ref GLOBAL_TIMER_HANDLE: Handle = start_global_timer(); +} + +fn start_global_timer() -> Handle { + let (tx, rx) = mpsc::channel(); + thread::Builder::new() + .name(thd_name!("timer")) + .spawn(move || { + let mut timer = tokio_timer::Timer::default(); + tx.send(timer.handle()).unwrap(); + loop { + timer.turn(None).unwrap(); + } + }) + .unwrap(); + rx.recv().unwrap() +} + +#[cfg(test)] +mod tests { + use super::*; + use futures::Future; + use std::*; + + #[test] + fn test_rwlock_deadlock() { + // If the test runs over 60s, then there is a deadlock. + let mu = RwLock::new(Some(1)); + { + let _clone = foo(&mu.rl()); + let mut data = mu.wl(); + assert!(data.is_some()); + *data = None; + } + + { + match foo(&mu.rl()) { + Some(_) | None => { + let res = mu.try_write(); + assert!(res.is_err()); + } + } + } + + #[cfg_attr(feature = "cargo-clippy", allow(clone_on_copy))] + fn foo(a: &Option) -> Option { + a.clone() + } + } + + #[test] + fn test_internal_error() { + let file_name = file!(); + let line_number = line!(); + let e = internal_err!("{}", "hi"); + assert_eq!( + format!("{}", e), + format!("[{}:{}]: hi", file_name, line_number + 1) + ); + } + + #[test] + fn test_global_timer() { + let handle = super::GLOBAL_TIMER_HANDLE.clone(); + let delay = + handle.delay(::std::time::Instant::now() + ::std::time::Duration::from_millis(100)); + let timer = ::std::time::Instant::now(); + delay.wait().unwrap(); + assert!(timer.elapsed() >= ::std::time::Duration::from_millis(100)); + } +} diff --git a/src/transaction.rs b/src/transaction.rs index 1b650d8c..c43f58f3 100644 --- a/src/transaction.rs +++ b/src/transaction.rs @@ -11,10 +11,24 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::{Config, Error, Key, KvPair, Value}; -use futures::{Future, Poll, Stream}; use std::ops::RangeBounds; +use futures::{Future, Poll, Stream}; + +use crate::{Config, Error, Key, KvPair, Value}; + +pub enum Mutation { + Put(Key, Value), + Del(Key), + Lock(Key), + Rollback(Key), +} + +pub struct TxnInfo { + pub txn: u64, + pub status: u64, +} + #[derive(Copy, Clone)] pub struct Timestamp(u64); @@ -242,10 +256,6 @@ impl Transaction { self.snapshot.scan(range) } - pub fn scan_reverse(&self, range: impl RangeBounds) -> Scanner { - self.snapshot.scan_reverse(range) - } - pub fn set(&mut self, key: impl Into, value: impl Into) -> Set { Set::new(key.into(), value.into()) } @@ -270,11 +280,6 @@ impl Snapshot { drop(range); unimplemented!() } - - pub fn scan_reverse(&self, range: impl RangeBounds) -> Scanner { - drop(range); - unimplemented!() - } } pub struct Connect { diff --git a/tests/raw.rs b/tests/raw.rs new file mode 100644 index 00000000..cfe17177 --- /dev/null +++ b/tests/raw.rs @@ -0,0 +1,165 @@ +// Copyright 2018 The TiKV Project Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use futures::future::Future; + +const NUM_TEST_KEYS: u32 = 100; +use tikv_client::{raw::Client, Config, Key, KvPair, Value}; + +fn generate_key(id: i32) -> Key { + format!("testkey_{}", id).into_bytes().into() +} + +fn generate_value(id: i32) -> Value { + format!("testvalue_{}", id).into_bytes().into() +} + +fn wipe_all(client: &Client) { + let test_key_start = generate_key(0); + let test_key_end = generate_key(NUM_TEST_KEYS as i32 - 1); + client + .delete_range(&test_key_start..&test_key_end) + .wait() + .expect("Could not delete test keys"); +} + +fn connect() -> Client { + let client = Client::new(&Config::new(vec!["127.0.0.1:2379"])) + .wait() + .expect("Could not connect to tikv"); + wipe_all(&client); + client +} + +fn test_empty(client: &Client) { + let test_key_start = generate_key(0); + let test_key_end = generate_key(NUM_TEST_KEYS as i32 - 1); + + assert!(client + .scan(&test_key_start..&test_key_end, NUM_TEST_KEYS) + .wait() + .expect("Could not scan") + .is_empty()); +} + +fn test_existence(client: &Client, existing_pairs: Vec, not_existing_keys: Vec) { + let test_key_start = generate_key(0); + let test_key_end = generate_key(NUM_TEST_KEYS as i32 - 1); + + for pair in existing_pairs.iter() { + assert_eq!( + client.get(pair.key()).wait().expect("Could not get value"), + pair.value().clone(), + ); + } + + for key in not_existing_keys.iter() { + let r = client.get(key).wait(); + assert!(r.is_err()); + } + + let mut existing_keys = Vec::with_capacity(existing_pairs.len()); + let mut existing_key_only_pairs = Vec::with_capacity(existing_pairs.len()); + for pair in existing_pairs.iter() { + let key = pair.key().clone(); + existing_keys.push(key.clone()); + existing_key_only_pairs.push(KvPair::new(key, Value::default())); + } + + let mut all_keys = existing_keys.clone(); + all_keys.extend_from_slice(¬_existing_keys); + + assert_eq!( + client + .batch_get(&all_keys) + .wait() + .expect("Could not get value in batch"), + existing_pairs, + ); + + assert_eq!( + client + .batch_get(¬_existing_keys) + .wait() + .expect("Could not get value in batch"), + Vec::new(), + ); + + assert_eq!( + client + .scan(&test_key_start..&test_key_end, NUM_TEST_KEYS) + .wait() + .expect("Could not scan"), + existing_pairs, + ); + + assert_eq!( + client + .scan(&test_key_start..&test_key_end, NUM_TEST_KEYS) + .key_only() + .wait() + .expect("Could not scan"), + existing_key_only_pairs, + ); +} + +#[test] +fn basic_raw_test() { + let client = connect(); + + test_empty(&client); + + assert!(client + .put(generate_key(0), generate_value(0)) + .wait() + .is_ok()); + test_existence( + &client, + vec![KvPair::new(generate_key(0), generate_value(0))], + vec![generate_key(1), generate_key(2)], + ); + + assert!(client.delete(&generate_key(0)).wait().is_ok()); + test_existence( + &client, + Vec::new(), + vec![generate_key(0), generate_key(1), generate_key(2)], + ); + + let pairs: Vec = (0..10) + .map(|i| KvPair::new(generate_key(i), generate_value(i))) + .collect(); + assert!(client.batch_put(pairs.clone()).wait().is_ok()); + test_existence( + &client, + pairs.clone(), + vec![generate_key(10), generate_key(11), generate_key(12)], + ); + + let keys: Vec = vec![generate_key(8), generate_key(9)]; + assert!(client.batch_delete(&keys).wait().is_ok()); + let mut pairs = pairs; + pairs.truncate(8); + test_existence( + &client, + pairs.clone(), + vec![generate_key(8), generate_key(9), generate_key(10)], + ); + + wipe_all(&client); + test_existence( + &client, + Vec::new(), + pairs.into_iter().map(|x| x.into_inner().0).collect(), + ); +} From d9fd81bdc6662a620646a5366bcab59be11181fe Mon Sep 17 00:00:00 2001 From: Ana Hobden Date: Fri, 11 Jan 2019 11:26:27 -0800 Subject: [PATCH 02/20] Document most public code. Signed-off-by: Ana Hobden --- examples/raw.rs | 79 +++--- examples/transaction.rs | 4 +- rust-toolchain | 2 +- src/errors.rs | 24 +- src/lib.rs | 398 ++++++++++++++++++++++++++--- src/raw.rs | 446 +++++++++++++++++++++++---------- src/rpc/client.rs | 4 +- src/rpc/tikv/client.rs | 4 +- src/transaction.rs | 537 +++++++++++++++++++++++++++++++--------- tests/raw.rs | 25 +- 10 files changed, 1182 insertions(+), 341 deletions(-) diff --git a/examples/raw.rs b/examples/raw.rs index 2f8b4d21..efa273ed 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -13,49 +13,69 @@ use futures::future::Future; use std::path::PathBuf; -use tikv_client::*; +use tikv_client::{Result, Config, raw::Client, Key, Value}; -fn main() { - let config = Config::new(vec!["127.0.0.1:2379"]).with_security( +const KEY: &str = "TiKV"; +const VALUE: &str = "Rust"; +const CUSTOM_CF: &str = "custom_cf"; + +fn main() -> Result<()> { + // Create a configuration to use for the example. + // Optionally encrypt the traffic. + let config = Config::new(vec![ + "192.168.0.101:3379", // Avoid a single point of failure, + "192.168.0.100:3379", // use at least two PD endpoints. + ]).with_security( PathBuf::from("/path/to/ca.pem"), PathBuf::from("/path/to/client.pem"), PathBuf::from("/path/to/client-key.pem"), ); - let raw = raw::Client::new(&config) - .wait() - .expect("Could not connect to tikv"); - let key: Key = b"Company".to_vec().into(); - let value: Value = b"PingCAP".to_vec().into(); + // When we first create a client we recieve a `Connect` structure which must be resolved before + // the client is actually connected and usable. + let unconnnected_client = Client::new(&config); + let client = unconnnected_client.wait()?; - raw.put(key.clone(), value.clone()) - .cf("test_cf") - .wait() - .expect("Could not put kv pair to tikv"); - println!("Successfully put {:?}:{:?} to tikv", key, value); + // 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 + // place. + // + // Here we set the key `TiKV` to have the value `Rust` associated with it. + let put_request = client.put(KEY, VALUE); + let put_result: () = put_request.wait()?; // Returns a `tikv_client::Error` on failure. + println!("Put key \"{}\", value \"{}\".", KEY, VALUE); - let value = raw - .get(&key) - .cf("test_cf") - .wait() - .expect("Could not get value"); - println!("Found val: {:?} for key: {:?}", value, key); + // + // Unlike a standard Rust HashMap all calls take owned values. This is because under the hood + // protobufs must take ownership of the data. If we only took a borrow we'd need to internally // clone it. This is against Rust API guidelines, so you must manage this yourself. + // + // Above, you saw we can use a `&'static str`, this is primarily for making examples short. + // This type is practical to use for real things, and usage forces an internal copy. + // + // It is best to pass a `Vec` in terms of explictness and speed. `String`s and a few other + // types are supported as well, but it all ends up as `Vec` in the end. + let key: String = String::from(KEY); + let value: Value = client.get(key.clone()).wait()?; + assert_eq!(value.as_ref(), VALUE.as_bytes()); + println!("Get key \"{:?}\" returned value \"{:?}\".", value, KEY); - raw.delete(&key) - .cf("test_cf") + // You can also set the `ColumnFamily` used by the request. + // This is *advanced usage* and should have some special considerations. + let req = client.delete(key.clone()) + .cf(CUSTOM_CF) .wait() .expect("Could not delete value"); println!("Key: {:?} deleted", key); - raw.get(&key) + client.get(key) .cf("test_cf") .wait() .expect_err("Get returned value for not existing key"); - let keys = vec![b"k1".to_vec().into(), b"k2".to_vec().into()]; + let keys = vec![Key::from(b"k1".to_vec()), Key::from(b"k2".to_vec())]; - let values = raw - .batch_get(&keys) + let values = client + .batch_get(keys.clone()) .cf("test_cf") .wait() .expect("Could not get values"); @@ -63,16 +83,19 @@ fn main() { let start: Key = b"k1".to_vec().into(); let end: Key = b"k2".to_vec().into(); - raw.scan(&start..&end, 10) + client.scan(start.clone()..end.clone(), 10) .cf("test_cf") .key_only() .wait() .expect("Could not scan"); - let ranges = [&start..&end, &start..&end]; - raw.batch_scan(&ranges, 10) + let ranges = vec![start.clone()..end.clone(), start.clone()..end.clone()]; + client.batch_scan(ranges, 10) .cf("test_cf") .key_only() .wait() .expect("Could not batch scan"); + + // Cleanly exit. + Ok(()) } diff --git a/examples/transaction.rs b/examples/transaction.rs index 7904ca8c..f2e7a567 100644 --- a/examples/transaction.rs +++ b/examples/transaction.rs @@ -32,7 +32,7 @@ fn puts(client: &Client, pairs: impl IntoIterator>) { txn.commit().wait().expect("Could not commit transaction"); } -fn get(client: &Client, key: &Key) -> Value { +fn get(client: &Client, key: Key) -> Value { let txn = client.begin(); txn.get(key).wait().expect("Could not get value") } @@ -88,7 +88,7 @@ fn main() { // get let key1: Key = b"key1".to_vec().into(); - let value1 = get(&txn, &key1); + let value1 = get(&txn, key1.clone()); println!("{:?}", (key1, value1)); // scan diff --git a/rust-toolchain b/rust-toolchain index 870bbe4e..2bf5ad04 100644 --- a/rust-toolchain +++ b/rust-toolchain @@ -1 +1 @@ -stable \ No newline at end of file +stable diff --git a/src/errors.rs b/src/errors.rs index 95a8d542..f9c93275 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -11,55 +11,70 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::{error, result}; - use grpcio; use quick_error::quick_error; +use std::{error, result}; -quick_error! { +quick_error!{ + /// An error originating from the TiKV client or dependencies. + /// + /// This client currently uses [`quick_error`](https://docs.rs/quick-error/1.2.2/quick_error/) + /// for errors. *This may change in future versions.* #[derive(Debug)] pub enum Error { + /// Wraps a a `std::io::Error`. Io(err: ::std::io::Error) { from() cause(err) description(err.description()) } + /// Wraps a `grpcio::Error`. Grpc(err: grpcio::Error) { from() cause(err) description(err.description()) } + /// Represents that a futures oneshot channel was cancelled. Canceled(err: ::futures::sync::oneshot::Canceled) { from() cause(err) description(err.description()) } + /// An unknown error. + /// + /// Generally, this is not an expected error. Please report it if encountered. Other(err: Box) { from() cause(err.as_ref()) description(err.description()) display("unknown error {:?}", err) } + /// A region was not found for the given key. RegionForKeyNotFound(key: Vec) { description("region is not found") display("region is not found for key {:?}", key) } + /// A region was not found. RegionNotFound(region_id: u64, message: Option) { description("region is not found") display("region {:?} is not found. {}", region_id, message.as_ref().unwrap_or(&"".to_owned())) } + /// The peer is not a leader of the given region. NotLeader(region_id: u64, message: Option) { description("peer is not leader") display("peer is not leader for region {}. {}", region_id, message.as_ref().unwrap_or(&"".to_owned())) } + /// The store does not match. StoreNotMatch(request_store_id: u64, actual_store_id: u64, message: String) { description("store not match") display("requesting store '{}' when actual store is '{}'. {}", request_store_id, actual_store_id, message) } + /// The given key is not eithin the given region. KeyNotInRegion(key: Vec, region_id: u64, start_key: Vec, end_key: Vec) { description("region is not found") display("key {:?} is not in region {:?}: [{:?}, {:?})", key, region_id, start_key, end_key) } + /// A stale epoch. StaleEpoch(message: Option) { description("stale epoch") display("{}", message.as_ref().unwrap_or(&"".to_owned())) @@ -68,10 +83,12 @@ quick_error! { description("stale command") display("{}", message) } + /// The server is too busy. ServerIsBusy(reason: String, backoff: u64) { description("server is busy") display("server is busy: {:?}. Backoff {} ms", reason, backoff) } + /// The given raft entry is too large for the region. RaftEntryTooLarge(region_id: u64, entry_size: u64, message: String) { description("raft entry too large") display("{:?} bytes raft entry of region {:?} is too large. {}", entry_size, region_id, message) @@ -111,4 +128,5 @@ quick_error! { } } +/// A result holding an [`Error`](enum.Error.html). pub type Result = result::Result; diff --git a/src/lib.rs b/src/lib.rs index 48cd651f..ccaabf40 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -10,37 +10,92 @@ // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. - #![recursion_limit = "128"] #![type_length_limit = "2097152"] -use std::{convert::AsRef, ops::Deref, path::PathBuf, time::Duration}; -use futures::Future; use serde_derive::*; +use std::{ + ops::{Deref, DerefMut, Bound, Range, RangeFrom, RangeFull, RangeInclusive, RangeTo, RangeToInclusive}, + path::PathBuf, + time::Duration, +}; +use futures::Future; -pub use crate::errors::{Error, Result}; - -pub mod errors; +mod errors; pub mod raw; mod rpc; pub mod transaction; +#[doc(inline)] +pub use crate::errors::Error; +#[doc(inline)] +pub use crate::errors::Result; + +/// The key part of a key/value pair. +/// +/// In TiKV, keys are an ordered sequence of bytes. This has an advantage over choosing `String` as valid `UTF-8` is not required. This means that the user is permitted to store any data they wish, +/// as long as it can be represented by bytes. (Which is to say, pretty much anything!) +/// +/// This is a *wrapper type* that implements `Deref>` so it can be used like one transparently. +/// +/// This type also implements `From` for many types. With one exception, these are all done without reallocation. Using a `&'static str`, like many examples do for simplicity, has an internal +/// allocation cost. +/// +/// This type wraps around an owned value, so it should be treated it like `String` or `Vec` +/// over a `&str` or `&[u8]`. +/// +/// ```rust +/// use tikv_client::Key; +/// +/// let static_str: &'static str = "TiKV"; +/// let from_static_str = Key::from(static_str); +/// +/// let string: String = String::from(static_str); +/// let from_string = Key::from(string); +/// assert_eq!(from_static_str, from_string); +/// +/// let vec: Vec = static_str.as_bytes().to_vec(); +/// let from_vec = Key::from(vec); +/// assert_eq!(from_static_str, from_vec); +/// +/// let bytes = static_str.as_bytes().to_vec(); +/// let from_bytes = Key::from(bytes); +/// assert_eq!(from_static_str, from_bytes); +/// ``` +/// +/// **But, you should not need to worry about all this:** Many functions which accept a `Key` +/// accept an `Into`, which means all of the above types can be passed directly to those +/// functions. #[derive(Default, Clone, Eq, PartialEq, Ord, PartialOrd, Hash, Debug)] pub struct Key(Vec); -#[derive(Default, Clone, Eq, PartialEq, Hash, Debug)] -pub struct Value(Vec); -#[derive(Default, Clone, Eq, PartialEq, Debug)] -pub struct KvPair(Key, Value); + +impl Key { + pub fn new(value: Vec) -> Self { + Key(value) + } + + fn into_inner(self) -> Vec { + self.0 + } +} impl From> for Key { - fn from(vec: Vec) -> Key { - Key(vec) + fn from(v: Vec) -> Self { + Key(v) } } -impl<'a> From<&'a [u8]> for Key { - fn from(s: &'a [u8]) -> Key { - Key(s.to_vec()) +impl From for Key { + fn from(v: String) -> Key { + Key(v.into_bytes()) + } +} + +impl<'a> From<&'static str> for Key { + fn from(v: &'static str) -> Key { + let mut vec = Vec::new(); + vec.extend_from_slice(v.as_bytes()); + Key(vec) } } @@ -64,33 +119,77 @@ impl Deref for Key { } } -impl Key { - fn into_inner(self) -> Vec { - self.0 +impl DerefMut for Key { + fn deref_mut<'a>(&'a mut self) -> &'a mut [u8] { + &mut self.0 } } -impl From> for Value { - fn from(vec: Vec) -> Value { - Value(vec) +/// The value part of a key/value pair. +/// +/// In TiKV, values are an ordered sequence of bytes. This has an advantage over choosing `String` as valid `UTF-8` is not required. This means that the user is permitted to store any data they wish, +/// as long as it can be represented by bytes. (Which is to say, pretty much anything!) +/// +/// This is a *wrapper type* that implements `Deref>` so it can be used like one transparently. +/// +/// This type also implements `From` for many types. With one exception, these are all done without reallocation. Using a `&'static str`, like many examples do for simplicity, has an internal +/// allocation cost. +/// +/// This type wraps around an owned value, so it should be treated it like `String` or `Vec` +/// over a `&str` or `&[u8]`. +/// +/// ```rust +/// use tikv_client::Value; +/// +/// let static_str: &'static str = "TiKV"; +/// let from_static_str = Value::from(static_str); +/// +/// let string: String = String::from(static_str); +/// let from_string = Value::from(string); +/// assert_eq!(from_static_str, from_string); +/// +/// let vec: Vec = static_str.as_bytes().to_vec(); +/// let from_vec = Value::from(vec); +/// assert_eq!(from_static_str, from_vec); +/// +/// let bytes = static_str.as_bytes().to_vec(); +/// let from_bytes = Value::from(bytes); +/// assert_eq!(from_static_str, from_bytes); +/// ``` +/// +/// **But, you should not need to worry about all this:** Many functions which accept a `Value` +/// accept an `Into`, which means all of the above types can be passed directly to those +/// functions. +#[derive(Default, Clone, Eq, PartialEq, Hash, Debug)] +pub struct Value(Vec); + +impl Value { + pub fn new(value: Vec) -> Self { + Value(value) + } + + fn into_inner(self) -> Vec { + self.0 } } -impl<'a> From<&'a [u8]> for Value { - fn from(s: &'a [u8]) -> Value { - Value(s.to_vec()) +impl From> for Value { + fn from(v: Vec) -> Self { + Value(v) } } -impl AsRef for Value { - fn as_ref(&self) -> &Self { - self +impl From for Value { + fn from(v: String) -> Value { + Value(v.into_bytes()) } } -impl AsRef<[u8]> for Value { - fn as_ref(&self) -> &[u8] { - &self.0 +impl From<&'static str> for Value { + fn from(v: &'static str) -> Value { + let mut vec = Vec::new(); + vec.extend_from_slice(v.as_bytes()); + Value(vec) } } @@ -102,21 +201,35 @@ impl Deref for Value { } } -impl Value { - fn into_inner(self) -> Vec { - self.0 - } -} +/// A key/value pair. +/// +/// ```rust +/// # use tikv_client::{Key, Value, KvPair}; +/// let key = "key"; +/// let value = "value"; +/// let constructed = KvPair::new(key, value); +/// let from_tuple = KvPair::from((key, value)); +/// assert_eq!(constructed, from_tuple); +/// ``` +/// +/// **But, you should not need to worry about all this:** Many functions which accept a `KvPair` +/// accept an `Into`, which means all of the above types can be passed directly to those +/// functions. +#[derive(Default, Clone, Eq, PartialEq, Debug)] +pub struct KvPair(Key, Value); impl KvPair { - pub fn new(key: Key, value: Value) -> Self { - KvPair(key, value) + /// Create a new `KvPair`. + pub fn new(key: impl Into, value: impl Into) -> Self { + KvPair(key.into(), value.into()) } + /// Immutably borrow the `Key` part of the `KvPair`. pub fn key(&self) -> &Key { &self.0 } + /// Immutably borrow the `Value` part of the `KvPair`. pub fn value(&self) -> &Value { &self.1 } @@ -124,14 +237,54 @@ impl KvPair { pub fn into_inner(self) -> (Key, Value) { (self.0, self.1) } + + pub fn into_key(self) -> Key { + self.0 + } + + pub fn into_value(self) -> Value { + self.1 + } + + /// Mutably borrow the `Key` part of the `KvPair`. + pub fn key_mut(&mut self) -> &mut Key { + &mut self.0 + } + + /// Mutably borrow the `Value` part of the `KvPair`. + pub fn value_mut(&mut self) -> &mut Value { + &mut self.1 + } + + /// Set the `Key` part of the `KvPair`. + pub fn set_key(&mut self, k: impl Into) { + self.0 = k.into(); + } + + /// Set the `Value` part of the `KvPair`. + pub fn set_value(&mut self, v: impl Into) { + self.1 = v.into(); + } } -impl From<(Key, Value)> for KvPair { - fn from(pair: (Key, Value)) -> KvPair { - KvPair(pair.0, pair.1) +impl From<(K, V)> for KvPair +where K: Into, V: Into { + fn from((k, v): (K, V)) -> Self { + KvPair(k.into(), v.into()) } } +/// The configuration for either a [`raw::Client`](raw/struct.Client.html) or a +/// [`transaction::Client`](transaction/struct.Client.html). +/// +/// Because TiKV is managed by a [PD](https://github.com/pingcap/pd/) cluster, the endpoints for PD +/// must be provided, **not** the TiKV nodes. +/// +/// It's important to **include more than one PD endpoint** (include all, if possible!) This helps avoid having a *single point of failure*. +/// +/// By default, this client will use an insecure connection over encryption-on-the-wire. Your +/// deployment may have chosen to rely on security measures such as a private network, or a VPN +/// layer providing secure transmission. TiKV does not currently offer encryption-at-rest. #[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq)] #[serde(default)] #[serde(rename_all = "kebab-case")] @@ -144,6 +297,14 @@ pub struct Config { } impl Config { + /// Create a new [`Config`](struct.Config.html) which coordinates with the given PD endpoints. + /// + /// It's important to **include more than one PD endpoint** (include all, if possible!) This helps avoid having a *single point of failure*. + /// + /// ```rust + /// # use tikv_client::Config; + /// let config = Config::new(vec!["192.168.0.100:2379", "192.168.0.101:2379"]); + /// ``` pub fn new(pd_endpoints: impl IntoIterator>) -> Self { Config { pd_endpoints: pd_endpoints.into_iter().map(Into::into).collect(), @@ -154,6 +315,15 @@ impl Config { } } + /// Set the certificate authority, certificate, and key locations for the [`Config`](struct.Config.html). + /// + /// By default, TiKV connections do not have utilize transport layer security. Enable it by setting these values. + /// + /// ```rust + /// # use tikv_client::Config; + /// let config = Config::new(vec!["192.168.0.100:2379", "192.168.0.101:2379"]) + /// .with_security("root.ca", "internal.cert", "internal.key"); + /// ``` pub fn with_security( mut self, ca_path: impl Into, @@ -173,3 +343,151 @@ impl Config { } pub type KvFuture = Box>; + +/// A convenience trait for expressing ranges. +/// +/// In TiKV, keys are an ordered sequence of bytes. This means we can have ranges over those +/// bytes. Eg `001` is before `010`. +/// +/// This trait has implementations for common range types like `a..b`, `a..=b` where `a` and `b` +/// `impl Into`. You could implement this for your own types. +/// +/// ```rust +/// use tikv_client::{KeyRange, Key}; +/// use std::ops::{Range, RangeInclusive, RangeTo, RangeToInclusive, RangeFrom, RangeFull, Bound}; +/// +/// let explict_range: Range = Range { start: Key::from("Rust"), end: Key::from("TiKV") }; +/// let from_explict_range = explict_range.into_bounds(); +/// +/// let range: Range<&str> = "Rust".."TiKV"; +/// let from_range = range.into_bounds(); +/// assert_eq!(from_explict_range, from_range); +/// +/// let range: RangeInclusive<&str> = "Rust"..="TiKV"; +/// let from_range = range.into_bounds(); +/// assert_eq!( +/// (Bound::Included(Key::from("Rust")), Bound::Included(Key::from("TiKV"))), +/// from_range +/// ); +/// +/// let range_from: RangeFrom<&str> = "Rust"..; +/// let from_range_from = range_from.into_bounds(); +/// assert_eq!( +/// (Bound::Included(Key::from("Rust")), Bound::Unbounded), +/// from_range_from, +/// ); +/// +/// let range_to: RangeTo<&str> = .."TiKV"; +/// let from_range_to = range_to.into_bounds(); +/// assert_eq!( +/// (Bound::Unbounded, Bound::Excluded(Key::from("TiKV"))), +/// from_range_to, +/// ); +/// +/// let range_to_inclusive: RangeToInclusive<&str> = ..="TiKV"; +/// let from_range_to_inclusive = range_to_inclusive.into_bounds(); +/// assert_eq!( +/// (Bound::Unbounded, Bound::Included(Key::from("TiKV"))), +/// from_range_to_inclusive, +/// ); +/// +/// let range_full: RangeFull = ..; +/// let from_range_full = range_full.into_bounds(); +/// assert_eq!( +/// (Bound::Unbounded, Bound::Unbounded), +/// from_range_full +/// ); +/// ``` +/// +/// **But, you should not need to worry about all this:** Many functions accept a `impl KeyRange` +/// which means all of the above types can be passed directly to those functions. +pub trait KeyRange: Sized { + fn into_bounds(self) -> (Bound, Bound); + fn into_keys(self) -> Result<(Key, Option)> { + range_to_keys(self.into_bounds()) + } +} + +fn range_to_keys(range: (Bound, Bound)) -> Result<(Key, Option)> { + let start = match range.0 { + Bound::Included(v) => v, + Bound::Excluded(mut v) => { + let len = v.len(); + if len > 0 { + v.deref_mut().get_mut(len -1).map(|v| { *v += 1; v }); + } + v + }, + Bound::Unbounded => return Err(Error::InvalidKeyRange), + }; + let end = match range.1 { + Bound::Included(v) => Some(v), + Bound::Excluded(mut v) => Some({ + let len = v.len(); + if len > 0 { + v.deref_mut().get_mut(len -1).map(|v| { *v -= 1; v }); + } + v + }), + Bound::Unbounded => None, + }; + Ok((start, end)) +} + +impl> KeyRange for Range { + fn into_bounds(self) -> (Bound, Bound) { + ( + Bound::Included(self.start.into()), + Bound::Excluded(self.end.into()), + ) + } +} + +impl> KeyRange for RangeFrom { + fn into_bounds(self) -> (Bound, Bound) { + (Bound::Included(self.start.into()), Bound::Unbounded) + } +} + +impl KeyRange for RangeFull { + fn into_bounds(self) -> (Bound, Bound) { + (Bound::Unbounded, Bound::Unbounded) + } +} + +impl> KeyRange for RangeInclusive { + fn into_bounds(self) -> (Bound, Bound) { + let (start, end) = self.into_inner(); + (Bound::Included(start.into()), Bound::Included(end.into())) + } +} + +impl> KeyRange for RangeTo { + fn into_bounds(self) -> (Bound, Bound) { + (Bound::Unbounded, Bound::Excluded(self.end.into())) + } +} + +impl> KeyRange for RangeToInclusive { + fn into_bounds(self) -> (Bound, Bound) { + (Bound::Unbounded, Bound::Included(self.end.into())) + } +} + +impl> KeyRange for (Bound, Bound) { + fn into_bounds(self) -> (Bound, Bound) { + (transmute_bound(self.0), transmute_bound(self.1)) + } +} + +fn transmute_bound(b: Bound) -> Bound +where + K: Into, +{ + use std::ops::Bound::*; + match b { + Included(k) => Included(k.into()), + Excluded(k) => Excluded(k.into()), + Unbounded => Unbounded, + } +} diff --git a/src/raw.rs b/src/raw.rs index d6a060fb..13ed10cf 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -11,15 +11,282 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::{ - ops::{Bound, Deref, DerefMut, RangeBounds}, - sync::Arc, -}; +/*! Raw related functionality. -use futures::{future, Async, Future, Poll}; +Using the [`raw::Client`](struct.Client.html) you can utilize TiKV's raw interface. -use crate::{rpc::RpcClient, Config, Error, Key, KvFuture, KvPair, Result, Value}; +This interface offers optimal performance as it does not require coordination with a timestamp +oracle, while the transactional interface does. +**Warning:** It is not advisible to use the both raw and transactional functionality in the same keyspace. + */ + +use crate::{rpc::RpcClient, Config, Error, Key, KeyRange, KvFuture, KvPair, Result, Value}; +use futures::{future, Future, Poll, Async}; +use std::{u32, sync::Arc, ops::{Bound, Deref, RangeBounds, DerefMut}}; + + +/// The TiKV raw [`Client`](struct.Client.html) is used to issue requests to the TiKV server and PD cluster. +pub struct Client { + rpc: Arc +} + +impl Client { + /// Create a new [`Client`](struct.Client.html) once the [`Connect`](struct.Connect.html) resolves. + /// + /// ```rust,no_run + /// use tikv_client::{Config, raw::Client}; + /// use futures::Future; + /// let connect = Client::new(&Config::default()); + /// let client = connect.wait(); + /// ``` + #[cfg_attr(feature = "cargo-clippy", allow(clippy::new_ret_no_self))] + pub fn new(config: &Config) -> Connect { + Connect::new(config.clone()) + } + + #[inline] + fn rpc(&self) -> Arc { + Arc::clone(&self.rpc) + } + + /// Create a new [`Get`](struct.Get.html) request. + /// + /// Once resolved this request will result in the fetching of the value associated with the + /// given key. + /// + /// ```rust,no_run + /// # use tikv_client::{Value, Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let key = "TiKV"; + /// let req = connected_client.get(key); + /// let result: Value = req.wait().unwrap(); + /// ``` + pub fn get(&self, key: impl Into) -> Get { + Get::new(self.rpc(), GetInner::new(key.into())) + } + + /// Create a new [`BatchGet`](struct.BatchGet.html) request. + /// + /// Once resolved this request will result in the fetching of the values associated with the + /// given keys. + /// + /// ```rust,no_run + /// # use tikv_client::{KvPair, Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let keys = vec!["TiKV", "TiDB"]; + /// let req = connected_client.batch_get(keys); + /// let result: Vec = req.wait().unwrap(); + /// ``` + pub fn batch_get(&self, keys: impl IntoIterator>) -> BatchGet { + BatchGet::new(self.rpc(), BatchGetInner::new(keys.into_iter().map(Into::into).collect())) + } + + /// Create a new [`Put`](struct.Put.html) request. + /// + /// Once resolved this request will result in the setting of the value associated with the given key. + /// + /// ```rust,no_run + /// # use tikv_client::{Key, Value, Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let key = "TiKV"; + /// let val = "TiKV"; + /// let req = connected_client.put(key, val); + /// let result: () = req.wait().unwrap(); + /// ``` + pub fn put(&self, key: impl Into, value: impl Into) -> Put { + Put::new(self.rpc(), PutInner::new(key.into(), value.into())) + } + + /// Create a new [`BatchPut`](struct.BatchPut.html) request. + /// + /// Once resolved this request will result in the setting of the value associated with the given key. + /// + /// ```rust,no_run + /// # use tikv_client::{Error, Result, KvPair, Key, Value, Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let kvpair1 = ("PD", "Go"); + /// let kvpair2 = ("TiKV", "Rust"); + /// let iterable = vec![kvpair1, kvpair2]; + /// let req = connected_client.batch_put(iterable); + /// let result: () = req.wait().unwrap(); + /// ``` + pub fn batch_put(&self, pairs: impl IntoIterator>) -> BatchPut { + BatchPut::new(self.rpc(), BatchPutInner::new(pairs.into_iter().map(Into::into).collect())) + } + + /// Create a new [`Delete`](struct.Delete.html) request. + /// + /// Once resolved this request will result in the deletion of the given key. + /// + /// ```rust,no_run + /// # use tikv_client::{Key, Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let key = "TiKV"; + /// let req = connected_client.delete(key); + /// let result: () = req.wait().unwrap(); + /// ``` + pub fn delete(&self, key: impl Into) -> Delete { + Delete::new(self.rpc(), DeleteInner::new(key.into())) + } + + /// Create a new [`BatchDelete`](struct.BatchDelete.html) request. + /// + /// Once resolved this request will result in the deletion of the given keys. + /// + /// ```rust,no_run + /// # use tikv_client::{Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let keys = vec!["TiKV", "TiDB"]; + /// let req = connected_client.batch_delete(keys); + /// let result: () = req.wait().unwrap(); + /// ``` + pub fn batch_delete(&self, keys: impl IntoIterator>) -> BatchDelete { + BatchDelete::new(self.rpc(), BatchDeleteInner::new(keys.into_iter().map(Into::into).collect())) + } + + /// Create a new [`Scan`](struct.Scan.html) request. + /// + /// Once resolved this request will result in a scanner over the given keys. + /// + /// If not passed a `limit` parameter, it will default to `u32::MAX`. + /// + /// ```rust,no_run + /// # use tikv_client::{KvPair, Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let inclusive_range = "TiKV"..="TiDB"; + /// let req = connected_client.scan(inclusive_range, 2); + /// let result: Vec = req.wait().unwrap(); + /// ``` + pub fn scan(&self, range: impl KeyRange, limit: impl Into>) -> Scan { + Scan::new(self.rpc(), ScanInner::new(range.into_bounds(), limit.into().unwrap_or(u32::MAX))) + } + + /// Create a new [`BatchScan`](struct.BatchScan.html) request. + /// + /// Once resolved this request will result in a set of scanners over the given keys. + /// + /// If not passed a `limit` parameter, it will default to `u32::MAX`. + /// + /// ```rust,no_run + /// # use tikv_client::{Key, Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().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.wait(); + /// ``` + pub fn batch_scan( + &self, + ranges: impl IntoIterator, + each_limit: impl Into>, + ) -> BatchScan { + BatchScan::new( + self.rpc(), + BatchScanInner::new( + ranges.into_iter().map(KeyRange::into_keys).collect(), + each_limit.into().unwrap_or(u32::MAX), + ), + ) + } + + /// Create a new [`DeleteRange`](struct.DeleteRange.html) request. + /// + /// Once resolved this request will result in the deletion of all keys over the given range. + /// + /// If not passed a `limit` parameter, it will default to `u32::MAX`. + /// + /// ```rust,no_run + /// # use tikv_client::{Key, Config, raw::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let inclusive_range = "TiKV"..="TiDB"; + /// let req = connected_client.delete_range(inclusive_range); + /// let result: () = req.wait().unwrap(); + /// ``` + pub fn delete_range(&self, range: impl KeyRange) -> DeleteRange { + DeleteRange::new(self.rpc(), DeleteRangeInner::new(range.into_keys())) + } +} + +/// An unresolved [`Client`](struct.Client.html) connection to a TiKV cluster. +/// +/// Once resolved it will result in a connected [`Client`](struct.Client.html). +/// +/// ```rust,no_run +/// use tikv_client::{Config, raw::{Client, Connect}}; +/// use futures::Future; +/// +/// let connect: Connect = Client::new(&Config::default()); +/// let client: Client = connect.wait().unwrap(); +/// ``` +pub struct Connect { + config: Config, +} + +impl Connect { + fn new(config: Config) -> Self { + Connect { config } + } +} + +impl Future for Connect { + type Item = Client; + type Error = Error; + + fn poll(&mut self) -> Poll { + let config = &self.config; + let rpc = Arc::new(RpcClient::connect(config)?); + Ok(Async::Ready(Client { rpc })) + } +} + + +/// A [`ColumnFamily`](struct.ColumnFamily.html) is an optional parameter for [`raw::Client`](struct.Client.html) requests. +/// +/// TiKV uses RocksDB's `ColumnFamily` support. You can learn more about RocksDB's `ColumnFamily`s [on their wiki](https://github.com/facebook/rocksdb/wiki/Column-Families). +/// +/// By default in TiKV data is stored in three different `ColumnFamily` values, configurable in the TiKV server's configuration: +/// +/// * Default: Where real user data is stored. Set by `[rocksdb.defaultcf]`. +/// * Write: Where MVCC and index related data are stored. Set by `[rocksdb.writecf]`. +/// * Lock: Where lock information is stored. Set by `[rocksdb.lockcf]`. +/// +/// Not providing a call a `ColumnFamily` means it will use the default value of `default`. +/// +/// The best (and only) way to create a [`ColumnFamily`](struct.ColumnFamily.html) is via the `From` implementation: +/// +/// ```rust +/// # use tikv_client::raw::ColumnFamily; +/// +/// let cf = ColumnFamily::from("write"); +/// let cf = ColumnFamily::from(String::from("write")); +/// let cf = ColumnFamily::from(&String::from("write")); +/// ``` +/// +/// This is a *wrapper type* that implements `Deref` so it can be used like one transparently. +/// +/// **But, you should not need to worry about all this:** Many functions which accept a +/// `ColumnFamily` accept an `Into`, which means all of the above types can be passed +/// directly to those functions. #[derive(Default, Clone, Eq, PartialEq, Ord, PartialOrd, Hash, Debug)] pub struct ColumnFamily(String); @@ -38,6 +305,15 @@ impl ColumnFamily { } } +impl Deref for ColumnFamily { + type Target = String; + fn deref(&self) -> &Self::Target { + &self.0 + } +} + + + pub trait RequestInner: Sized { type Resp; @@ -65,6 +341,7 @@ where } } + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { self.cf = Some(cf.into()); self @@ -128,6 +405,10 @@ impl RequestInner for GetInner { } } +/// An unresolved [`Client::get`](struct.Client.html#method.get) request. +/// +/// Once resolved this request will result in the fetching of the value associated with the given +/// key. pub type Get = Request; pub struct BatchGetInner { @@ -148,8 +429,14 @@ impl BatchGetInner { } } +/// An unresolved [`Client::batch_get`](struct.Client.html#method.batch_get) request. +/// +/// Once resolved this request will result in the fetching of the values associated with the given +/// keys. pub type BatchGet = Request; + + pub struct PutInner { key: Key, value: Value, @@ -170,8 +457,15 @@ impl RequestInner for PutInner { } } +/// An unresolved [`Client::put`](struct.Client.html#method.put) request. +/// +/// Once resolved this request will result in the putting of the value associated with the given +/// key. pub type Put = Request; +/// An unresolved [`Client::batch_put`](struct.Client.html#method.batch_put) request. +/// +/// Once resolved this request will result in the setting of the value associated with the given key. pub struct BatchPutInner { pairs: Vec, } @@ -190,6 +484,10 @@ impl RequestInner for BatchPutInner { } } + +/// An unresolved [`Client::batch_put`](struct.Client.html#method.batch_put) request. +/// +/// Once resolved this request will result in the setting of the value associated with the given key. pub type BatchPut = Request; pub struct DeleteInner { @@ -210,6 +508,9 @@ impl RequestInner for DeleteInner { } } +/// An unresolved [`Client::delete`](struct.Client.html#method.delete) request. +/// +/// Once resolved this request will result in the deletion of the given key. pub type Delete = Request; pub struct BatchDeleteInner { @@ -233,13 +534,13 @@ impl RequestInner for BatchDeleteInner { pub type BatchDelete = Request; pub struct ScanInner { - range: Result<(Key, Option)>, + range: (Bound, Bound), limit: u32, key_only: bool, } impl ScanInner { - fn new(range: Result<(Key, Option)>, limit: u32) -> Self { + fn new(range: (Bound, Bound), limit: u32) -> Self { ScanInner { range, limit, @@ -252,10 +553,11 @@ impl RequestInner for ScanInner { type Resp = Vec; fn execute(self, client: Arc, cf: Option) -> KvFuture { - match self.range { - Ok(range) => Box::new(client.raw_scan(range, self.limit, self.key_only, cf)), - Err(e) => Box::new(future::err(e)), - } + let keys = match self.range.into_keys() { + Err(e) => return Box::new(future::err(e)), + Ok(v) => v, + }; + Box::new(client.raw_scan(keys, self.limit, self.key_only, cf)) } } @@ -270,6 +572,7 @@ impl Scan { } } + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { self.request = self.request.cf(cf); self @@ -339,6 +642,7 @@ impl BatchScan { } } + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { self.request = self.request.cf(cf); self @@ -362,6 +666,8 @@ impl Future for BatchScan { } } +pub type DeleteRange = Request; + pub struct DeleteRangeInner { range: Result<(Key, Option)>, } @@ -382,119 +688,3 @@ impl RequestInner for DeleteRangeInner { } } } - -pub type DeleteRange = Request; - -pub struct Connect { - config: Config, -} - -impl Connect { - fn new(config: Config) -> Self { - Connect { config } - } -} - -impl Future for Connect { - type Item = Client; - type Error = Error; - - fn poll(&mut self) -> Poll { - let config = &self.config; - let rpc = Arc::new(RpcClient::connect(config)?); - Ok(Async::Ready(Client { rpc })) - } -} - -pub struct Client { - rpc: Arc, -} - -impl Client { - #![allow(clippy::new_ret_no_self)] - pub fn new(config: &Config) -> Connect { - Connect::new(config.clone()) - } - - #[inline] - fn rpc(&self) -> Arc { - Arc::clone(&self.rpc) - } - - pub fn get(&self, key: impl AsRef) -> Get { - Get::new(self.rpc(), GetInner::new(key.as_ref().clone())) - } - - pub fn batch_get(&self, keys: impl AsRef<[Key]>) -> BatchGet { - BatchGet::new(self.rpc(), BatchGetInner::new(keys.as_ref().to_vec())) - } - - pub fn put(&self, key: impl Into, value: impl Into) -> Put { - Put::new(self.rpc(), PutInner::new(key.into(), value.into())) - } - - pub fn batch_put(&self, pairs: impl IntoIterator>) -> BatchPut { - BatchPut::new( - self.rpc(), - BatchPutInner::new(pairs.into_iter().map(Into::into).collect()), - ) - } - - pub fn delete(&self, key: impl AsRef) -> Delete { - Delete::new(self.rpc(), DeleteInner::new(key.as_ref().clone())) - } - - pub fn batch_delete(&self, keys: impl AsRef<[Key]>) -> BatchDelete { - BatchDelete::new(self.rpc(), BatchDeleteInner::new(keys.as_ref().to_vec())) - } - - pub fn scan(&self, range: impl RangeBounds, limit: u32) -> Scan { - Scan::new( - self.rpc(), - ScanInner::new(Self::range_bounds(&range), limit), - ) - } - - pub fn batch_scan(&self, ranges: Ranges, each_limit: u32) -> BatchScan - where - Ranges: AsRef<[Bounds]>, - Bounds: RangeBounds, - { - BatchScan::new( - self.rpc(), - BatchScanInner::new( - ranges.as_ref().iter().map(Self::range_bounds).collect(), - each_limit, - ), - ) - } - - pub fn delete_range(&self, range: impl RangeBounds) -> DeleteRange { - DeleteRange::new( - self.rpc(), - DeleteRangeInner::new(Self::range_bounds(&range)), - ) - } - - fn bound(bound: Bound<&Key>) -> Option { - match bound { - Bound::Included(k) => Some(k.clone()), - Bound::Excluded(k) => Some(k.clone()), - Bound::Unbounded => None, - } - } - - fn range_bounds(range: &impl RangeBounds) -> Result<(Key, Option)> { - if let Bound::Included(_) = range.end_bound() { - return Err(Error::InvalidKeyRange); - } - if let Bound::Excluded(_) = range.start_bound() { - return Err(Error::InvalidKeyRange); - } - - Ok(( - Self::bound(range.start_bound()).unwrap_or_else(Key::default), - Self::bound(range.end_bound()), - )) - } -} diff --git a/src/rpc/client.rs b/src/rpc/client.rs index 3680a0dc..a33ce31d 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -486,11 +486,11 @@ impl RegionContext { } fn start_key(&self) -> Key { - self.region.start_key().into() + self.region.start_key().to_vec().into() } fn end_key(&self) -> Key { - self.region.end_key().into() + self.region.end_key().to_vec().into() } fn range(&self) -> (Key, Key) { diff --git a/src/rpc/tikv/client.rs b/src/rpc/tikv/client.rs index 0dfc591e..e7d85b3c 100644 --- a/src/rpc/tikv/client.rs +++ b/src/rpc/tikv/client.rs @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::{fmt, sync::Arc, time::Duration}; +use std::{fmt, sync::Arc, time::Duration, ops::Bound}; use futures::Future; use grpcio::{CallOption, Environment}; @@ -662,7 +662,7 @@ impl KvClient { #[inline] fn convert_from_grpc_pair(mut pair: kvrpcpb::KvPair) -> KvPair { - KvPair::new(pair.take_key().into(), pair.take_value().into()) + KvPair::new(Key::from(pair.take_key()), Value::from(pair.take_value())) } #[inline] diff --git a/src/transaction.rs b/src/transaction.rs index c43f58f3..1d6e425c 100644 --- a/src/transaction.rs +++ b/src/transaction.rs @@ -11,11 +11,118 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::RangeBounds; +/*! Transactional related functionality. -use futures::{Future, Poll, Stream}; +Using the [`transaction::Client`](struct.Client.html) you can utilize TiKV's transactional interface. +This interface offers SQL-like transactions on top of the raw interface. + +**Warning:** It is not advisible to use the both raw and transactional functionality in the same keyspace. + */ use crate::{Config, Error, Key, KvPair, Value}; +use futures::{Future, Poll, Stream}; +use std::ops::RangeBounds; + +/// The TiKV transactional [`Client`](struct.Client.html) is used to issue requests to the TiKV server and PD cluster. +pub struct Client; + +impl Client { + /// Create a new [`Client`](struct.Client.html) once the [`Connect`](struct.Connect.html) resolves. + /// + /// ```rust,no_run + /// use tikv_client::{Config, transaction::Client}; + /// use futures::Future; + /// let connect = Client::new(&Config::default()); + /// let client = connect.wait(); + /// ``` + #[cfg_attr(feature = "cargo-clippy", allow(clippy::new_ret_no_self))] + pub fn new(config: &Config) -> Connect { + Connect::new(config.clone()) + } + + /// Create a new [`Transaction`](struct.Transaction.html) using the timestamp from [`current_timestamp`](struct.Client.html#method.current_timestamp). + /// + /// Using the transaction you can issue commands like [`get`](struct.Transaction.html#method.get) or [`set`](file:///home/hoverbear/git/client-rust/target/doc/tikv_client/transaction/struct.Transaction.html#method.set). + /// + /// ```rust,no_run + /// use tikv_client::{Config, transaction::Client}; + /// use futures::Future; + /// let connect = Client::new(&Config::default()); + /// let client = connect.wait().unwrap(); + /// let transaction = client.begin(); + /// // ... Issue some commands. + /// let commit = transaction.commit(); + /// let result: () = commit.wait().unwrap(); + /// ``` + pub fn begin(&self) -> Transaction { + unimplemented!() + } + + /// Create a new [`Transaction`](struct.Transaction.html) at the provded timestamp. + /// + /// ```rust,no_run + /// use tikv_client::{Config, transaction::Client}; + /// use futures::Future; + /// let connect = Client::new(&Config::default()); + /// let client = connect.wait().unwrap(); + /// let timestamp = client.current_timestamp(); + /// let transaction = client.begin_with_timestamp(timestamp); + /// // ... Issue some commands. + /// let commit = transaction.commit(); + /// let result: () = commit.wait().unwrap(); + /// ``` + pub fn begin_with_timestamp(&self, _timestamp: Timestamp) -> Transaction { + unimplemented!() + } + + /// Get a [`Snapshot`](struct.Snapshot.html) using the timestamp from [`current_timestamp`](struct.Client.html#method.current_timestamp). + /// + /// ```rust,no_run + /// use tikv_client::{Config, transaction::Client}; + /// use futures::Future; + /// let connect = Client::new(&Config::default()); + /// let client = connect.wait().unwrap(); + /// let snapshot = client.snapshot(); + /// // ... Issue some commands. + /// ``` + pub fn snapshot(&self) -> Snapshot { + unimplemented!() + } + + /// Retrieve the current [`Timestamp`](struct.Timestamp.html). + /// + /// ```rust,no_run + /// use tikv_client::{Config, transaction::Client}; + /// use futures::Future; + /// let connect = Client::new(&Config::default()); + /// let client = connect.wait().unwrap(); + /// let timestamp = client.current_timestamp(); + /// ``` + pub fn current_timestamp(&self) -> Timestamp { + unimplemented!() + } +} + +/// An unresolved [`Client`](struct.Client.html) connection to a TiKV cluster. +/// +/// Once resolved it will result in a connected [`Client`](struct.Client.html). +/// +/// ```rust,no_run +/// use tikv_client::{Config, transaction::{Client, Connect}}; +/// use futures::Future; +/// +/// let connect: Connect = Client::new(&Config::default()); +/// let client: Client = connect.wait().unwrap(); +/// ``` +pub struct Connect { + config: Config, +} + +impl Connect { + fn new(config: Config) -> Self { + Connect { config } + } +} pub enum Mutation { Put(Key, Value), @@ -29,12 +136,23 @@ pub struct TxnInfo { pub status: u64, } +impl Future for Connect { + type Item = Client; + type Error = Error; + + fn poll(&mut self) -> Poll { + let _config = &self.config; + unimplemented!() + } +} + +/// A logical timestamp produced by PD. #[derive(Copy, Clone)] pub struct Timestamp(u64); -impl Into for u64 { - fn into(self) -> Timestamp { - Timestamp(self) +impl From for Timestamp { + fn from(v: u64) -> Self { + Timestamp(v) } } @@ -52,6 +170,246 @@ impl Timestamp { } } +/// 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. +pub struct Transaction { + snapshot: Snapshot, +} + +impl Transaction { + /// Create a new transaction operating on the given snapshot. + /// + /// ```rust,no_run + /// use tikv_client::{Config, transaction::Client}; + /// use futures::Future; + /// let connect = Client::new(&Config::default()); + /// let client = connect.wait().unwrap(); + /// let txn = client.begin(); + /// ``` + pub fn new(snapshot: Snapshot) -> Self { + Self { snapshot } + } + + /// Commit the actions of the transaction. + /// + /// Once committed, it is no longer possible to `rollback` the actions in the transaction. + /// + /// ```rust,no_run + /// # use tikv_client::{Config, transaction::Client}; + /// # use futures::Future; + /// # let connect = Client::new(&Config::default()); + /// # let connected_client = connect.wait().unwrap(); + /// let txn = connected_client.begin(); + /// // ... Do some actions. + /// let req = txn.commit(); + /// let result: () = req.wait().unwrap(); + /// ``` + pub fn commit(self) -> Commit { + Commit::new(self) + } + + /// Rollback the actions of the transaction. + /// + /// ```rust,no_run + /// # use tikv_client::{Config, transaction::Client}; + /// # use futures::Future; + /// # let connect = Client::new(&Config::default()); + /// # let connected_client = connect.wait().unwrap(); + /// let txn = connected_client.begin(); + /// // ... Do some actions. + /// let req = txn.rollback(); + /// let result: () = req.wait().unwrap(); + /// ``` + pub fn rollback(self) -> Rollback { + Rollback::new(self) + } + + /// Lock the given keys. + /// + /// ```rust,no_run + /// # use tikv_client::{Config, transaction::Client}; + /// # use futures::Future; + /// # let connect = Client::new(&Config::default()); + /// # let connected_client = connect.wait().unwrap(); + /// let mut txn = connected_client.begin(); + /// // ... Do some actions. + /// let req = txn.lock_keys(vec!["TiKV", "Rust"]); + /// let result: () = req.wait().unwrap(); + /// ``` + pub fn lock_keys(&mut self, keys: impl IntoIterator>) -> LockKeys { + LockKeys::new(keys.into_iter().map(|v| v.into()).collect()) + } + + pub fn is_readonly(&self) -> bool { + unimplemented!() + } + + /// Returns the timestamp which the transaction started at. + /// + /// ```rust,no_run + /// # use tikv_client::{Config, transaction::{Client, Timestamp}}; + /// # use futures::Future; + /// # let connect = Client::new(&Config::default()); + /// # let connected_client = connect.wait().unwrap(); + /// let txn = connected_client.begin(); + /// // ... Do some actions. + /// let ts: Timestamp = txn.start_ts(); + /// ``` + pub fn start_ts(&self) -> Timestamp { + unimplemented!() + } + + /// Get the `Snapshot` the transaction is operating on. + /// + /// ```rust,no_run + /// # use tikv_client::{Config, transaction::{Client, Snapshot}}; + /// # use futures::Future; + /// # let connect = Client::new(&Config::default()); + /// # let connected_client = connect.wait().unwrap(); + /// let txn = connected_client.begin(); + /// // ... Do some actions. + /// let snap: Snapshot = txn.snapshot(); + /// ``` + pub fn snapshot(&self) -> Snapshot { + unimplemented!() + } + + /// Set the isolation level of the transaction. + /// + /// ```rust,no_run + /// # use tikv_client::{Config, transaction::{Client, IsolationLevel}}; + /// # use futures::Future; + /// # let connect = Client::new(&Config::default()); + /// # let connected_client = connect.wait().unwrap(); + /// let mut txn = connected_client.begin(); + /// txn.set_isolation_level(IsolationLevel::SnapshotIsolation); + /// ``` + pub fn set_isolation_level(&mut self, _level: IsolationLevel) { + unimplemented!() + } + + /// Create a new [`Get`](struct.Get.html) request. + /// + /// Once resolved this request will result in the fetching of the value associated with the + /// given key. + /// + /// ```rust,no_run + /// # use tikv_client::{Value, Config, transaction::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let mut txn = connected_client.begin(); + /// let key = "TiKV"; + /// let req = txn.get(key); + /// let result: Value = req.wait().unwrap(); + /// // Finish the transaction... + /// txn.commit().wait().unwrap(); + /// ``` + pub fn get(&self, key: impl Into) -> Get { + self.snapshot.get(key.into()) + } + + /// Create a new [`BatchGet`](struct.BatchGet.html) request. + /// + /// Once resolved this request will result in the fetching of the values associated with the + /// given keys. + /// + /// ```rust,no_run + /// # use tikv_client::{KvPair, Config, transaction::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let mut txn = connected_client.begin(); + /// let keys = vec!["TiKV", "TiDB"]; + /// let req = txn.batch_get(keys); + /// let result: Vec = req.wait().unwrap(); + /// // Finish the transaction... + /// txn.commit().wait().unwrap(); + /// ``` + pub fn batch_get(&self, keys: impl IntoIterator>) -> BatchGet { + self.snapshot.batch_get(keys) + } + + pub fn scan(&self, range: impl RangeBounds) -> Scanner { + self.snapshot.scan(range) + } + + pub fn scan_reverse(&self, range: impl RangeBounds) -> Scanner { + self.snapshot.scan_reverse(range) + } + + /// Create a new [`Set`](struct.Set.html) request. + /// + /// Once resolved this request will result in the setting of the value associated with the given key. + /// + /// ```rust,no_run + /// # use tikv_client::{Key, Value, Config, transaction::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let mut txn = connected_client.begin(); + /// let key = "TiKV"; + /// let val = "TiKV"; + /// let req = txn.set(key, val); + /// let result: () = req.wait().unwrap(); + /// // Finish the transaction... + /// txn.commit().wait().unwrap(); + /// ``` + pub fn set(&mut self, key: impl Into, value: impl Into) -> Set { + Set::new(key.into(), value.into()) + } + + /// Create a new [`Delete`](struct.Delete.html) request. + /// + /// Once resolved this request will result in the deletion of the given key. + /// + /// ```rust,no_run + /// # use tikv_client::{Key, Config, transaction::Client}; + /// # use futures::Future; + /// # let connecting_client = Client::new(&Config::new(vec!["192.168.0.100", "192.168.0.101"])); + /// # let connected_client = connecting_client.wait().unwrap(); + /// let mut txn = connected_client.begin(); + /// let key = "TiKV"; + /// let req = txn.delete(key); + /// let result: () = req.wait().unwrap(); + /// // Finish the transaction... + /// txn.commit().wait().unwrap(); + /// ``` + pub fn delete(&mut self, key: impl Into) -> Delete { + Delete::new(key.into()) + } +} + +/// A snapshot of dataset at a particular point in time. +pub struct Snapshot; + +impl Snapshot { + pub fn get(&self, key: impl Into) -> Get { + Get::new(key.into()) + } + + pub fn batch_get(&self, keys: impl IntoIterator>) -> BatchGet { + BatchGet::new(keys.into_iter().map(|v| v.into()).collect()) + } + + pub fn scan(&self, range: impl RangeBounds) -> Scanner { + drop(range); + unimplemented!() + } + + pub fn scan_reverse(&self, range: impl RangeBounds) -> Scanner { + drop(range); + unimplemented!() + } +} + +/// An unresolved [`Transaction::scan`](struct.Transaction.html#method.scan) request. +/// +/// Once resolved this request will result in a scanner over the given keys. pub struct Scanner; impl Stream for Scanner { @@ -63,12 +421,41 @@ impl Stream for Scanner { } } +/// The isolation level guarantees provided by the transaction. #[derive(Copy, Clone, Eq, PartialEq, Debug)] pub enum IsolationLevel { + /// Consistent reads and conflict free writes. + /// + /// Snapshot isolation guarantees: + /// * All reads will see the last committed value of the data at the snapshot timestamp. + /// * The transaction will only successfully commit if no updates to the data have created a + /// conflict with concurrent updates made sine the snapshot. + /// + /// Using this level means: + /// * Lost updates don't occur. + /// * Dirty reads don't occur. + /// * Non-repeatable reads don't occur. + /// * Phantom reads don't occur. SnapshotIsolation, + /// Reads may not be consistent, but writes are conflict free. + /// + /// Read committed guarantees: + /// * All reads are committed at the moment it is read. + /// not repeatable. + /// * Write locks are only released at the end of the transaction. + /// + /// Using this level means: + /// * Lost updates don't occur. + /// * Dirty reads don't occur. + /// * Non-repeatable reads may occur. + /// * Phantom reads may occur. ReadCommitted, } +/// An unresolved [`Transaction::get`](struct.Transaction.html#method.get) request. +/// +/// Once resolved this request will result in the fetching of the value associated with the given +/// key. pub struct Get { key: Key, } @@ -89,6 +476,10 @@ impl Future for Get { } } +/// An unresolved [`Transaction::batch_get`](struct.Transaction.html#method.batch_get) request. +/// +/// Once resolved this request will result in the fetching of the values associated with the given +/// keys. pub struct BatchGet { keys: Vec, } @@ -100,7 +491,7 @@ impl BatchGet { } impl Future for BatchGet { - type Item = Value; + type Item = Vec; type Error = Error; fn poll(&mut self) -> Poll { @@ -109,6 +500,9 @@ impl Future for BatchGet { } } +/// An unresolved [`Transaction::commit`](struct.Transaction.html#method.commit) request. +/// +/// Once resolved this request will result in the committing of the transaction. pub struct Commit { txn: Transaction, } @@ -129,6 +523,9 @@ impl Future for Commit { } } +/// An unresolved [`Transaction::rollback`](struct.Transaction.html#method.rollback) request. +/// +/// Once resolved this request will result in the rolling back of the transaction. pub struct Rollback { txn: Transaction, } @@ -149,6 +546,9 @@ impl Future for Rollback { } } +/// An unresolved [`Transaction::lock_keys`](struct.Transaction.html#method.lock_keys) request. +/// +/// Once resolved this request will result in the locking of the given keys. pub struct LockKeys { keys: Vec, } @@ -169,6 +569,10 @@ impl Future for LockKeys { } } +/// An unresolved [`Transaction::set`](struct.Transaction.html#method.set) request. +/// +/// Once resolved this request will result in the setting of the value associated with the given +/// key. pub struct Set { key: Key, value: Value, @@ -191,6 +595,9 @@ impl Future for Set { } } +/// An unresolved [`Transaction::delete`](struct.Transaction.html#method.delete) request. +/// +/// Once resolved this request will result in the deletion of the given key. pub struct Delete { key: Key, } @@ -209,120 +616,4 @@ impl Future for Delete { let _key = &self.key; unimplemented!() } -} - -pub struct Transaction { - snapshot: Snapshot, -} - -impl Transaction { - pub fn commit(self) -> Commit { - Commit::new(self) - } - - pub fn rollback(self) -> Rollback { - Rollback::new(self) - } - - pub fn lock_keys(&mut self, keys: impl AsRef<[Key]>) -> LockKeys { - LockKeys::new(keys.as_ref().to_vec().clone()) - } - - pub fn is_readonly(&self) -> bool { - unimplemented!() - } - - pub fn start_ts(&self) -> Timestamp { - unimplemented!() - } - - pub fn snapshot(&self) -> Snapshot { - unimplemented!() - } - - pub fn set_isolation_level(&mut self, _level: IsolationLevel) { - unimplemented!() - } - - pub fn get(&self, key: impl AsRef) -> Get { - self.snapshot.get(key) - } - - pub fn batch_get(&self, keys: impl AsRef<[Key]>) -> BatchGet { - self.snapshot.batch_get(keys) - } - - pub fn scan(&self, range: impl RangeBounds) -> Scanner { - self.snapshot.scan(range) - } - - pub fn set(&mut self, key: impl Into, value: impl Into) -> Set { - Set::new(key.into(), value.into()) - } - - pub fn delete(&mut self, key: impl AsRef) -> Delete { - Delete::new(key.as_ref().clone()) - } -} - -pub struct Snapshot; - -impl Snapshot { - pub fn get(&self, key: impl AsRef) -> Get { - Get::new(key.as_ref().clone()) - } - - pub fn batch_get(&self, keys: impl AsRef<[Key]>) -> BatchGet { - BatchGet::new(keys.as_ref().to_vec().clone()) - } - - pub fn scan(&self, range: impl RangeBounds) -> Scanner { - drop(range); - unimplemented!() - } -} - -pub struct Connect { - config: Config, -} - -impl Connect { - fn new(config: Config) -> Self { - Connect { config } - } -} - -impl Future for Connect { - type Item = Client; - type Error = Error; - - fn poll(&mut self) -> Poll { - let _config = &self.config; - unimplemented!() - } -} - -pub struct Client {} - -impl Client { - #![allow(clippy::new_ret_no_self)] - pub fn new(config: &Config) -> Connect { - Connect::new(config.clone()) - } - - pub fn begin(&self) -> Transaction { - unimplemented!() - } - - pub fn begin_with_timestamp(&self, _timestamp: Timestamp) -> Transaction { - unimplemented!() - } - - pub fn snapshot(&self) -> Snapshot { - unimplemented!() - } - - pub fn current_timestamp(&self) -> Timestamp { - unimplemented!() - } -} +} \ No newline at end of file diff --git a/tests/raw.rs b/tests/raw.rs index cfe17177..46e85b54 100644 --- a/tests/raw.rs +++ b/tests/raw.rs @@ -28,7 +28,7 @@ fn wipe_all(client: &Client) { let test_key_start = generate_key(0); let test_key_end = generate_key(NUM_TEST_KEYS as i32 - 1); client - .delete_range(&test_key_start..&test_key_end) + .delete_range(test_key_start..test_key_end) .wait() .expect("Could not delete test keys"); } @@ -46,7 +46,7 @@ fn test_empty(client: &Client) { let test_key_end = generate_key(NUM_TEST_KEYS as i32 - 1); assert!(client - .scan(&test_key_start..&test_key_end, NUM_TEST_KEYS) + .scan(test_key_start..test_key_end, NUM_TEST_KEYS) .wait() .expect("Could not scan") .is_empty()); @@ -56,14 +56,15 @@ fn test_existence(client: &Client, existing_pairs: Vec, not_existing_key let test_key_start = generate_key(0); let test_key_end = generate_key(NUM_TEST_KEYS as i32 - 1); - for pair in existing_pairs.iter() { + for pair in existing_pairs.clone().into_iter() { + let (key, value) = pair.into_inner(); assert_eq!( - client.get(pair.key()).wait().expect("Could not get value"), - pair.value().clone(), + client.get(key).wait().expect("Could not get value"), + value.clone(), ); } - for key in not_existing_keys.iter() { + for key in not_existing_keys.clone().into_iter() { let r = client.get(key).wait(); assert!(r.is_err()); } @@ -81,7 +82,7 @@ fn test_existence(client: &Client, existing_pairs: Vec, not_existing_key assert_eq!( client - .batch_get(&all_keys) + .batch_get(all_keys) .wait() .expect("Could not get value in batch"), existing_pairs, @@ -89,7 +90,7 @@ fn test_existence(client: &Client, existing_pairs: Vec, not_existing_key assert_eq!( client - .batch_get(¬_existing_keys) + .batch_get(not_existing_keys) .wait() .expect("Could not get value in batch"), Vec::new(), @@ -97,7 +98,7 @@ fn test_existence(client: &Client, existing_pairs: Vec, not_existing_key assert_eq!( client - .scan(&test_key_start..&test_key_end, NUM_TEST_KEYS) + .scan(test_key_start.clone()..test_key_end.clone(), NUM_TEST_KEYS) .wait() .expect("Could not scan"), existing_pairs, @@ -105,7 +106,7 @@ fn test_existence(client: &Client, existing_pairs: Vec, not_existing_key assert_eq!( client - .scan(&test_key_start..&test_key_end, NUM_TEST_KEYS) + .scan(test_key_start.clone()..test_key_end.clone(), NUM_TEST_KEYS) .key_only() .wait() .expect("Could not scan"), @@ -129,7 +130,7 @@ fn basic_raw_test() { vec![generate_key(1), generate_key(2)], ); - assert!(client.delete(&generate_key(0)).wait().is_ok()); + assert!(client.delete(generate_key(0)).wait().is_ok()); test_existence( &client, Vec::new(), @@ -147,7 +148,7 @@ fn basic_raw_test() { ); let keys: Vec = vec![generate_key(8), generate_key(9)]; - assert!(client.batch_delete(&keys).wait().is_ok()); + assert!(client.batch_delete(keys).wait().is_ok()); let mut pairs = pairs; pairs.truncate(8); test_existence( From f354590056c6bb9502ffda2e7b82f8ec521137c7 Mon Sep 17 00:00:00 2001 From: Ana Hobden Date: Fri, 11 Jan 2019 11:26:49 -0800 Subject: [PATCH 03/20] Reduce pub surface Signed-off-by: Ana Hobden --- src/raw.rs | 96 +++++++++++++++++++++++++++++++++--------------------- 1 file changed, 59 insertions(+), 37 deletions(-) diff --git a/src/raw.rs b/src/raw.rs index 13ed10cf..6800dd47 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -387,7 +387,15 @@ where } } -pub struct GetInner { +/// An unresolved [`Client::get`](struct.Client.html#method.get) request. +/// +/// Once resolved this request will result in the fetching of the value associated with the given +/// key. +pub struct Get { + request: Request, +} + +pub(crate) struct GetInner { key: Key, } @@ -405,13 +413,15 @@ impl RequestInner for GetInner { } } -/// An unresolved [`Client::get`](struct.Client.html#method.get) request. -/// -/// Once resolved this request will result in the fetching of the value associated with the given -/// key. -pub type Get = Request; +/// An unresolved [`Client::batch_get`](struct.Client.html#method.batch_get) request. +/// +/// Once resolved this request will result in the fetching of the values associated with the given +/// keys. +pub struct BatchGet { + request: Request, +} -pub struct BatchGetInner { +pub(crate) struct BatchGetInner { keys: Vec, } @@ -429,15 +439,15 @@ impl BatchGetInner { } } -/// An unresolved [`Client::batch_get`](struct.Client.html#method.batch_get) request. -/// -/// Once resolved this request will result in the fetching of the values associated with the given -/// keys. -pub type BatchGet = Request; - - +/// An unresolved [`Client::put`](struct.Client.html#method.put) request. +/// +/// Once resolved this request will result in the putting of the value associated with the given +/// key. +pub struct Put { + request: Request, +} -pub struct PutInner { +pub(crate) struct PutInner { key: Key, value: Value, } @@ -457,16 +467,14 @@ impl RequestInner for PutInner { } } -/// An unresolved [`Client::put`](struct.Client.html#method.put) request. -/// -/// Once resolved this request will result in the putting of the value associated with the given -/// key. -pub type Put = Request; - /// An unresolved [`Client::batch_put`](struct.Client.html#method.batch_put) request. /// /// Once resolved this request will result in the setting of the value associated with the given key. -pub struct BatchPutInner { +pub struct BatchPut { + request: Request, +} + +pub(crate) struct BatchPutInner { pairs: Vec, } @@ -484,13 +492,14 @@ impl RequestInner for BatchPutInner { } } +/// An unresolved [`Client::delete`](struct.Client.html#method.delete) request. +/// +/// Once resolved this request will result in the deletion of the given key. +pub struct Delete { + request: Request, +} -/// An unresolved [`Client::batch_put`](struct.Client.html#method.batch_put) request. -/// -/// Once resolved this request will result in the setting of the value associated with the given key. -pub type BatchPut = Request; - -pub struct DeleteInner { +pub(crate) struct DeleteInner { key: Key, } @@ -508,12 +517,14 @@ impl RequestInner for DeleteInner { } } -/// An unresolved [`Client::delete`](struct.Client.html#method.delete) request. +/// An unresolved [`Client::batch_delete`](struct.Client.html#method.batch_delete) request. /// -/// Once resolved this request will result in the deletion of the given key. -pub type Delete = Request; +/// Once resolved this request will result in the deletion of the given keys. +pub struct BatchDelete { + request: Request, +} -pub struct BatchDeleteInner { +pub(crate) struct BatchDeleteInner { keys: Vec, } @@ -531,9 +542,8 @@ impl RequestInner for BatchDeleteInner { } } -pub type BatchDelete = Request; -pub struct ScanInner { +pub(crate) struct ScanInner { range: (Bound, Bound), limit: u32, key_only: bool, @@ -561,6 +571,9 @@ impl RequestInner for ScanInner { } } +/// An unresolved [`Client::scan`](struct.Client.html#method.scan) request. +/// +/// Once resolved this request will result in a scanner over the given range. pub struct Scan { request: Request, } @@ -596,7 +609,7 @@ impl Future for Scan { } } -pub struct BatchScanInner { +pub(crate) struct BatchScanInner { ranges: Vec)>>, each_limit: u32, key_only: bool, @@ -631,6 +644,9 @@ impl RequestInner for BatchScanInner { } } +/// An unresolved [`Client::batch_scan`](struct.Client.html#method.batch_scan) request. +/// +/// Once resolved this request will result in a scanner over the given ranges. pub struct BatchScan { request: Request, } @@ -666,9 +682,15 @@ impl Future for BatchScan { } } -pub type DeleteRange = Request; +/// An unresolved [`Client::delete_range`](struct.Client.html#method.delete_range) request. +/// +/// Once resolved this request will result in the deletion of the values in the given +/// range. +pub struct DeleteRange { + request: Request, +} -pub struct DeleteRangeInner { +pub(crate) struct DeleteRangeInner { range: Result<(Key, Option)>, } From 4048f749facaac54f0e3db25c9de76cfbddd4d5b Mon Sep 17 00:00:00 2001 From: Ana Hobden Date: Fri, 11 Jan 2019 11:27:24 -0800 Subject: [PATCH 04/20] fmt/lint Signed-off-by: Ana Hobden --- examples/raw.rs | 21 +++-- src/errors.rs | 2 +- src/lib.rs | 78 +++++++++-------- src/raw.rs | 184 +++++++++++++++++++++++++++++++++++------ src/rpc/tikv/client.rs | 2 +- src/transaction.rs | 8 +- 6 files changed, 221 insertions(+), 74 deletions(-) diff --git a/examples/raw.rs b/examples/raw.rs index efa273ed..5c166c4b 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -13,7 +13,7 @@ use futures::future::Future; use std::path::PathBuf; -use tikv_client::{Result, Config, raw::Client, Key, Value}; +use tikv_client::{raw::Client, Config, Key, Result, Value}; const KEY: &str = "TiKV"; const VALUE: &str = "Rust"; @@ -25,7 +25,8 @@ fn main() -> Result<()> { let config = Config::new(vec![ "192.168.0.101:3379", // Avoid a single point of failure, "192.168.0.100:3379", // use at least two PD endpoints. - ]).with_security( + ]) + .with_security( PathBuf::from("/path/to/ca.pem"), PathBuf::from("/path/to/client.pem"), PathBuf::from("/path/to/client-key.pem"), @@ -36,7 +37,7 @@ fn main() -> Result<()> { let unconnnected_client = Client::new(&config); let client = unconnnected_client.wait()?; - // Requests are created from the connected client. These calls return structures which + // 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 // place. // @@ -45,7 +46,7 @@ fn main() -> Result<()> { let put_result: () = put_request.wait()?; // Returns a `tikv_client::Error` on failure. println!("Put key \"{}\", value \"{}\".", KEY, VALUE); - // + // // Unlike a standard Rust HashMap all calls take owned values. This is because under the hood // protobufs must take ownership of the data. If we only took a borrow we'd need to internally // clone it. This is against Rust API guidelines, so you must manage this yourself. // @@ -61,13 +62,15 @@ fn main() -> Result<()> { // You can also set the `ColumnFamily` used by the request. // This is *advanced usage* and should have some special considerations. - let req = client.delete(key.clone()) + let req = client + .delete(key.clone()) .cf(CUSTOM_CF) .wait() .expect("Could not delete value"); println!("Key: {:?} deleted", key); - client.get(key) + client + .get(key) .cf("test_cf") .wait() .expect_err("Get returned value for not existing key"); @@ -83,14 +86,16 @@ fn main() -> Result<()> { let start: Key = b"k1".to_vec().into(); let end: Key = b"k2".to_vec().into(); - client.scan(start.clone()..end.clone(), 10) + client + .scan(start.clone()..end.clone(), 10) .cf("test_cf") .key_only() .wait() .expect("Could not scan"); let ranges = vec![start.clone()..end.clone(), start.clone()..end.clone()]; - client.batch_scan(ranges, 10) + client + .batch_scan(ranges, 10) .cf("test_cf") .key_only() .wait() diff --git a/src/errors.rs b/src/errors.rs index f9c93275..deee4291 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -15,7 +15,7 @@ use grpcio; use quick_error::quick_error; use std::{error, result}; -quick_error!{ +quick_error! { /// An error originating from the TiKV client or dependencies. /// /// This client currently uses [`quick_error`](https://docs.rs/quick-error/1.2.2/quick_error/) diff --git a/src/lib.rs b/src/lib.rs index ccaabf40..4ca64c78 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -13,13 +13,16 @@ #![recursion_limit = "128"] #![type_length_limit = "2097152"] +use futures::Future; use serde_derive::*; use std::{ - ops::{Deref, DerefMut, Bound, Range, RangeFrom, RangeFull, RangeInclusive, RangeTo, RangeToInclusive}, + ops::{ + Bound, Deref, DerefMut, Range, RangeFrom, RangeFull, RangeInclusive, RangeTo, + RangeToInclusive, + }, path::PathBuf, time::Duration, }; -use futures::Future; mod errors; pub mod raw; @@ -32,38 +35,38 @@ pub use crate::errors::Error; pub use crate::errors::Result; /// The key part of a key/value pair. -/// +/// /// In TiKV, keys are an ordered sequence of bytes. This has an advantage over choosing `String` as valid `UTF-8` is not required. This means that the user is permitted to store any data they wish, /// as long as it can be represented by bytes. (Which is to say, pretty much anything!) -/// +/// /// This is a *wrapper type* that implements `Deref>` so it can be used like one transparently. -/// +/// /// This type also implements `From` for many types. With one exception, these are all done without reallocation. Using a `&'static str`, like many examples do for simplicity, has an internal -/// allocation cost. -/// +/// allocation cost. +/// /// This type wraps around an owned value, so it should be treated it like `String` or `Vec` /// over a `&str` or `&[u8]`. -/// +/// /// ```rust /// use tikv_client::Key; -/// +/// /// let static_str: &'static str = "TiKV"; /// let from_static_str = Key::from(static_str); -/// +/// /// let string: String = String::from(static_str); /// let from_string = Key::from(string); /// assert_eq!(from_static_str, from_string); -/// +/// /// let vec: Vec = static_str.as_bytes().to_vec(); /// let from_vec = Key::from(vec); /// assert_eq!(from_static_str, from_vec); -/// +/// /// let bytes = static_str.as_bytes().to_vec(); /// let from_bytes = Key::from(bytes); /// assert_eq!(from_static_str, from_bytes); /// ``` -/// -/// **But, you should not need to worry about all this:** Many functions which accept a `Key` +/// +/// **But, you should not need to worry about all this:** Many functions which accept a `Key` /// accept an `Into`, which means all of the above types can be passed directly to those /// functions. #[derive(Default, Clone, Eq, PartialEq, Ord, PartialOrd, Hash, Debug)] @@ -114,13 +117,13 @@ impl AsRef<[u8]> for Key { impl Deref for Key { type Target = [u8]; - fn deref<'a>(&'a self) -> &'a Self::Target { + fn deref(&self) -> &Self::Target { &self.0 } } impl DerefMut for Key { - fn deref_mut<'a>(&'a mut self) -> &'a mut [u8] { + fn deref_mut(&mut self) -> &mut [u8] { &mut self.0 } } @@ -196,13 +199,13 @@ impl From<&'static str> for Value { impl Deref for Value { type Target = [u8]; - fn deref<'a>(&'a self) -> &'a Self::Target { + fn deref(&self) -> &Self::Target { &self.0 } } /// A key/value pair. -/// +/// /// ```rust /// # use tikv_client::{Key, Value, KvPair}; /// let key = "key"; @@ -211,7 +214,7 @@ impl Deref for Value { /// let from_tuple = KvPair::from((key, value)); /// assert_eq!(constructed, from_tuple); /// ``` -/// +/// /// **But, you should not need to worry about all this:** Many functions which accept a `KvPair` /// accept an `Into`, which means all of the above types can be passed directly to those /// functions. @@ -268,7 +271,10 @@ impl KvPair { } impl From<(K, V)> for KvPair -where K: Into, V: Into { +where + K: Into, + V: Into, +{ fn from((k, v): (K, V)) -> Self { KvPair(k.into(), v.into()) } @@ -276,12 +282,12 @@ where K: Into, V: Into { /// The configuration for either a [`raw::Client`](raw/struct.Client.html) or a /// [`transaction::Client`](transaction/struct.Client.html). -/// +/// /// Because TiKV is managed by a [PD](https://github.com/pingcap/pd/) cluster, the endpoints for PD /// must be provided, **not** the TiKV nodes. /// /// It's important to **include more than one PD endpoint** (include all, if possible!) This helps avoid having a *single point of failure*. -/// +/// /// By default, this client will use an insecure connection over encryption-on-the-wire. Your /// deployment may have chosen to rely on security measures such as a private network, or a VPN /// layer providing secure transmission. TiKV does not currently offer encryption-at-rest. @@ -298,9 +304,9 @@ pub struct Config { impl Config { /// Create a new [`Config`](struct.Config.html) which coordinates with the given PD endpoints. - /// + /// /// It's important to **include more than one PD endpoint** (include all, if possible!) This helps avoid having a *single point of failure*. - /// + /// /// ```rust /// # use tikv_client::Config; /// let config = Config::new(vec!["192.168.0.100:2379", "192.168.0.101:2379"]); @@ -316,9 +322,9 @@ impl Config { } /// Set the certificate authority, certificate, and key locations for the [`Config`](struct.Config.html). - /// + /// /// By default, TiKV connections do not have utilize transport layer security. Enable it by setting these values. - /// + /// /// ```rust /// # use tikv_client::Config; /// let config = Config::new(vec!["192.168.0.100:2379", "192.168.0.101:2379"]) @@ -358,18 +364,18 @@ pub type KvFuture = Box>; /// /// let explict_range: Range = Range { start: Key::from("Rust"), end: Key::from("TiKV") }; /// let from_explict_range = explict_range.into_bounds(); -/// +/// /// let range: Range<&str> = "Rust".."TiKV"; /// let from_range = range.into_bounds(); /// assert_eq!(from_explict_range, from_range); -/// +/// /// let range: RangeInclusive<&str> = "Rust"..="TiKV"; /// let from_range = range.into_bounds(); /// assert_eq!( /// (Bound::Included(Key::from("Rust")), Bound::Included(Key::from("TiKV"))), /// from_range /// ); -/// +/// /// let range_from: RangeFrom<&str> = "Rust"..; /// let from_range_from = range_from.into_bounds(); /// assert_eq!( @@ -383,7 +389,7 @@ pub type KvFuture = Box>; /// (Bound::Unbounded, Bound::Excluded(Key::from("TiKV"))), /// from_range_to, /// ); -/// +/// /// let range_to_inclusive: RangeToInclusive<&str> = ..="TiKV"; /// let from_range_to_inclusive = range_to_inclusive.into_bounds(); /// assert_eq!( @@ -414,10 +420,13 @@ fn range_to_keys(range: (Bound, Bound)) -> Result<(Key, Option)> Bound::Excluded(mut v) => { let len = v.len(); if len > 0 { - v.deref_mut().get_mut(len -1).map(|v| { *v += 1; v }); + v.deref_mut().get_mut(len - 1).map(|v| { + *v += 1; + v + }); } v - }, + } Bound::Unbounded => return Err(Error::InvalidKeyRange), }; let end = match range.1 { @@ -425,7 +434,10 @@ fn range_to_keys(range: (Bound, Bound)) -> Result<(Key, Option)> Bound::Excluded(mut v) => Some({ let len = v.len(); if len > 0 { - v.deref_mut().get_mut(len -1).map(|v| { *v -= 1; v }); + v.deref_mut().get_mut(len - 1).map(|v| { + *v -= 1; + v + }); } v }), diff --git a/src/raw.rs b/src/raw.rs index 6800dd47..674e3155 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -22,13 +22,16 @@ oracle, while the transactional interface does. */ use crate::{rpc::RpcClient, Config, Error, Key, KeyRange, KvFuture, KvPair, Result, Value}; -use futures::{future, Future, Poll, Async}; -use std::{u32, sync::Arc, ops::{Bound, Deref, RangeBounds, DerefMut}}; - +use futures::{future, Async, Future, Poll}; +use std::{ + ops::{Bound, Deref, DerefMut}, + sync::Arc, + u32, +}; /// The TiKV raw [`Client`](struct.Client.html) is used to issue requests to the TiKV server and PD cluster. pub struct Client { - rpc: Arc + rpc: Arc, } impl Client { @@ -70,7 +73,7 @@ impl Client { /// Create a new [`BatchGet`](struct.BatchGet.html) request. /// - /// Once resolved this request will result in the fetching of the values associated with the + /// Once resolved this request will result in the fetching of the values associated with the /// given keys. /// /// ```rust,no_run @@ -83,7 +86,10 @@ impl Client { /// let result: Vec = req.wait().unwrap(); /// ``` pub fn batch_get(&self, keys: impl IntoIterator>) -> BatchGet { - BatchGet::new(self.rpc(), BatchGetInner::new(keys.into_iter().map(Into::into).collect())) + BatchGet::new( + self.rpc(), + BatchGetInner::new(keys.into_iter().map(Into::into).collect()), + ) } /// Create a new [`Put`](struct.Put.html) request. @@ -120,7 +126,10 @@ impl Client { /// let result: () = req.wait().unwrap(); /// ``` pub fn batch_put(&self, pairs: impl IntoIterator>) -> BatchPut { - BatchPut::new(self.rpc(), BatchPutInner::new(pairs.into_iter().map(Into::into).collect())) + BatchPut::new( + self.rpc(), + BatchPutInner::new(pairs.into_iter().map(Into::into).collect()), + ) } /// Create a new [`Delete`](struct.Delete.html) request. @@ -154,7 +163,10 @@ impl Client { /// let result: () = req.wait().unwrap(); /// ``` pub fn batch_delete(&self, keys: impl IntoIterator>) -> BatchDelete { - BatchDelete::new(self.rpc(), BatchDeleteInner::new(keys.into_iter().map(Into::into).collect())) + BatchDelete::new( + self.rpc(), + BatchDeleteInner::new(keys.into_iter().map(Into::into).collect()), + ) } /// Create a new [`Scan`](struct.Scan.html) request. @@ -173,7 +185,10 @@ impl Client { /// let result: Vec = req.wait().unwrap(); /// ``` pub fn scan(&self, range: impl KeyRange, limit: impl Into>) -> Scan { - Scan::new(self.rpc(), ScanInner::new(range.into_bounds(), limit.into().unwrap_or(u32::MAX))) + Scan::new( + self.rpc(), + ScanInner::new(range.into_bounds(), limit.into().unwrap_or(u32::MAX)), + ) } /// Create a new [`BatchScan`](struct.BatchScan.html) request. @@ -228,13 +243,13 @@ impl Client { } /// An unresolved [`Client`](struct.Client.html) connection to a TiKV cluster. -/// +/// /// Once resolved it will result in a connected [`Client`](struct.Client.html). /// /// ```rust,no_run /// use tikv_client::{Config, raw::{Client, Connect}}; /// use futures::Future; -/// +/// /// let connect: Connect = Client::new(&Config::default()); /// let client: Client = connect.wait().unwrap(); /// ``` @@ -259,31 +274,30 @@ impl Future for Connect { } } - /// A [`ColumnFamily`](struct.ColumnFamily.html) is an optional parameter for [`raw::Client`](struct.Client.html) requests. -/// +/// /// TiKV uses RocksDB's `ColumnFamily` support. You can learn more about RocksDB's `ColumnFamily`s [on their wiki](https://github.com/facebook/rocksdb/wiki/Column-Families). -/// +/// /// By default in TiKV data is stored in three different `ColumnFamily` values, configurable in the TiKV server's configuration: -/// +/// /// * Default: Where real user data is stored. Set by `[rocksdb.defaultcf]`. /// * Write: Where MVCC and index related data are stored. Set by `[rocksdb.writecf]`. /// * Lock: Where lock information is stored. Set by `[rocksdb.lockcf]`. -/// +/// /// Not providing a call a `ColumnFamily` means it will use the default value of `default`. -/// +/// /// The best (and only) way to create a [`ColumnFamily`](struct.ColumnFamily.html) is via the `From` implementation: -/// +/// /// ```rust /// # use tikv_client::raw::ColumnFamily; -/// +/// /// let cf = ColumnFamily::from("write"); /// let cf = ColumnFamily::from(String::from("write")); /// let cf = ColumnFamily::from(&String::from("write")); /// ``` -/// +/// /// This is a *wrapper type* that implements `Deref` so it can be used like one transparently. -/// +/// /// **But, you should not need to worry about all this:** Many functions which accept a /// `ColumnFamily` accept an `Into`, which means all of the above types can be passed /// directly to those functions. @@ -312,8 +326,6 @@ impl Deref for ColumnFamily { } } - - pub trait RequestInner: Sized { type Resp; @@ -388,13 +400,30 @@ where } /// An unresolved [`Client::get`](struct.Client.html#method.get) request. -/// +/// /// Once resolved this request will result in the fetching of the value associated with the given /// key. pub struct Get { request: Request, } +impl Get { + fn new(client: Arc, inner: GetInner) -> Self { + Self { + request: Request::new(client, inner) + } + } +} + +impl Future for Get { + type Item = Value; + type Error = Error; + + fn poll(&mut self) -> Poll { + self.request.poll() + } +} + pub(crate) struct GetInner { key: Key, } @@ -421,6 +450,23 @@ pub struct BatchGet { request: Request, } +impl BatchGet { + fn new(client: Arc, inner: BatchGetInner) -> Self { + Self { + request: Request::new(client, inner) + } + } +} + +impl Future for BatchGet { + type Item = Vec; + type Error = Error; + + fn poll(&mut self) -> Poll { + self.request.poll() + } +} + pub(crate) struct BatchGetInner { keys: Vec, } @@ -440,13 +486,30 @@ impl BatchGetInner { } /// An unresolved [`Client::put`](struct.Client.html#method.put) request. -/// +/// /// Once resolved this request will result in the putting of the value associated with the given /// key. pub struct Put { request: Request, } +impl Put { + fn new(client: Arc, inner: PutInner) -> Self { + Self { + request: Request::new(client, inner) + } + } +} + +impl Future for Put { + type Item = (); + type Error = Error; + + fn poll(&mut self) -> Poll { + self.request.poll() + } +} + pub(crate) struct PutInner { key: Key, value: Value, @@ -468,12 +531,29 @@ impl RequestInner for PutInner { } /// An unresolved [`Client::batch_put`](struct.Client.html#method.batch_put) request. -/// +/// /// Once resolved this request will result in the setting of the value associated with the given key. pub struct BatchPut { request: Request, } +impl BatchPut { + fn new(client: Arc, inner: BatchPutInner) -> Self { + Self { + request: Request::new(client, inner) + } + } +} + +impl Future for BatchPut { + type Item = (); + type Error = Error; + + fn poll(&mut self) -> Poll { + self.request.poll() + } +} + pub(crate) struct BatchPutInner { pairs: Vec, } @@ -499,6 +579,23 @@ pub struct Delete { request: Request, } +impl Delete { + fn new(client: Arc, inner: DeleteInner) -> Self { + Self { + request: Request::new(client, inner) + } + } +} + +impl Future for Delete { + type Item = (); + type Error = Error; + + fn poll(&mut self) -> Poll { + self.request.poll() + } +} + pub(crate) struct DeleteInner { key: Key, } @@ -524,6 +621,23 @@ pub struct BatchDelete { request: Request, } +impl BatchDelete { + fn new(client: Arc, inner: BatchDeleteInner) -> Self { + Self { + request: Request::new(client, inner) + } + } +} + +impl Future for BatchDelete { + type Item = (); + type Error = Error; + + fn poll(&mut self) -> Poll { + self.request.poll() + } +} + pub(crate) struct BatchDeleteInner { keys: Vec, } @@ -542,7 +656,6 @@ impl RequestInner for BatchDeleteInner { } } - pub(crate) struct ScanInner { range: (Bound, Bound), limit: u32, @@ -690,6 +803,23 @@ pub struct DeleteRange { request: Request, } +impl DeleteRange { + fn new(client: Arc, inner: DeleteRangeInner) -> Self { + Self { + request: Request::new(client, inner) + } + } +} + +impl Future for DeleteRange { + type Item = (); + type Error = Error; + + fn poll(&mut self) -> Poll { + self.request.poll() + } +} + pub(crate) struct DeleteRangeInner { range: Result<(Key, Option)>, } diff --git a/src/rpc/tikv/client.rs b/src/rpc/tikv/client.rs index e7d85b3c..febe1541 100644 --- a/src/rpc/tikv/client.rs +++ b/src/rpc/tikv/client.rs @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::{fmt, sync::Arc, time::Duration, ops::Bound}; +use std::{fmt, sync::Arc, time::Duration}; use futures::Future; use grpcio::{CallOption, Environment}; diff --git a/src/transaction.rs b/src/transaction.rs index 1d6e425c..28cb88d6 100644 --- a/src/transaction.rs +++ b/src/transaction.rs @@ -213,7 +213,7 @@ impl Transaction { } /// Rollback the actions of the transaction. - /// + /// /// ```rust,no_run /// # use tikv_client::{Config, transaction::Client}; /// # use futures::Future; @@ -229,7 +229,7 @@ impl Transaction { } /// Lock the given keys. - /// + /// /// ```rust,no_run /// # use tikv_client::{Config, transaction::Client}; /// # use futures::Future; @@ -315,7 +315,7 @@ impl Transaction { /// Create a new [`BatchGet`](struct.BatchGet.html) request. /// - /// Once resolved this request will result in the fetching of the values associated with the + /// Once resolved this request will result in the fetching of the values associated with the /// given keys. /// /// ```rust,no_run @@ -616,4 +616,4 @@ impl Future for Delete { let _key = &self.key; unimplemented!() } -} \ No newline at end of file +} From cf2eec934b41af3a6886d7366418701d570371c3 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 31 Dec 2018 22:11:16 +0800 Subject: [PATCH 05/20] Add cf to concrete builder types Signed-off-by: Xiaoguang Sun --- examples/raw.rs | 12 ++++++------ src/raw.rs | 42 ++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 48 insertions(+), 6 deletions(-) diff --git a/examples/raw.rs b/examples/raw.rs index 5c166c4b..eedef511 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -43,7 +43,7 @@ fn main() -> Result<()> { // // Here we set the key `TiKV` to have the value `Rust` associated with it. let put_request = client.put(KEY, VALUE); - let put_result: () = put_request.wait()?; // Returns a `tikv_client::Error` on failure. + let _put_result: () = put_request.wait()?; // Returns a `tikv_client::Error` on failure. println!("Put key \"{}\", value \"{}\".", KEY, VALUE); // @@ -62,7 +62,7 @@ fn main() -> Result<()> { // You can also set the `ColumnFamily` used by the request. // This is *advanced usage* and should have some special considerations. - let req = client + let _delete_result: () = client .delete(key.clone()) .cf(CUSTOM_CF) .wait() @@ -71,7 +71,7 @@ fn main() -> Result<()> { client .get(key) - .cf("test_cf") + .cf(CUSTOM_CF) .wait() .expect_err("Get returned value for not existing key"); @@ -79,7 +79,7 @@ fn main() -> Result<()> { let values = client .batch_get(keys.clone()) - .cf("test_cf") + .cf(CUSTOM_CF) .wait() .expect("Could not get values"); println!("Found values: {:?} for keys: {:?}", values, keys); @@ -88,7 +88,7 @@ fn main() -> Result<()> { let end: Key = b"k2".to_vec().into(); client .scan(start.clone()..end.clone(), 10) - .cf("test_cf") + .cf(CUSTOM_CF) .key_only() .wait() .expect("Could not scan"); @@ -96,7 +96,7 @@ fn main() -> Result<()> { let ranges = vec![start.clone()..end.clone(), start.clone()..end.clone()]; client .batch_scan(ranges, 10) - .cf("test_cf") + .cf(CUSTOM_CF) .key_only() .wait() .expect("Could not batch scan"); diff --git a/src/raw.rs b/src/raw.rs index 674e3155..11b4260a 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -413,6 +413,12 @@ impl Get { request: Request::new(client, inner) } } + + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). + pub fn cf(mut self, cf: impl Into) -> Self { + self.request = self.request.cf(cf); + self + } } impl Future for Get { @@ -456,6 +462,12 @@ impl BatchGet { request: Request::new(client, inner) } } + + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). + pub fn cf(mut self, cf: impl Into) -> Self { + self.request = self.request.cf(cf); + self + } } impl Future for BatchGet { @@ -499,6 +511,12 @@ impl Put { request: Request::new(client, inner) } } + + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). + pub fn cf(mut self, cf: impl Into) -> Self { + self.request = self.request.cf(cf); + self + } } impl Future for Put { @@ -543,6 +561,12 @@ impl BatchPut { request: Request::new(client, inner) } } + + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). + pub fn cf(mut self, cf: impl Into) -> Self { + self.request = self.request.cf(cf); + self + } } impl Future for BatchPut { @@ -585,6 +609,12 @@ impl Delete { request: Request::new(client, inner) } } + + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). + pub fn cf(mut self, cf: impl Into) -> Self { + self.request = self.request.cf(cf); + self + } } impl Future for Delete { @@ -627,6 +657,12 @@ impl BatchDelete { request: Request::new(client, inner) } } + + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). + pub fn cf(mut self, cf: impl Into) -> Self { + self.request = self.request.cf(cf); + self + } } impl Future for BatchDelete { @@ -809,6 +845,12 @@ impl DeleteRange { request: Request::new(client, inner) } } + + /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). + pub fn cf(mut self, cf: impl Into) -> Self { + self.request = self.request.cf(cf); + self + } } impl Future for DeleteRange { From a745bb05896e4af12341539dd7f574685a58997a Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 1 Jan 2019 16:45:37 +0800 Subject: [PATCH 06/20] Fixed some comments and confusing name Signed-off-by: Xiaoguang Sun --- src/errors.rs | 4 ++-- src/lib.rs | 4 +++- src/raw.rs | 7 +++---- src/rpc/client.rs | 24 ++++++++++++------------ src/rpc/pd/client.rs | 27 ++++++++++++++++----------- src/rpc/pd/leader.rs | 8 ++++---- src/rpc/tikv/client.rs | 1 + src/rpc/util.rs | 2 +- 8 files changed, 42 insertions(+), 35 deletions(-) diff --git a/src/errors.rs b/src/errors.rs index deee4291..aeb81e69 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -22,7 +22,7 @@ quick_error! { /// for errors. *This may change in future versions.* #[derive(Debug)] pub enum Error { - /// Wraps a a `std::io::Error`. + /// Wraps a `std::io::Error`. Io(err: ::std::io::Error) { from() cause(err) @@ -69,7 +69,7 @@ quick_error! { description("store not match") display("requesting store '{}' when actual store is '{}'. {}", request_store_id, actual_store_id, message) } - /// The given key is not eithin the given region. + /// The given key is not within the given region. KeyNotInRegion(key: Vec, region_id: u64, start_key: Vec, end_key: Vec) { description("region is not found") display("key {:?} is not in region {:?}: [{:?}, {:?})", key, region_id, start_key, end_key) diff --git a/src/lib.rs b/src/lib.rs index 4ca64c78..ca1db26e 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -302,6 +302,8 @@ pub struct Config { timeout: Duration, } +const DEFAULT_REQUEST_TIMEOUT: Duration = Duration::from_secs(2); + impl Config { /// Create a new [`Config`](struct.Config.html) which coordinates with the given PD endpoints. /// @@ -317,7 +319,7 @@ impl Config { ca_path: None, cert_path: None, key_path: None, - timeout: Duration::from_secs(2), + timeout: DEFAULT_REQUEST_TIMEOUT, } } diff --git a/src/raw.rs b/src/raw.rs index 11b4260a..f658bc6e 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -226,8 +226,6 @@ impl Client { /// /// Once resolved this request will result in the deletion of all keys over the given range. /// - /// If not passed a `limit` parameter, it will default to `u32::MAX`. - /// /// ```rust,no_run /// # use tikv_client::{Key, Config, raw::Client}; /// # use futures::Future; @@ -778,10 +776,11 @@ impl RequestInner for BatchScanInner { type Resp = Vec; fn execute(self, client: Arc, cf: Option) -> KvFuture { - let (mut errors, ranges): (Vec<_>, Vec<_>) = + let (errors, ranges): (Vec<_>, Vec<_>) = self.ranges.into_iter().partition(Result::is_err); if !errors.is_empty() { - Box::new(future::err(errors.pop().unwrap().unwrap_err())) + // All errors must be InvalidKeyRange so we can simply return a new InvalidKeyRange + Box::new(future::err(Error::InvalidKeyRange)) } else { Box::new(client.raw_batch_scan( ranges.into_iter().map(Result::unwrap).collect(), diff --git a/src/rpc/client.rs b/src/rpc/client.rs index a33ce31d..c6e15083 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -267,10 +267,10 @@ impl RpcClient { ) -> impl Future, Error = Error> { let inner = self.inner(); self.group_tasks_by_region(keys) - .and_then(move |gk| { - let gk = gk.into_inner(); - let mut tasks = Vec::with_capacity(gk.len()); - for (region, keys) in gk.into_iter() { + .and_then(move |task_groups| { + let task_groups = task_groups.into_inner(); + let mut tasks = Vec::with_capacity(task_groups.len()); + for (region, keys) in task_groups.into_iter() { let inner = Arc::clone(&inner); let cf = cf.clone(); let task = Self::region_context_by_id(inner, region.id) @@ -312,10 +312,10 @@ impl RpcClient { let inner = self.inner(); Either::B( self.group_tasks_by_region(pairs) - .and_then(move |gk| { - let gk = gk.into_inner(); - let mut tasks = Vec::with_capacity(gk.len()); - for (region, pairs) in gk.into_iter() { + .and_then(move |task_groups| { + let task_groups = task_groups.into_inner(); + let mut tasks = Vec::with_capacity(task_groups.len()); + for (region, pairs) in task_groups.into_iter() { let inner = Arc::clone(&inner); let cf = cf.clone(); let task = Self::region_context_by_id(inner, region.id) @@ -348,10 +348,10 @@ impl RpcClient { ) -> impl Future { let inner = self.inner(); self.group_tasks_by_region(keys) - .and_then(move |gk| { - let gk = gk.into_inner(); - let mut tasks = Vec::with_capacity(gk.len()); - for (region, keys) in gk.into_iter() { + .and_then(move |task_groups| { + let task_groups = task_groups.into_inner(); + let mut tasks = Vec::with_capacity(task_groups.len()); + for (region, keys) in task_groups.into_iter() { let inner = Arc::clone(&inner); let cf = cf.clone(); let task = Self::region_context_by_id(inner, region.id) diff --git a/src/rpc/pd/client.rs b/src/rpc/pd/client.rs index fd99c6b9..5f3a25bd 100644 --- a/src/rpc/pd/client.rs +++ b/src/rpc/pd/client.rs @@ -41,19 +41,23 @@ trait PdResponse { fn header(&self) -> &pdpb::ResponseHeader; } -macro_rules! pd_response { - ($type:ty) => { - impl PdResponse for $type { - fn header(&self) -> &pdpb::ResponseHeader { - self.get_header() - } - } - }; +impl PdResponse for pdpb::GetStoreResponse { + fn header(&self) -> &pdpb::ResponseHeader { + self.get_header() + } +} + +impl PdResponse for pdpb::GetRegionResponse { + fn header(&self) -> &pdpb::ResponseHeader { + self.get_header() + } } -pd_response!(pdpb::GetStoreResponse); -pd_response!(pdpb::GetRegionResponse); -pd_response!(pdpb::GetAllStoresResponse); +impl PdResponse for pdpb::GetAllStoresResponse { + fn header(&self) -> &pdpb::ResponseHeader { + self.get_header() + } +} pub struct PdClient { cluster_id: u64, @@ -213,6 +217,7 @@ impl fmt::Debug for PdClient { fmt.debug_struct("PdClient") .field("cluster_id", &self.cluster_id) .field("leader", &self.get_leader()) + .field("timeout", &self.timeout) .finish() } } diff --git a/src/rpc/pd/leader.rs b/src/rpc/pd/leader.rs index 5fbc388c..3e50c6d8 100644 --- a/src/rpc/pd/leader.rs +++ b/src/rpc/pd/leader.rs @@ -106,7 +106,7 @@ impl PdReactor { .unwrap(), ) } else { - warn!("tso sender and receiver are stale, refreshing.."); + warn!("tso sender and receiver are stale, refreshing..."); let (tso_tx, tso_rx) = unbounded(); self.tso_tx = tso_tx; self.tso_rx = Some(tso_rx); @@ -162,7 +162,7 @@ impl PdReactor { let tso_pending = reactor.tso_pending.take().unwrap(); reactor.schedule(PdTask::Response(tso_pending, resp)); if !reactor.tso_batch.is_empty() { - /* schedule another tso_batch of request */ + // Schedule another tso_batch of request reactor.schedule(PdTask::Request); } Ok(()) @@ -214,7 +214,7 @@ impl PdReactor { let (tx, rx) = oneshot::channel::(); self.tso_batch.push(tx); if self.tso_pending.is_none() { - /* schedule tso request to run */ + // Schedule tso request to run. self.schedule(PdTask::Request); } rx.map_err(Error::Canceled).then(move |r| context.done(r)) @@ -267,7 +267,7 @@ impl LeaderClient { // Re-establish connection with PD leader in synchronized fashion. pub fn reconnect(leader: &Arc>, interval: u64) -> Result<()> { - warn!("updating pd client, block the tokio core"); + warn!("updating pd client, blocking the tokio core"); let ((client, members), start) = { let leader = leader.rl(); if leader.last_update.elapsed() < Duration::from_secs(interval) { diff --git a/src/rpc/tikv/client.rs b/src/rpc/tikv/client.rs index febe1541..4fa81e26 100644 --- a/src/rpc/tikv/client.rs +++ b/src/rpc/tikv/client.rs @@ -695,6 +695,7 @@ impl fmt::Debug for KvClient { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fmt.debug_struct("KvClient") .field("address", &self.address) + .field("timeout", &self.timeout) .finish() } } diff --git a/src/rpc/util.rs b/src/rpc/util.rs index 1bf07604..76ae297e 100644 --- a/src/rpc/util.rs +++ b/src/rpc/util.rs @@ -66,7 +66,7 @@ pub fn get_tag_from_thread_name() -> Option { #[inline] pub fn duration_to_sec(d: Duration) -> f64 { let nanos = f64::from(d.subsec_nanos()); - // Most of case, we can't have so large Duration, so here just panic if overflow now. + // In most cases, we can't have so large Duration, so here just panic if overflow now. d.as_secs() as f64 + (nanos / 1_000_000_000.0) } From b3aebe3d2c2f6df843e9e9f5872e47b04779b7ad Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 1 Jan 2019 18:31:25 +0800 Subject: [PATCH 07/20] Change Request from struct to enum Signed-off-by: Xiaoguang Sun --- src/raw.rs | 155 ++++++++++++++++++++++------------------------------- 1 file changed, 64 insertions(+), 91 deletions(-) diff --git a/src/raw.rs b/src/raw.rs index f658bc6e..bcef68b7 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -24,7 +24,7 @@ oracle, while the transactional interface does. use crate::{rpc::RpcClient, Config, Error, Key, KeyRange, KvFuture, KvPair, Result, Value}; use futures::{future, Async, Future, Poll}; use std::{ - ops::{Bound, Deref, DerefMut}, + ops::{Bound, Deref}, sync::Arc, u32, }; @@ -330,69 +330,43 @@ pub trait RequestInner: Sized { fn execute(self, client: Arc, cf: Option) -> KvFuture; } -pub struct Request +enum RequestState where Inner: RequestInner, { - inner: Option<(Arc, Inner)>, - future: Option>, - cf: Option, + Uninitiated(Option<(Arc, Inner, Option)>), + Initiated(KvFuture), } -impl Request +impl RequestState where Inner: RequestInner, { fn new(client: Arc, inner: Inner) -> Self { - Request { - inner: Some((client, inner)), - future: None, - cf: None, - } + RequestState::Uninitiated(Some((client, inner, None))) } - /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). - pub fn cf(mut self, cf: impl Into) -> Self { - self.cf = Some(cf.into()); - self - } -} - -impl Deref for Request -where - Inner: RequestInner, -{ - type Target = Inner; - - fn deref(&self) -> &Self::Target { - &self.inner.as_ref().unwrap().1 + fn cf(&mut self, new_cf: impl Into) { + if let RequestState::Uninitiated(Some((_, _, ref mut cf))) = self { + cf.replace(new_cf.into()); + } } -} -impl DerefMut for Request -where - Inner: RequestInner, -{ - fn deref_mut(&mut self) -> &mut Self::Target { - &mut self.inner.as_mut().unwrap().1 + fn inner_mut(&mut self) -> Option<&mut Inner> { + match self { + RequestState::Uninitiated(Some((_, ref mut inner, _))) => Some(inner), + _ => None, + } } -} -impl Future for Request -where - Inner: RequestInner, -{ - type Item = Inner::Resp; - type Error = Error; - - fn poll(&mut self) -> Poll { - loop { - if self.inner.is_some() { - let (client, inner) = self.inner.take().unwrap(); - self.future = Some(inner.execute(client, self.cf.take())); - } else { - break self.future.as_mut().map(|x| x.poll()).unwrap(); - } + fn poll(&mut self) -> Poll { + if let RequestState::Uninitiated(state) = self { + let (client, inner, cf) = state.take().unwrap(); + *self = RequestState::Initiated(inner.execute(client, cf)); + } + match self { + RequestState::Initiated(ref mut future) => future.poll(), + _ => unreachable!(), } } } @@ -402,19 +376,19 @@ where /// Once resolved this request will result in the fetching of the value associated with the given /// key. pub struct Get { - request: Request, + state: RequestState, } impl Get { fn new(client: Arc, inner: GetInner) -> Self { Self { - request: Request::new(client, inner) + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } } @@ -424,7 +398,7 @@ impl Future for Get { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } @@ -451,19 +425,19 @@ impl RequestInner for GetInner { /// Once resolved this request will result in the fetching of the values associated with the given /// keys. pub struct BatchGet { - request: Request, + state: RequestState, } impl BatchGet { fn new(client: Arc, inner: BatchGetInner) -> Self { Self { - request: Request::new(client, inner) + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } } @@ -473,7 +447,7 @@ impl Future for BatchGet { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } @@ -500,19 +474,19 @@ impl BatchGetInner { /// Once resolved this request will result in the putting of the value associated with the given /// key. pub struct Put { - request: Request, + state: RequestState, } impl Put { fn new(client: Arc, inner: PutInner) -> Self { Self { - request: Request::new(client, inner) + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } } @@ -522,7 +496,7 @@ impl Future for Put { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } @@ -550,19 +524,19 @@ impl RequestInner for PutInner { /// /// Once resolved this request will result in the setting of the value associated with the given key. pub struct BatchPut { - request: Request, + state: RequestState, } impl BatchPut { fn new(client: Arc, inner: BatchPutInner) -> Self { Self { - request: Request::new(client, inner) + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } } @@ -572,7 +546,7 @@ impl Future for BatchPut { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } @@ -598,19 +572,19 @@ impl RequestInner for BatchPutInner { /// /// Once resolved this request will result in the deletion of the given key. pub struct Delete { - request: Request, + state: RequestState, } impl Delete { fn new(client: Arc, inner: DeleteInner) -> Self { Self { - request: Request::new(client, inner) + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } } @@ -620,7 +594,7 @@ impl Future for Delete { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } @@ -646,19 +620,19 @@ impl RequestInner for DeleteInner { /// /// Once resolved this request will result in the deletion of the given keys. pub struct BatchDelete { - request: Request, + state: RequestState, } impl BatchDelete { fn new(client: Arc, inner: BatchDeleteInner) -> Self { Self { - request: Request::new(client, inner) + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } } @@ -668,7 +642,7 @@ impl Future for BatchDelete { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } @@ -722,25 +696,25 @@ impl RequestInner for ScanInner { /// /// Once resolved this request will result in a scanner over the given range. pub struct Scan { - request: Request, + state: RequestState, } impl Scan { fn new(client: Arc, inner: ScanInner) -> Self { Scan { - request: Request::new(client, inner), + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } pub fn key_only(mut self) -> Self { - self.request.inner = self.request.inner.map(|mut x| { - x.1.key_only = true; + self.state.inner_mut().map(|mut x| { + x.key_only = true; x }); self @@ -752,7 +726,7 @@ impl Future for Scan { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } @@ -776,8 +750,7 @@ impl RequestInner for BatchScanInner { type Resp = Vec; fn execute(self, client: Arc, cf: Option) -> KvFuture { - let (errors, ranges): (Vec<_>, Vec<_>) = - self.ranges.into_iter().partition(Result::is_err); + let (errors, ranges): (Vec<_>, Vec<_>) = self.ranges.into_iter().partition(Result::is_err); if !errors.is_empty() { // All errors must be InvalidKeyRange so we can simply return a new InvalidKeyRange Box::new(future::err(Error::InvalidKeyRange)) @@ -796,25 +769,25 @@ impl RequestInner for BatchScanInner { /// /// Once resolved this request will result in a scanner over the given ranges. pub struct BatchScan { - request: Request, + state: RequestState, } impl BatchScan { fn new(client: Arc, inner: BatchScanInner) -> Self { BatchScan { - request: Request::new(client, inner), + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } pub fn key_only(mut self) -> Self { - self.request.inner = self.request.inner.map(|mut x| { - x.1.key_only = true; + self.state.inner_mut().map(|mut x| { + x.key_only = true; x }); self @@ -826,7 +799,7 @@ impl Future for BatchScan { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } @@ -835,19 +808,19 @@ impl Future for BatchScan { /// Once resolved this request will result in the deletion of the values in the given /// range. pub struct DeleteRange { - request: Request, + state: RequestState, } impl DeleteRange { fn new(client: Arc, inner: DeleteRangeInner) -> Self { Self { - request: Request::new(client, inner) + state: RequestState::new(client, inner), } } /// Set the (optional) [`ColumnFamily`](struct.ColumnFamily.html). pub fn cf(mut self, cf: impl Into) -> Self { - self.request = self.request.cf(cf); + self.state.cf(cf); self } } @@ -857,7 +830,7 @@ impl Future for DeleteRange { type Error = Error; fn poll(&mut self) -> Poll { - self.request.poll() + self.state.poll() } } From f70ce8863d5b75d3881f7c073227b6fe9782c68a Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 1 Jan 2019 19:25:16 +0800 Subject: [PATCH 08/20] Change tso_tx/rx channel to bounded Signed-off-by: Xiaoguang Sun --- src/rpc/pd/leader.rs | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/rpc/pd/leader.rs b/src/rpc/pd/leader.rs index 3e50c6d8..d24bacb2 100644 --- a/src/rpc/pd/leader.rs +++ b/src/rpc/pd/leader.rs @@ -19,7 +19,7 @@ use std::{ use futures::{ sync::{ - mpsc::{unbounded, UnboundedReceiver, UnboundedSender}, + mpsc::{channel, unbounded, Receiver, Sender, UnboundedReceiver, UnboundedSender}, oneshot, }, Future, Sink, Stream, @@ -62,8 +62,8 @@ enum PdTask { struct PdReactor { task_tx: Option>>, - tso_tx: UnboundedSender, - tso_rx: Option>, + tso_tx: Sender, + tso_rx: Option>, handle: Option>, tso_pending: Option>, @@ -81,7 +81,7 @@ impl Drop for PdReactor { impl PdReactor { fn new() -> Self { - let (tso_tx, tso_rx) = unbounded(); + let (tso_tx, tso_rx) = channel(1); PdReactor { task_tx: None, tso_tx, @@ -107,7 +107,7 @@ impl PdReactor { ) } else { warn!("tso sender and receiver are stale, refreshing..."); - let (tso_tx, tso_rx) = unbounded(); + let (tso_tx, tso_rx) = channel(1); self.tso_tx = tso_tx; self.tso_rx = Some(tso_rx); self.schedule(PdTask::Init); @@ -181,7 +181,10 @@ impl PdReactor { let batch_size = observe_tso_batch(tso_batch.len()); request.set_count(batch_size); reactor.tso_pending = Some(tso_batch); - reactor.tso_tx.unbounded_send(request).unwrap(); + reactor + .tso_tx + .try_send(request) + .expect("channel can never be full"); } fn tso_response( From c0709ee23f7edd3d6611de58d67268aa4c579c63 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 5 Jan 2019 20:02:38 +0800 Subject: [PATCH 09/20] Fix format issues and improve implementations Signed-off-by: Xiaoguang Sun --- examples/raw.rs | 17 ++++++++--- src/errors.rs | 4 +++ src/lib.rs | 58 ++++++++++++++++++++--------------- src/raw.rs | 73 +++++++++++++++++++++++--------------------- src/rpc/pd/leader.rs | 7 ++--- 5 files changed, 92 insertions(+), 67 deletions(-) diff --git a/examples/raw.rs b/examples/raw.rs index eedef511..6fbf0d9b 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -13,7 +13,7 @@ use futures::future::Future; use std::path::PathBuf; -use tikv_client::{raw::Client, Config, Key, Result, Value}; +use tikv_client::{raw::Client, Config, Key, KvPair, Result, Value}; const KEY: &str = "TiKV"; const VALUE: &str = "Rust"; @@ -43,12 +43,13 @@ fn main() -> Result<()> { // // Here we set the key `TiKV` to have the value `Rust` associated with it. let put_request = client.put(KEY, VALUE); - let _put_result: () = put_request.wait()?; // Returns a `tikv_client::Error` on failure. + put_request.wait()?; // Returns a `tikv_client::Error` on failure. println!("Put key \"{}\", value \"{}\".", KEY, VALUE); // // Unlike a standard Rust HashMap all calls take owned values. This is because under the hood - // protobufs must take ownership of the data. If we only took a borrow we'd need to internally // clone it. This is against Rust API guidelines, so you must manage this yourself. + // protobufs must take ownership of the data. If we only took a borrow we'd need to internally + // clone it. This is against Rust API guidelines, so you must manage this yourself. // // Above, you saw we can use a `&'static str`, this is primarily for making examples short. // This type is practical to use for real things, and usage forces an internal copy. @@ -62,7 +63,7 @@ fn main() -> Result<()> { // You can also set the `ColumnFamily` used by the request. // This is *advanced usage* and should have some special considerations. - let _delete_result: () = client + client .delete(key.clone()) .cf(CUSTOM_CF) .wait() @@ -75,6 +76,14 @@ fn main() -> Result<()> { .wait() .expect_err("Get returned value for not existing key"); + let pairs: Vec = (1..3) + .map(|i| KvPair::from((Key::from(format!("k{}", i)), Value::from(format!("v{}", i))))) + .collect(); + client + .batch_put(pairs.clone()) + .wait() + .expect("Could not put pairs"); + let keys = vec![Key::from(b"k1".to_vec()), Key::from(b"k2".to_vec())]; let values = client diff --git a/src/errors.rs b/src/errors.rs index aeb81e69..118ba202 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -125,6 +125,10 @@ quick_error! { description("ranges can not be overlapping") display("Ranges can not be overlapping") } + MaxScanLimitExceeded(limit: u32, max_limit: u32) { + description("limit exceeds max scan limit") + display("Limit {} excceds max scan limit {}", limit, max_limit) + } } } diff --git a/src/lib.rs b/src/lib.rs index ca1db26e..011c82f3 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. #![recursion_limit = "128"] -#![type_length_limit = "2097152"] +#![type_length_limit = "1572864"] use futures::Future; use serde_derive::*; @@ -22,6 +22,7 @@ use std::{ }, path::PathBuf, time::Duration, + u8::{MAX as U8_MAX, MIN as U8_MIN}, }; mod errors; @@ -36,12 +37,14 @@ pub use crate::errors::Result; /// The key part of a key/value pair. /// -/// In TiKV, keys are an ordered sequence of bytes. This has an advantage over choosing `String` as valid `UTF-8` is not required. This means that the user is permitted to store any data they wish, +/// In TiKV, keys are an ordered sequence of bytes. This has an advantage over choosing `String` as +/// valid `UTF-8` is not required. This means that the user is permitted to store any data they wish, /// as long as it can be represented by bytes. (Which is to say, pretty much anything!) /// -/// This is a *wrapper type* that implements `Deref>` so it can be used like one transparently. +/// This is a *wrapper type* that implements `Deref` so it can be used like one transparently. /// -/// This type also implements `From` for many types. With one exception, these are all done without reallocation. Using a `&'static str`, like many examples do for simplicity, has an internal +/// This type also implements `From` for many types. With one exception, these are all done without +/// reallocation. Using a `&'static str`, like many examples do for simplicity, has an internal /// allocation cost. /// /// This type wraps around an owned value, so it should be treated it like `String` or `Vec` @@ -80,6 +83,14 @@ impl Key { fn into_inner(self) -> Vec { self.0 } + + fn push(&mut self, v: u8) { + self.0.push(v) + } + + fn pop(&mut self) { + self.0.pop(); + } } impl From> for Key { @@ -130,12 +141,14 @@ impl DerefMut for Key { /// The value part of a key/value pair. /// -/// In TiKV, values are an ordered sequence of bytes. This has an advantage over choosing `String` as valid `UTF-8` is not required. This means that the user is permitted to store any data they wish, +/// In TiKV, values are an ordered sequence of bytes. This has an advantage over choosing `String` +/// as valid `UTF-8` is not required. This means that the user is permitted to store any data they wish, /// as long as it can be represented by bytes. (Which is to say, pretty much anything!) /// -/// This is a *wrapper type* that implements `Deref>` so it can be used like one transparently. +/// This is a *wrapper type* that implements `Deref` so it can be used like one transparently. /// -/// This type also implements `From` for many types. With one exception, these are all done without reallocation. Using a `&'static str`, like many examples do for simplicity, has an internal +/// This type also implements `From` for many types. With one exception, these are all done without +/// reallocation. Using a `&'static str`, like many examples do for simplicity, has an internal /// allocation cost. /// /// This type wraps around an owned value, so it should be treated it like `String` or `Vec` @@ -286,7 +299,8 @@ where /// Because TiKV is managed by a [PD](https://github.com/pingcap/pd/) cluster, the endpoints for PD /// must be provided, **not** the TiKV nodes. /// -/// It's important to **include more than one PD endpoint** (include all, if possible!) This helps avoid having a *single point of failure*. +/// It's important to **include more than one PD endpoint** (include all, if possible!) +/// This helps avoid having a *single point of failure*. /// /// By default, this client will use an insecure connection over encryption-on-the-wire. Your /// deployment may have chosen to rely on security measures such as a private network, or a VPN @@ -307,7 +321,8 @@ const DEFAULT_REQUEST_TIMEOUT: Duration = Duration::from_secs(2); impl Config { /// Create a new [`Config`](struct.Config.html) which coordinates with the given PD endpoints. /// - /// It's important to **include more than one PD endpoint** (include all, if possible!) This helps avoid having a *single point of failure*. + /// It's important to **include more than one PD endpoint** (include all, if possible!) + /// This helps avoid having a *single point of failure*. /// /// ```rust /// # use tikv_client::Config; @@ -325,7 +340,7 @@ impl Config { /// Set the certificate authority, certificate, and key locations for the [`Config`](struct.Config.html). /// - /// By default, TiKV connections do not have utilize transport layer security. Enable it by setting these values. + /// By default, TiKV connections do not utilize transport layer security. Enable it by setting these values. /// /// ```rust /// # use tikv_client::Config; @@ -420,12 +435,9 @@ fn range_to_keys(range: (Bound, Bound)) -> Result<(Key, Option)> let start = match range.0 { Bound::Included(v) => v, Bound::Excluded(mut v) => { - let len = v.len(); - if len > 0 { - v.deref_mut().get_mut(len - 1).map(|v| { - *v += 1; - v - }); + match v.last_mut() { + None | Some(&mut U8_MAX) => v.push(0), + Some(v) => *v += 1, } v } @@ -434,12 +446,10 @@ fn range_to_keys(range: (Bound, Bound)) -> Result<(Key, Option)> let end = match range.1 { Bound::Included(v) => Some(v), Bound::Excluded(mut v) => Some({ - let len = v.len(); - if len > 0 { - v.deref_mut().get_mut(len - 1).map(|v| { - *v -= 1; - v - }); + match v.last_mut() { + None => (), + Some(&mut U8_MIN) => v.pop(), + Some(v) => *v -= 1, } v }), @@ -490,11 +500,11 @@ impl> KeyRange for RangeToInclusive { impl> KeyRange for (Bound, Bound) { fn into_bounds(self) -> (Bound, Bound) { - (transmute_bound(self.0), transmute_bound(self.1)) + (convert_to_bound_key(self.0), convert_to_bound_key(self.1)) } } -fn transmute_bound(b: Bound) -> Bound +fn convert_to_bound_key(b: Bound) -> Bound where K: Into, { diff --git a/src/raw.rs b/src/raw.rs index bcef68b7..3e3a432e 100644 --- a/src/raw.rs +++ b/src/raw.rs @@ -11,16 +11,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -/*! Raw related functionality. - -Using the [`raw::Client`](struct.Client.html) you can utilize TiKV's raw interface. - -This interface offers optimal performance as it does not require coordination with a timestamp -oracle, while the transactional interface does. - -**Warning:** It is not advisible to use the both raw and transactional functionality in the same keyspace. - */ - +/// Raw related functionality. +/// +/// Using the [`raw::Client`](struct.Client.html) you can utilize TiKV's raw interface. +/// +/// This interface offers optimal performance as it does not require coordination with a timestamp +/// oracle, while the transactional interface does. +/// +/// **Warning:** It is not advisible to use the both raw and transactional functionality in the same keyspace. +/// use crate::{rpc::RpcClient, Config, Error, Key, KeyRange, KvFuture, KvPair, Result, Value}; use futures::{future, Async, Future, Poll}; use std::{ @@ -29,6 +28,8 @@ use std::{ u32, }; +const MAX_RAW_KV_SCAN_LIMIT: u32 = 10240; + /// The TiKV raw [`Client`](struct.Client.html) is used to issue requests to the TiKV server and PD cluster. pub struct Client { rpc: Arc, @@ -173,8 +174,6 @@ impl Client { /// /// Once resolved this request will result in a scanner over the given keys. /// - /// If not passed a `limit` parameter, it will default to `u32::MAX`. - /// /// ```rust,no_run /// # use tikv_client::{KvPair, Config, raw::Client}; /// # use futures::Future; @@ -184,19 +183,14 @@ impl Client { /// let req = connected_client.scan(inclusive_range, 2); /// let result: Vec = req.wait().unwrap(); /// ``` - pub fn scan(&self, range: impl KeyRange, limit: impl Into>) -> Scan { - Scan::new( - self.rpc(), - ScanInner::new(range.into_bounds(), limit.into().unwrap_or(u32::MAX)), - ) + pub fn scan(&self, range: impl KeyRange, limit: u32) -> Scan { + Scan::new(self.rpc(), ScanInner::new(range.into_bounds(), limit)) } /// Create a new [`BatchScan`](struct.BatchScan.html) request. /// /// Once resolved this request will result in a set of scanners over the given keys. /// - /// If not passed a `limit` parameter, it will default to `u32::MAX`. - /// /// ```rust,no_run /// # use tikv_client::{Key, Config, raw::Client}; /// # use futures::Future; @@ -211,13 +205,13 @@ impl Client { pub fn batch_scan( &self, ranges: impl IntoIterator, - each_limit: impl Into>, + each_limit: u32, ) -> BatchScan { BatchScan::new( self.rpc(), BatchScanInner::new( ranges.into_iter().map(KeyRange::into_keys).collect(), - each_limit.into().unwrap_or(u32::MAX), + each_limit, ), ) } @@ -684,11 +678,18 @@ impl RequestInner for ScanInner { type Resp = Vec; fn execute(self, client: Arc, cf: Option) -> KvFuture { - let keys = match self.range.into_keys() { - Err(e) => return Box::new(future::err(e)), - Ok(v) => v, - }; - Box::new(client.raw_scan(keys, self.limit, self.key_only, cf)) + if self.limit > MAX_RAW_KV_SCAN_LIMIT { + Box::new(future::err(Error::MaxScanLimitExceeded( + self.limit, + MAX_RAW_KV_SCAN_LIMIT, + ))) + } else { + let keys = match self.range.into_keys() { + Err(e) => return Box::new(future::err(e)), + Ok(v) => v, + }; + Box::new(client.raw_scan(keys, self.limit, self.key_only, cf)) + } } } @@ -713,10 +714,9 @@ impl Scan { } pub fn key_only(mut self) -> Self { - self.state.inner_mut().map(|mut x| { + if let Some(x) = self.state.inner_mut() { x.key_only = true; - x - }); + }; self } } @@ -750,13 +750,17 @@ impl RequestInner for BatchScanInner { type Resp = Vec; fn execute(self, client: Arc, cf: Option) -> KvFuture { - let (errors, ranges): (Vec<_>, Vec<_>) = self.ranges.into_iter().partition(Result::is_err); - if !errors.is_empty() { + if self.each_limit > MAX_RAW_KV_SCAN_LIMIT { + Box::new(future::err(Error::MaxScanLimitExceeded( + self.each_limit, + MAX_RAW_KV_SCAN_LIMIT, + ))) + } else if self.ranges.iter().any(Result::is_err) { // All errors must be InvalidKeyRange so we can simply return a new InvalidKeyRange Box::new(future::err(Error::InvalidKeyRange)) } else { Box::new(client.raw_batch_scan( - ranges.into_iter().map(Result::unwrap).collect(), + self.ranges.into_iter().map(Result::unwrap).collect(), self.each_limit, self.key_only, cf, @@ -786,10 +790,9 @@ impl BatchScan { } pub fn key_only(mut self) -> Self { - self.state.inner_mut().map(|mut x| { + if let Some(x) = self.state.inner_mut() { x.key_only = true; - x - }); + }; self } } diff --git a/src/rpc/pd/leader.rs b/src/rpc/pd/leader.rs index d24bacb2..5a1a9a31 100644 --- a/src/rpc/pd/leader.rs +++ b/src/rpc/pd/leader.rs @@ -423,10 +423,9 @@ pub fn try_connect_leader( if let Some(resp) = resp { let leader = resp.get_leader().clone(); for ep in leader.get_client_urls() { - if let Ok((client, r)) = - try_connect(&env, security_mgr, ep.as_str(), cluster_id, timeout) - { - return Ok((client, r)); + let r = try_connect(&env, security_mgr, ep.as_str(), cluster_id, timeout); + if r.is_ok() { + return r; } } } From 51be42603aa95c7e45739e44dd7faf3fd4c4e55b Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 10 Jan 2019 08:31:23 +0800 Subject: [PATCH 10/20] Change to dyn trait syntax Signed-off-by: Xiaoguang Sun --- src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/lib.rs b/src/lib.rs index 011c82f3..771442be 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -365,7 +365,7 @@ impl Config { } } -pub type KvFuture = Box>; +pub type KvFuture = Box>; /// A convenience trait for expressing ranges. /// From 08064e012f2b404b19578afa550cb92dcc51bf49 Mon Sep 17 00:00:00 2001 From: Ana Hobden Date: Thu, 10 Jan 2019 14:43:24 -0800 Subject: [PATCH 11/20] inline some functions Signed-off-by: Ana Hobden --- src/lib.rs | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/lib.rs b/src/lib.rs index 771442be..470e0c2d 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -76,18 +76,21 @@ pub use crate::errors::Result; pub struct Key(Vec); impl Key { + #[inline] pub fn new(value: Vec) -> Self { Key(value) } - + #[inline] fn into_inner(self) -> Vec { self.0 } + #[inline] fn push(&mut self, v: u8) { self.0.push(v) } + #[inline] fn pop(&mut self) { self.0.pop(); } @@ -180,10 +183,12 @@ impl DerefMut for Key { pub struct Value(Vec); impl Value { + #[inline] pub fn new(value: Vec) -> Self { Value(value) } + #[inline] fn into_inner(self) -> Vec { self.0 } @@ -236,48 +241,58 @@ pub struct KvPair(Key, Value); impl KvPair { /// Create a new `KvPair`. + #[inline] pub fn new(key: impl Into, value: impl Into) -> Self { KvPair(key.into(), value.into()) } /// Immutably borrow the `Key` part of the `KvPair`. + #[inline] pub fn key(&self) -> &Key { &self.0 } /// Immutably borrow the `Value` part of the `KvPair`. + #[inline] pub fn value(&self) -> &Value { &self.1 } + #[inline] pub fn into_inner(self) -> (Key, Value) { (self.0, self.1) } + #[inline] pub fn into_key(self) -> Key { self.0 } + #[inline] pub fn into_value(self) -> Value { self.1 } /// Mutably borrow the `Key` part of the `KvPair`. + #[inline] pub fn key_mut(&mut self) -> &mut Key { &mut self.0 } /// Mutably borrow the `Value` part of the `KvPair`. + #[inline] pub fn value_mut(&mut self) -> &mut Value { &mut self.1 } /// Set the `Key` part of the `KvPair`. + #[inline] pub fn set_key(&mut self, k: impl Into) { self.0 = k.into(); } /// Set the `Value` part of the `KvPair`. + #[inline] pub fn set_value(&mut self, v: impl Into) { self.1 = v.into(); } From 7d1dee71236b6bc604dba1298af257fcfb9239ca Mon Sep 17 00:00:00 2001 From: Ana Hobden Date: Thu, 10 Jan 2019 14:44:40 -0800 Subject: [PATCH 12/20] Better note on KvPair Signed-off-by: Ana Hobden --- src/lib.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index 470e0c2d..eddf5f68 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -233,9 +233,8 @@ impl Deref for Value { /// assert_eq!(constructed, from_tuple); /// ``` /// -/// **But, you should not need to worry about all this:** Many functions which accept a `KvPair` -/// accept an `Into`, which means all of the above types can be passed directly to those -/// functions. +/// Many functions which accept a `KvPair` accept an `Into`, which means all of the above +/// types (Like a `(Key, Value)`) can be passed directly to those functions. #[derive(Default, Clone, Eq, PartialEq, Debug)] pub struct KvPair(Key, Value); From da466d6e35f0a43d04bce369e4fad638c787dec7 Mon Sep 17 00:00:00 2001 From: Ana Hobden Date: Thu, 10 Jan 2019 15:14:39 -0800 Subject: [PATCH 13/20] Use 3 PDs in raw example Signed-off-by: Ana Hobden --- examples/raw.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/examples/raw.rs b/examples/raw.rs index 6fbf0d9b..5710a39a 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -23,8 +23,9 @@ fn main() -> Result<()> { // Create a configuration to use for the example. // Optionally encrypt the traffic. let config = Config::new(vec![ - "192.168.0.101:3379", // Avoid a single point of failure, - "192.168.0.100:3379", // use at least two PD endpoints. + "192.168.0.100:3379", // Avoid a single point of failure, + "192.168.0.101:3379", // use more than one PD endpoint. + "192.168.0.102:3379", ]) .with_security( PathBuf::from("/path/to/ca.pem"), From a7f762a0f2e704e7391300137d3be593d35ae309 Mon Sep 17 00:00:00 2001 From: Ana Hobden Date: Thu, 10 Jan 2019 15:21:55 -0800 Subject: [PATCH 14/20] Clarify documentation Signed-off-by: Ana Hobden --- src/lib.rs | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index eddf5f68..53d02b2f 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -316,9 +316,14 @@ where /// It's important to **include more than one PD endpoint** (include all, if possible!) /// This helps avoid having a *single point of failure*. /// -/// By default, this client will use an insecure connection over encryption-on-the-wire. Your -/// deployment may have chosen to rely on security measures such as a private network, or a VPN -/// layer providing secure transmission. TiKV does not currently offer encryption-at-rest. +/// By default, this client will use an insecure connection over instead of one protected by +/// Transport Layer Security (TLS). Your deployment may have chosen to rely on security measures +/// such as a private network, or a VPN layer to provid secure transmission. +/// +/// To use a TLS secured connection, use the `with_security` function to set the required +/// parameters. +/// +/// TiKV does not currently offer encrypted storage (or encryption-at-rest). #[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq)] #[serde(default)] #[serde(rename_all = "kebab-case")] @@ -352,9 +357,11 @@ impl Config { } } - /// Set the certificate authority, certificate, and key locations for the [`Config`](struct.Config.html). + /// Set the certificate authority, certificate, and key locations for the + /// [`Config`](struct.Config.html). /// - /// By default, TiKV connections do not utilize transport layer security. Enable it by setting these values. + /// By default, TiKV connections do not utilize transport layer security. Enable it by setting + /// these values. /// /// ```rust /// # use tikv_client::Config; From 19fec55fba27504e8fea9d6a604dbf2af1b24f85 Mon Sep 17 00:00:00 2001 From: Ana Hobden Date: Fri, 11 Jan 2019 20:35:15 +0000 Subject: [PATCH 15/20] Get CI green Signed-off-by: Ana Hobden --- .travis.yml | 6 ++++++ src/lib.rs | 6 +++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index 8e51d238..66225120 100644 --- a/.travis.yml +++ b/.travis.yml @@ -16,6 +16,10 @@ env: cache: cargo rust: +os: + - linux + - windows + - osx matrix: include: @@ -33,6 +37,8 @@ matrix: script: + - docker run -d --net=host --name pd --rm pingcap/pd --name "pd" --data-dir "pd" --client-urls "http://127.0.0.1:2379" --advertise-client-urls "http://127.0.0.1:2379" + - docker run -d --net=host --name kv --rm pingcap/tikv --pd-endpoints "127.0.0.1:2379" --addr "127.0.0.1:2378" --data-dir "kv" - cargo test --all -- --nocapture # Validate benches still work. - cargo bench --all -- --test diff --git a/src/lib.rs b/src/lib.rs index 53d02b2f..58d026ee 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -316,13 +316,13 @@ where /// It's important to **include more than one PD endpoint** (include all, if possible!) /// This helps avoid having a *single point of failure*. /// -/// By default, this client will use an insecure connection over instead of one protected by +/// By default, this client will use an insecure connection over instead of one protected by /// Transport Layer Security (TLS). Your deployment may have chosen to rely on security measures /// such as a private network, or a VPN layer to provid secure transmission. -/// +/// /// To use a TLS secured connection, use the `with_security` function to set the required /// parameters. -/// +/// /// TiKV does not currently offer encrypted storage (or encryption-at-rest). #[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq)] #[serde(default)] From 8994e055a688a9f599102c4b0fb7ae1ce39b23dd Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jan 2019 18:14:42 +0800 Subject: [PATCH 16/20] Remove not useful PrivateKey type Signed-off-by: Xiaoguang Sun --- src/rpc/security.rs | 80 ++++++++++++--------------------------------- src/rpc/util.rs | 2 +- tests/raw.rs | 15 +++++---- 3 files changed, 30 insertions(+), 67 deletions(-) diff --git a/src/rpc/security.rs b/src/rpc/security.rs index 52503831..7d849535 100644 --- a/src/rpc/security.rs +++ b/src/rpc/security.rs @@ -15,7 +15,6 @@ use std::{ fs::File, io::Read, path::{Path, PathBuf}, - ptr, sync::Arc, time::Duration, }; @@ -25,60 +24,24 @@ use log::*; use crate::Result; -fn check_pem_file(tag: &str, path: &Path) -> Result> { - match File::open(path) { - Err(e) => Err(internal_err!( - "failed to open {} to load {}: {:?}", - path.display(), - tag, - e - )), - Ok(f) => Ok(Some(f)), - } +fn check_pem_file(tag: &str, path: &Path) -> Result { + File::open(path) + .map_err(|e| internal_err!("failed to open {} to load {}: {:?}", path.display(), tag, e)) } -fn load_pem(tag: &str, path: &Path) -> Result> { +fn load_pem_file(tag: &str, path: &Path) -> Result> { + let mut file = check_pem_file(tag, path)?; let mut key = vec![]; - let f = check_pem_file(tag, path)?; - match f { - None => return Ok(vec![]), - Some(mut f) => { - if let Err(e) = f.read_to_end(&mut key) { - return Err(internal_err!( - "failed to load {} from path {}: {:?}", - tag, - path.display(), - e - )); - } - } - } - Ok(key) -} - -struct PrivateKey(Vec); - -impl PrivateKey { - fn load(path: &Path) -> Result { - let key = load_pem("private key", path)?; - Ok(PrivateKey(key)) - } -} - -impl Drop for PrivateKey { - fn drop(&mut self) { - unsafe { - for b in &mut self.0 { - ptr::write_volatile(b, 0); - } - } - } -} - -impl PrivateKey { - fn get(&self) -> Vec { - self.0.clone() - } + file.read_to_end(&mut key) + .map_err(|e| { + internal_err!( + "failed to load {} from path {}: {:?}", + tag, + path.display(), + e + ) + }) + .map(|_| key) } #[derive(Default)] @@ -95,10 +58,10 @@ impl SecurityManager { key_path: impl Into, ) -> Result { let key_path = key_path.into(); - let _ = PrivateKey::load(&key_path)?; + check_pem_file("private key", &key_path)?; Ok(SecurityManager { - ca: load_pem("ca", ca_path.as_ref())?, - cert: load_pem("certificate", cert_path.as_ref())?, + ca: load_pem_file("ca", ca_path.as_ref())?, + cert: load_pem_file("certificate", cert_path.as_ref())?, key: key_path, }) } @@ -123,10 +86,9 @@ impl SecurityManager { let channel = if self.ca.is_empty() { cb.connect(addr) } else { - let key = PrivateKey::load(&self.key)?; let cred = ChannelCredentialsBuilder::new() .root_cert(self.ca.clone()) - .cert(self.cert.clone(), key.get()) + .cert(self.cert.clone(), load_pem_file("private key", &self.key)?) .build(); cb.secure_connect(addr, cred) }; @@ -164,7 +126,7 @@ mod tests { let mgr = SecurityManager::load(&ca_path, &cert_path, &key_path).unwrap(); assert_eq!(mgr.ca, vec![0]); assert_eq!(mgr.cert, vec![1]); - let key = PrivateKey::load(&key_path).unwrap(); - assert_eq!(key.get(), vec![2]); + let key = load_pem_file("private key", &key_path).unwrap(); + assert_eq!(key, vec![2]); } } diff --git a/src/rpc/util.rs b/src/rpc/util.rs index 76ae297e..42c0c5fe 100644 --- a/src/rpc/util.rs +++ b/src/rpc/util.rs @@ -115,7 +115,7 @@ mod tests { } } - #[cfg_attr(feature = "cargo-clippy", allow(clone_on_copy))] + #[cfg_attr(feature = "cargo-clippy", allow(clippy::clone_on_copy))] fn foo(a: &Option) -> Option { a.clone() } diff --git a/tests/raw.rs b/tests/raw.rs index 46e85b54..12b71a12 100644 --- a/tests/raw.rs +++ b/tests/raw.rs @@ -52,11 +52,11 @@ fn test_empty(client: &Client) { .is_empty()); } -fn test_existence(client: &Client, existing_pairs: Vec, not_existing_keys: Vec) { +fn test_existence(client: &Client, existing_pairs: &[KvPair], not_existing_keys: Vec) { let test_key_start = generate_key(0); let test_key_end = generate_key(NUM_TEST_KEYS as i32 - 1); - for pair in existing_pairs.clone().into_iter() { + for pair in existing_pairs.iter().map(Clone::clone) { let (key, value) = pair.into_inner(); assert_eq!( client.get(key).wait().expect("Could not get value"), @@ -126,14 +126,15 @@ fn basic_raw_test() { .is_ok()); test_existence( &client, - vec![KvPair::new(generate_key(0), generate_value(0))], + &[KvPair::new(generate_key(0), generate_value(0))], vec![generate_key(1), generate_key(2)], ); + let empty_pairs = Vec::new(); assert!(client.delete(generate_key(0)).wait().is_ok()); test_existence( &client, - Vec::new(), + &empty_pairs, vec![generate_key(0), generate_key(1), generate_key(2)], ); @@ -143,7 +144,7 @@ fn basic_raw_test() { assert!(client.batch_put(pairs.clone()).wait().is_ok()); test_existence( &client, - pairs.clone(), + &pairs, vec![generate_key(10), generate_key(11), generate_key(12)], ); @@ -153,14 +154,14 @@ fn basic_raw_test() { pairs.truncate(8); test_existence( &client, - pairs.clone(), + &pairs, vec![generate_key(8), generate_key(9), generate_key(10)], ); wipe_all(&client); test_existence( &client, - Vec::new(), + &empty_pairs, pairs.into_iter().map(|x| x.into_inner().0).collect(), ); } From 54ddc6aff0c686b914b6714e5391b6c4066f4d0c Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jan 2019 18:33:58 +0800 Subject: [PATCH 17/20] Change CUSTOM_CF to "default" in examples/raw.rs Signed-off-by: Xiaoguang Sun --- examples/raw.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/raw.rs b/examples/raw.rs index 5710a39a..b65d88d5 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -17,7 +17,7 @@ use tikv_client::{raw::Client, Config, Key, KvPair, Result, Value}; const KEY: &str = "TiKV"; const VALUE: &str = "Rust"; -const CUSTOM_CF: &str = "custom_cf"; +const CUSTOM_CF: &str = "default"; fn main() -> Result<()> { // Create a configuration to use for the example. From 1dea8eefe28b5b721b1594862bbcb8ad29244cb0 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Sun, 13 Jan 2019 00:09:39 +0800 Subject: [PATCH 18/20] Use command line args in examples. Signed-off-by: Yilin Chen --- Cargo.toml | 1 + examples/raw.rs | 79 +++++++++++++++++++++++++++++++++++------ examples/transaction.rs | 77 ++++++++++++++++++++++++++++++++++++--- 3 files changed, 142 insertions(+), 15 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 078b6a0b..b490850d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,3 +34,4 @@ features = ["push", "process"] [dev-dependencies] tempdir = "0.3" +clap = "2.32" \ No newline at end of file diff --git a/examples/raw.rs b/examples/raw.rs index b65d88d5..69be36ad 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -11,6 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use clap::{crate_version, App, Arg}; use futures::future::Future; use std::path::PathBuf; use tikv_client::{raw::Client, Config, Key, KvPair, Result, Value}; @@ -20,18 +21,15 @@ const VALUE: &str = "Rust"; const CUSTOM_CF: &str = "default"; fn main() -> Result<()> { + let (pd, security) = parse_args(); + // Create a configuration to use for the example. // Optionally encrypt the traffic. - let config = Config::new(vec![ - "192.168.0.100:3379", // Avoid a single point of failure, - "192.168.0.101:3379", // use more than one PD endpoint. - "192.168.0.102:3379", - ]) - .with_security( - PathBuf::from("/path/to/ca.pem"), - PathBuf::from("/path/to/client.pem"), - PathBuf::from("/path/to/client-key.pem"), - ); + let config = if let Some((ca, cert, key)) = security { + Config::new(pd).with_security(ca, cert, key) + } else { + Config::new(pd) + }; // When we first create a client we recieve a `Connect` structure which must be resolved before // the client is actually connected and usable. @@ -114,3 +112,64 @@ fn main() -> Result<()> { // Cleanly exit. Ok(()) } + +fn parse_args() -> (Vec, Option<(PathBuf, PathBuf, PathBuf)>) { + let matches = App::new("Raw API Example of the Rust Client for TiKV") + .version(crate_version!()) + .author("The TiKV Project Authors") + .arg( + Arg::with_name("pd") + .long("pd") + .aliases(&["pd-endpoint", "pd-endpoints"]) + .value_name("PD_URL") + .help("Sets PD endpoints") + .long_help("Sets PD endpoints. Uses `,` to separate multiple PDs") + .takes_value(true) + .multiple(true) + .value_delimiter(",") + .required(true), + ) + // A cyclic dependency between CA, cert and key is made + // to ensure that no security options are missing. + .arg( + Arg::with_name("ca") + .long("ca") + .value_name("CA_PATH") + .help("Sets the CA") + .long_help("Sets the CA. Must be used with --cert and --key") + .takes_value(true) + .requires("cert"), + ) + .arg( + Arg::with_name("cert") + .long("cert") + .value_name("CERT_PATH") + .help("Sets the certificate") + .long_help("Sets the certificate. Must be used with --ca and --key") + .takes_value(true) + .requires("key"), + ) + .arg( + Arg::with_name("key") + .long("key") + .alias("private-key") + .value_name("KEY_PATH") + .help("Sets the private key") + .long_help("Sets the private key. Must be used with --ca and --cert") + .takes_value(true) + .requires("ca"), + ) + .get_matches(); + + let pd: Vec<_> = matches.values_of("pd").unwrap().map(String::from).collect(); + let security = if let (Some(ca), Some(cert), Some(key)) = ( + matches.value_of("ca"), + matches.value_of("cert"), + matches.value_of("key"), + ) { + Some((PathBuf::from(ca), PathBuf::from(cert), PathBuf::from(key))) + } else { + None + }; + (pd, security) +} diff --git a/examples/transaction.rs b/examples/transaction.rs index f2e7a567..c6f94453 100644 --- a/examples/transaction.rs +++ b/examples/transaction.rs @@ -11,6 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use clap::{crate_version, App, Arg}; use futures::{future, Future, Stream}; use std::ops::RangeBounds; use std::path::PathBuf; @@ -70,11 +71,16 @@ fn dels(client: &Client, keys: impl IntoIterator) { } fn main() { - let config = Config::new(vec!["127.0.0.1:2379"]).with_security( - PathBuf::from("/path/to/ca.pem"), - PathBuf::from("/path/to/client.pem"), - PathBuf::from("/path/to/client-key.pem"), - ); + let (pd, security) = parse_args(); + + // Create a configuration to use for the example. + // Optionally encrypt the traffic. + let config = if let Some((ca, cert, key)) = security { + Config::new(pd).with_security(ca, cert, key) + } else { + Config::new(pd) + }; + let txn = Client::new(&config) .wait() .expect("Could not connect to tikv"); @@ -100,3 +106,64 @@ fn main() { let key2: Key = b"key2".to_vec().into(); dels(&txn, vec![key1, key2]); } + +fn parse_args() -> (Vec, Option<(PathBuf, PathBuf, PathBuf)>) { + let matches = App::new("Raw API Example of the Rust Client for TiKV") + .version(crate_version!()) + .author("The TiKV Project Authors") + .arg( + Arg::with_name("pd") + .long("pd") + .aliases(&["pd-endpoint", "pd-endpoints"]) + .value_name("PD_URL") + .help("Sets PD endpoints") + .long_help("Sets PD endpoints. Uses `,` to separate multiple PDs") + .takes_value(true) + .multiple(true) + .value_delimiter(",") + .required(true), + ) + // A cyclic dependency between CA, cert and key is made + // to ensure that no security options are missing. + .arg( + Arg::with_name("ca") + .long("ca") + .value_name("CA_PATH") + .help("Sets the CA") + .long_help("Sets the CA. Must be used with --cert and --key") + .takes_value(true) + .requires("cert"), + ) + .arg( + Arg::with_name("cert") + .long("cert") + .value_name("CERT_PATH") + .help("Sets the certificate") + .long_help("Sets the certificate. Must be used with --ca and --key") + .takes_value(true) + .requires("key"), + ) + .arg( + Arg::with_name("key") + .long("key") + .alias("private-key") + .value_name("KEY_PATH") + .help("Sets the private key") + .long_help("Sets the private key. Must be used with --ca and --cert") + .takes_value(true) + .requires("ca"), + ) + .get_matches(); + + let pd: Vec<_> = matches.values_of("pd").unwrap().map(String::from).collect(); + let security = if let (Some(ca), Some(cert), Some(key)) = ( + matches.value_of("ca"), + matches.value_of("cert"), + matches.value_of("key"), + ) { + Some((PathBuf::from(ca), PathBuf::from(cert), PathBuf::from(key))) + } else { + None + }; + (pd, security) +} From 2ed808babcda7544dee758761d63dfdea96d0216 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Sun, 13 Jan 2019 01:06:29 +0800 Subject: [PATCH 19/20] Fix the wrong app name in the transactional API example. Signed-off-by: Yilin Chen --- examples/transaction.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/transaction.rs b/examples/transaction.rs index c6f94453..c676ce2a 100644 --- a/examples/transaction.rs +++ b/examples/transaction.rs @@ -108,7 +108,7 @@ fn main() { } fn parse_args() -> (Vec, Option<(PathBuf, PathBuf, PathBuf)>) { - let matches = App::new("Raw API Example of the Rust Client for TiKV") + let matches = App::new("Transactional API Example of the Rust Client for TiKV") .version(crate_version!()) .author("The TiKV Project Authors") .arg( From 1daf02e8e44b06c7e8352b214f9928399675597c Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Sun, 13 Jan 2019 15:22:42 +0800 Subject: [PATCH 20/20] Extract duplicate code to a common mod Signed-off-by: Yilin Chen --- examples/common/mod.rs | 65 +++++++++++++++++++++++++++++++++++ examples/raw.rs | 76 +++++------------------------------------ examples/transaction.rs | 76 +++++------------------------------------ 3 files changed, 83 insertions(+), 134 deletions(-) create mode 100644 examples/common/mod.rs diff --git a/examples/common/mod.rs b/examples/common/mod.rs new file mode 100644 index 00000000..37580594 --- /dev/null +++ b/examples/common/mod.rs @@ -0,0 +1,65 @@ +use clap::{crate_version, App, Arg}; +use std::path::PathBuf; + +pub struct CommandArgs { + pub pd: Vec, + pub ca: Option, + pub cert: Option, + pub key: Option, +} + +pub fn parse_args(app_name: &str) -> CommandArgs { + let matches = App::new(app_name) + .version(crate_version!()) + .author("The TiKV Project Authors") + .arg( + Arg::with_name("pd") + .long("pd") + .aliases(&["pd-endpoint", "pd-endpoints"]) + .value_name("PD_URL") + .help("Sets PD endpoints") + .long_help("Sets PD endpoints. Uses `,` to separate multiple PDs") + .takes_value(true) + .multiple(true) + .value_delimiter(",") + .required(true), + ) + // A cyclic dependency between CA, cert and key is made + // to ensure that no security options are missing. + .arg( + Arg::with_name("ca") + .long("ca") + .value_name("CA_PATH") + .help("Sets the CA") + .long_help("Sets the CA. Must be used with --cert and --key") + .takes_value(true) + .requires("cert"), + ) + .arg( + Arg::with_name("cert") + .long("cert") + .value_name("CERT_PATH") + .help("Sets the certificate") + .long_help("Sets the certificate. Must be used with --ca and --key") + .takes_value(true) + .requires("key"), + ) + .arg( + Arg::with_name("key") + .long("key") + .alias("private-key") + .value_name("KEY_PATH") + .help("Sets the private key") + .long_help("Sets the private key. Must be used with --ca and --cert") + .takes_value(true) + .requires("ca"), + ) + .get_matches(); + + CommandArgs { + pd: matches.values_of("pd").unwrap().map(String::from).collect(), + ca: matches.value_of("ca").map(PathBuf::from), + cert: matches.value_of("cert").map(PathBuf::from), + key: matches.value_of("key").map(PathBuf::from), + } +} diff --git a/examples/raw.rs b/examples/raw.rs index 69be36ad..b379805f 100644 --- a/examples/raw.rs +++ b/examples/raw.rs @@ -11,9 +11,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use clap::{crate_version, App, Arg}; +mod common; + +use crate::common::parse_args; use futures::future::Future; -use std::path::PathBuf; use tikv_client::{raw::Client, Config, Key, KvPair, Result, Value}; const KEY: &str = "TiKV"; @@ -21,14 +22,16 @@ const VALUE: &str = "Rust"; const CUSTOM_CF: &str = "default"; fn main() -> Result<()> { - let (pd, security) = parse_args(); + // You can try running this example by passing your pd endpoints + // (and SSL options if necessary) through command line arguments. + let args = parse_args("raw"); // Create a configuration to use for the example. // Optionally encrypt the traffic. - let config = if let Some((ca, cert, key)) = security { - Config::new(pd).with_security(ca, cert, key) + let config = if let (Some(ca), Some(cert), Some(key)) = (args.ca, args.cert, args.key) { + Config::new(args.pd).with_security(ca, cert, key) } else { - Config::new(pd) + Config::new(args.pd) }; // When we first create a client we recieve a `Connect` structure which must be resolved before @@ -112,64 +115,3 @@ fn main() -> Result<()> { // Cleanly exit. Ok(()) } - -fn parse_args() -> (Vec, Option<(PathBuf, PathBuf, PathBuf)>) { - let matches = App::new("Raw API Example of the Rust Client for TiKV") - .version(crate_version!()) - .author("The TiKV Project Authors") - .arg( - Arg::with_name("pd") - .long("pd") - .aliases(&["pd-endpoint", "pd-endpoints"]) - .value_name("PD_URL") - .help("Sets PD endpoints") - .long_help("Sets PD endpoints. Uses `,` to separate multiple PDs") - .takes_value(true) - .multiple(true) - .value_delimiter(",") - .required(true), - ) - // A cyclic dependency between CA, cert and key is made - // to ensure that no security options are missing. - .arg( - Arg::with_name("ca") - .long("ca") - .value_name("CA_PATH") - .help("Sets the CA") - .long_help("Sets the CA. Must be used with --cert and --key") - .takes_value(true) - .requires("cert"), - ) - .arg( - Arg::with_name("cert") - .long("cert") - .value_name("CERT_PATH") - .help("Sets the certificate") - .long_help("Sets the certificate. Must be used with --ca and --key") - .takes_value(true) - .requires("key"), - ) - .arg( - Arg::with_name("key") - .long("key") - .alias("private-key") - .value_name("KEY_PATH") - .help("Sets the private key") - .long_help("Sets the private key. Must be used with --ca and --cert") - .takes_value(true) - .requires("ca"), - ) - .get_matches(); - - let pd: Vec<_> = matches.values_of("pd").unwrap().map(String::from).collect(); - let security = if let (Some(ca), Some(cert), Some(key)) = ( - matches.value_of("ca"), - matches.value_of("cert"), - matches.value_of("key"), - ) { - Some((PathBuf::from(ca), PathBuf::from(cert), PathBuf::from(key))) - } else { - None - }; - (pd, security) -} diff --git a/examples/transaction.rs b/examples/transaction.rs index c676ce2a..2362470d 100644 --- a/examples/transaction.rs +++ b/examples/transaction.rs @@ -11,10 +11,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use clap::{crate_version, App, Arg}; +mod common; + +use crate::common::parse_args; use futures::{future, Future, Stream}; use std::ops::RangeBounds; -use std::path::PathBuf; use tikv_client::{ transaction::{Client, IsolationLevel}, Config, Key, KvPair, Value, @@ -71,14 +72,16 @@ fn dels(client: &Client, keys: impl IntoIterator) { } fn main() { - let (pd, security) = parse_args(); + // You can try running this example by passing your pd endpoints + // (and SSL options if necessary) through command line arguments. + let args = parse_args("txn"); // Create a configuration to use for the example. // Optionally encrypt the traffic. - let config = if let Some((ca, cert, key)) = security { - Config::new(pd).with_security(ca, cert, key) + let config = if let (Some(ca), Some(cert), Some(key)) = (args.ca, args.cert, args.key) { + Config::new(args.pd).with_security(ca, cert, key) } else { - Config::new(pd) + Config::new(args.pd) }; let txn = Client::new(&config) @@ -106,64 +109,3 @@ fn main() { let key2: Key = b"key2".to_vec().into(); dels(&txn, vec![key1, key2]); } - -fn parse_args() -> (Vec, Option<(PathBuf, PathBuf, PathBuf)>) { - let matches = App::new("Transactional API Example of the Rust Client for TiKV") - .version(crate_version!()) - .author("The TiKV Project Authors") - .arg( - Arg::with_name("pd") - .long("pd") - .aliases(&["pd-endpoint", "pd-endpoints"]) - .value_name("PD_URL") - .help("Sets PD endpoints") - .long_help("Sets PD endpoints. Uses `,` to separate multiple PDs") - .takes_value(true) - .multiple(true) - .value_delimiter(",") - .required(true), - ) - // A cyclic dependency between CA, cert and key is made - // to ensure that no security options are missing. - .arg( - Arg::with_name("ca") - .long("ca") - .value_name("CA_PATH") - .help("Sets the CA") - .long_help("Sets the CA. Must be used with --cert and --key") - .takes_value(true) - .requires("cert"), - ) - .arg( - Arg::with_name("cert") - .long("cert") - .value_name("CERT_PATH") - .help("Sets the certificate") - .long_help("Sets the certificate. Must be used with --ca and --key") - .takes_value(true) - .requires("key"), - ) - .arg( - Arg::with_name("key") - .long("key") - .alias("private-key") - .value_name("KEY_PATH") - .help("Sets the private key") - .long_help("Sets the private key. Must be used with --ca and --cert") - .takes_value(true) - .requires("ca"), - ) - .get_matches(); - - let pd: Vec<_> = matches.values_of("pd").unwrap().map(String::from).collect(); - let security = if let (Some(ca), Some(cert), Some(key)) = ( - matches.value_of("ca"), - matches.value_of("cert"), - matches.value_of("key"), - ) { - Some((PathBuf::from(ca), PathBuf::from(cert), PathBuf::from(key))) - } else { - None - }; - (pd, security) -}