├── .gitignore ├── .dockerignore ├── peers.txt ├── Dockerfile ├── proto ├── wal │ └── wal.proto └── kv │ ├── kv.proto │ └── eraftpb.proto ├── Procfile ├── Makefile ├── benches └── wal.rs ├── Cargo.toml ├── src ├── wal.rs ├── lib.rs ├── bin │ ├── kv_server.rs │ └── kv_client.rs ├── pb.rs ├── storage.rs ├── network.rs └── node.rs └── Cargo.lock /.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | **/*.rs.bk 3 | -------------------------------------------------------------------------------- /.dockerignore: -------------------------------------------------------------------------------- 1 | * 2 | !target/bin/kv-server 3 | -------------------------------------------------------------------------------- /peers.txt: -------------------------------------------------------------------------------- 1 | 0.0.0.0:12345 2 | 0.0.0.0:12346 3 | 0.0.0.0:12347 4 | -------------------------------------------------------------------------------- /Dockerfile: -------------------------------------------------------------------------------- 1 | FROM alpine 2 | 3 | COPY target/bin/kv-server /usr/local/bin 4 | 5 | ENTRYPOINT ["kv-server"] 6 | -------------------------------------------------------------------------------- /proto/wal/wal.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package wal; 4 | 5 | message Record { 6 | uint32 type = 1; 7 | bytes data = 2; 8 | } 9 | -------------------------------------------------------------------------------- /Procfile: -------------------------------------------------------------------------------- 1 | node1: cargo run --bin kv-server bootstrap 1 2 | node2: cargo run --bin kv-server join 2 1 3 | node3: cargo run --bin kv-server join 3 1 4 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | run: 2 | @mkdir -p target/bin 3 | @cargo build --target x86_64-unknown-linux-musl 4 | @cp target/x86_64-unknown-linux-musl/debug/kv-server target/bin 5 | @cp target/x86_64-unknown-linux-musl/debug/kv-client target/bin 6 | @docker build -t kv . 7 | @docker run -it --rm kv 8 | -------------------------------------------------------------------------------- /proto/kv/kv.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package kv; 4 | 5 | import "eraftpb.proto"; 6 | import "google/protobuf/empty.proto"; 7 | 8 | service Kv { 9 | // Required to get connection going. 10 | rpc Handshake(google.protobuf.Empty) returns (google.protobuf.Empty); 11 | 12 | rpc Join(JoinRequest) returns (JoinResponse); 13 | 14 | rpc Put(PutRequest) returns (PutResponse); 15 | rpc Range(RangeRequest) returns (RangeResponse); 16 | 17 | rpc Raft(stream Message) returns (google.protobuf.Empty); 18 | } 19 | 20 | message PutRequest { 21 | bytes key = 1; 22 | bytes value = 2; 23 | } 24 | 25 | message PutResponse {} 26 | 27 | message RangeRequest { 28 | bytes key = 1; 29 | bytes range_end = 2; 30 | } 31 | 32 | message RangeResponse { 33 | repeated KeyValue kvs = 1; 34 | } 35 | 36 | message KeyValue { 37 | bytes key = 1; 38 | bytes value = 2; 39 | } 40 | 41 | message JoinRequest { 42 | uint64 id = 1; 43 | } 44 | 45 | message JoinResponse { 46 | bool joined = 1; 47 | } 48 | 49 | message InternalRaftMessage { 50 | PutRequest put = 1; 51 | } 52 | -------------------------------------------------------------------------------- /benches/wal.rs: -------------------------------------------------------------------------------- 1 | use criterion::{criterion_group, criterion_main, Criterion}; 2 | use kv::{pb::wal as pb, wal::Log}; 3 | 4 | fn criterion_benchmark(c: &mut Criterion) { 5 | c.bench_function("save_entry", |b| { 6 | b.iter_with_setup( 7 | || { 8 | let temp_dir = std::env::temp_dir(); 9 | let wal = Log::new(temp_dir).unwrap(); 10 | let data = vec![0u8; 1024]; 11 | 12 | let entry = pb::Record { 13 | data, 14 | ..Default::default() 15 | }; 16 | 17 | let entries = (0..100) 18 | .into_iter() 19 | .map(|_| entry.clone()) 20 | .collect::>(); 21 | 22 | (wal, entries) 23 | }, 24 | |(mut wal, entries)| { 25 | for entry in entries { 26 | wal.save(entry).unwrap(); 27 | } 28 | }, 29 | ) 30 | }); 31 | } 32 | 33 | criterion_group!(benches, criterion_benchmark); 34 | criterion_main!(benches); 35 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "kv" 3 | version = "0.1.0" 4 | authors = ["Lucio Franco "] 5 | edition = "2018" 6 | 7 | # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html 8 | 9 | [[bin]] 10 | name = "kv-client" 11 | path = "src/bin/kv_client.rs" 12 | 13 | [[bin]] 14 | name = "kv-server" 15 | path = "src/bin/kv_server.rs" 16 | 17 | [[bench]] 18 | name = "wal" 19 | harness = false 20 | 21 | [dependencies] 22 | raft = { git = "https://github.com/pingcap/raft-rs", default-features = false, features = ["prost-codec"] } 23 | tonic = { git = "https://github.com/hyperium/tonic" } 24 | prost = "0.5" 25 | bytes = "0.4" 26 | tokio = { version = "0.2.0-alpha.6", features = ["signal"] } 27 | futures-util-preview = { version = "0.3.0-alpha.19" } 28 | slog = "2.5" 29 | tracing = "0.1" 30 | tracing-log = "0.1" 31 | tracing-subscriber = "0.1" 32 | tracing-futures = { version = "0.0.1-alpha.1", default-features = false, features = ["std-future"] } 33 | structopt = "0.2" 34 | async-stream = "0.1" 35 | sled = "0.28" 36 | uuid = { version = "0.7", features = ["v4"] } 37 | 38 | [dev-dependencies] 39 | criterion = "0.3" 40 | 41 | [build-dependencies] 42 | tonic-build = { git = "https://github.com/hyperium/tonic" } 43 | prost-build = "0.5" 44 | -------------------------------------------------------------------------------- /src/wal.rs: -------------------------------------------------------------------------------- 1 | use crate::pb::{self, wal as walpb}; 2 | use bytes::{BufMut, BytesMut}; 3 | use prost::Message; 4 | use std::{ 5 | fs::{self, File, OpenOptions}, 6 | io::Write, 7 | path::{Path, PathBuf}, 8 | }; 9 | 10 | pub struct Log { 11 | dir: PathBuf, 12 | file: File, 13 | 14 | last_entry: u64, 15 | 16 | buf: BytesMut, 17 | } 18 | 19 | impl Log { 20 | pub fn new(dir: impl AsRef) -> Result { 21 | fs::create_dir_all(&dir)?; 22 | 23 | let file_path = dir.as_ref().join(wal_name(0, 0)); 24 | 25 | let file = OpenOptions::new() 26 | .write(true) 27 | .append(true) 28 | .create(true) 29 | .open(file_path)?; 30 | 31 | let buf = BytesMut::with_capacity(1024 * 64); 32 | 33 | Ok(Self { 34 | dir: dir.as_ref().to_path_buf(), 35 | file, 36 | buf, 37 | last_entry: 0, 38 | }) 39 | } 40 | 41 | pub fn save_entry(&mut self, entry: pb::Entry) -> Result<(), crate::Error> { 42 | 43 | } 44 | 45 | pub fn save(&mut self, data: walpb::Record) -> Result<(), crate::Error> { 46 | if self.buf.remaining_mut() > data.encoded_len() { 47 | self.buf.clear(); 48 | } 49 | 50 | data.encode(&mut self.buf)?; 51 | 52 | self.file.write_all(&self.buf[..])?; 53 | 54 | Ok(()) 55 | } 56 | } 57 | 58 | fn wal_name(seq: usize, index: usize) -> String { 59 | format!("{:016}-{:016}", seq, index) 60 | } 61 | -------------------------------------------------------------------------------- /src/lib.rs: -------------------------------------------------------------------------------- 1 | mod network; 2 | mod node; 3 | mod storage; 4 | 5 | pub mod pb; 6 | pub mod wal; 7 | 8 | pub use network::{Peers, Server}; 9 | pub use node::Node; 10 | pub use storage::{MemStorage, SledStorage}; 11 | 12 | pub type Error = Box; 13 | 14 | use std::{future::Future, net::SocketAddr, path::Path}; 15 | use tracing::debug; 16 | use tracing_futures::Instrument; 17 | 18 | pub async fn load_peers(file: impl AsRef) -> Result { 19 | let bytes = tokio::fs::read(file).await?; 20 | 21 | let s = String::from_utf8(bytes)?; 22 | 23 | let mut peers = std::collections::HashMap::new(); 24 | 25 | for (id, line) in s.lines().enumerate() { 26 | let addr = line.parse::()?; 27 | peers.insert(id as u64 + 1, addr); 28 | } 29 | 30 | debug!(message = "Loaded peers.", ?peers); 31 | 32 | Ok(std::sync::Arc::new(peers)) 33 | } 34 | 35 | pub fn spawn(f: impl Future + Send + 'static, span: tracing::Span) { 36 | tokio::spawn(f.instrument(span)); 37 | } 38 | 39 | pub fn init_tracing(name: &'static str) -> Result<(), Error> { 40 | if let Ok(_) = std::env::var("RUST_LOG") { 41 | let subscriber = tracing_subscriber::fmt::Subscriber::builder() 42 | .with_env_filter(tracing_subscriber::filter::EnvFilter::from_default_env()) 43 | .finish(); 44 | 45 | tracing::subscriber::set_global_default(subscriber)?; 46 | } else { 47 | let subscriber = tracing_subscriber::fmt::Subscriber::builder() 48 | .with_env_filter(format!("kv=info,{}=info", name)) 49 | .finish(); 50 | 51 | tracing::subscriber::set_global_default(subscriber)?; 52 | } 53 | 54 | tracing_log::LogTracer::init()?; 55 | 56 | Ok(()) 57 | } 58 | -------------------------------------------------------------------------------- /src/bin/kv_server.rs: -------------------------------------------------------------------------------- 1 | use futures_util::StreamExt; 2 | use kv::{Error, MemStorage, Node, Server}; 3 | use raft::eraftpb::ConfState; 4 | use structopt::StructOpt; 5 | use tokio::net::signal; 6 | use tracing::{error, info, info_span}; 7 | use tracing_futures::Instrument; 8 | 9 | #[derive(Clone, Debug, StructOpt)] 10 | #[structopt(name = "kv server", about = "A consitient kv store.")] 11 | #[allow(non_camel_case_types)] 12 | enum Opts { 13 | bootstrap { id: u64 }, 14 | join { id: u64, target: u64 }, 15 | } 16 | 17 | #[tokio::main] 18 | async fn main() -> Result<(), Error> { 19 | let opts = Opts::from_args(); 20 | 21 | kv::init_tracing("kv_server")?; 22 | 23 | info!("Starting the KV Store."); 24 | 25 | let peer_file = "peers.txt"; 26 | 27 | info!(message = "Loading peers from.", file = %peer_file); 28 | 29 | let peers = kv::load_peers(peer_file).await?; 30 | 31 | let id = match opts { 32 | Opts::bootstrap { id, .. } => id, 33 | Opts::join { id, .. } => id, 34 | }; 35 | 36 | let bind = peers.get(&id).expect("Provided id not in peers list."); 37 | 38 | let db_file = uuid::Uuid::new_v4(); 39 | let mut db_path = std::env::temp_dir().join(db_file.to_string()); 40 | db_path.set_extension("sled"); 41 | 42 | let db = sled::Db::open(db_path)?; 43 | 44 | let mut server = Server::new(*bind, peers.clone(), db.clone()); 45 | 46 | let raft_inbound_events = server.start()?; 47 | 48 | let node_span = info_span!("node"); 49 | let enter = node_span.enter(); 50 | 51 | let mut node = match opts { 52 | Opts::bootstrap { .. } => { 53 | let storage = MemStorage::new_with_conf_state(ConfState::from((vec![id], vec![]))); 54 | Node::bootstrap(id, peers, raft_inbound_events, storage, db)? 55 | } 56 | Opts::join { target, .. } => { 57 | // TODO: find better method to let bootstrapper win its campagin 58 | tokio::timer::delay_for(std::time::Duration::from_secs(3)).await; 59 | let storage = MemStorage::new(); 60 | Node::join(id, target, peers, raft_inbound_events, storage, db) 61 | .instrument(node_span.clone()) 62 | .await? 63 | } 64 | }; 65 | 66 | kv::spawn( 67 | async move { 68 | info!("Starting raft module."); 69 | if let Err(error) = node.run().await { 70 | error!(message = "Node error.", %error); 71 | } 72 | }, 73 | node_span.clone(), 74 | ); 75 | 76 | drop(enter); 77 | 78 | signal::ctrl_c()?.next().await; 79 | 80 | Ok(()) 81 | } 82 | -------------------------------------------------------------------------------- /src/bin/kv_client.rs: -------------------------------------------------------------------------------- 1 | use futures_util::StreamExt; 2 | use kv::{pb, Error}; 3 | use std::net::SocketAddr; 4 | use structopt::StructOpt; 5 | use tokio::net::signal; 6 | use tonic::Request; 7 | use tracing::{debug, error}; 8 | 9 | #[derive(Clone, Debug, StructOpt)] 10 | #[structopt(name = "kv client", about = "A consitient kv store client")] 11 | struct Opts { 12 | #[structopt(subcommand)] 13 | command: Command, 14 | 15 | #[structopt(short, long)] 16 | endpoint: Option, 17 | 18 | #[structopt(short, long)] 19 | peer_id: Option, 20 | 21 | #[structopt(long)] 22 | peer_config: Option, 23 | } 24 | 25 | #[derive(Debug, Clone, StructOpt)] 26 | #[allow(non_camel_case_types)] 27 | enum Command { 28 | put { key: String, value: String }, 29 | get { key: String }, 30 | } 31 | 32 | #[tokio::main] 33 | async fn main() -> Result<(), Error> { 34 | let opts = Opts::from_args(); 35 | 36 | tokio::spawn(async move { 37 | signal::ctrl_c().unwrap().next().await; 38 | std::process::exit(0); 39 | }); 40 | 41 | kv::init_tracing("kv_server")?; 42 | 43 | let dst = if let Some(endpoint) = &opts.endpoint { 44 | endpoint.to_string() 45 | } else { 46 | let peer_file = opts 47 | .peer_config 48 | .clone() 49 | .unwrap_or_else(|| "peers.txt".to_string()); 50 | 51 | debug!(message = "Loading peers from.", file = %peer_file); 52 | 53 | let peers = kv::load_peers(peer_file).await?; 54 | 55 | if let Some(peer_id) = &opts.peer_id { 56 | peers.get(peer_id).expect("Invalid peer id").to_string() 57 | } else { 58 | peers[&1].to_string() 59 | } 60 | }; 61 | 62 | let mut client = pb::client::KvClient::connect(format!("http://{}", dst))?; 63 | 64 | match opts.command { 65 | Command::put { key, value } => { 66 | let put = pb::PutRequest { 67 | key: key.into_bytes(), 68 | value: value.into_bytes(), 69 | }; 70 | 71 | debug!(message = "Sending put request."); 72 | match client.put(Request::new(put)).await { 73 | Ok(response) => { 74 | debug!(message = "Put rpc completed.", ?response); 75 | println!("Ok"); 76 | } 77 | Err(error) => error!(message = "Put rpc failed.", %error), 78 | } 79 | } 80 | 81 | Command::get { key } => { 82 | let get = pb::RangeRequest { 83 | key: key.into_bytes(), 84 | range_end: Vec::new(), 85 | }; 86 | 87 | debug!(message = "Sending get request."); 88 | match client.range(Request::new(get)).await { 89 | Ok(response) => { 90 | for kv in &response.get_ref().kvs { 91 | let s = String::from_utf8_lossy(&kv.value[..]); 92 | println!("{}", s); 93 | } 94 | 95 | debug!(message = "Get rpc completed.", ?response); 96 | } 97 | Err(error) => error!(message = "Get rpc failed.", %error), 98 | } 99 | } 100 | } 101 | 102 | Ok(()) 103 | } 104 | -------------------------------------------------------------------------------- /proto/kv/eraftpb.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package kv; 3 | 4 | enum EntryType { 5 | EntryNormal = 0; 6 | EntryConfChange = 1; 7 | } 8 | 9 | // The entry is a type of change that needs to be applied. It contains two data fields. 10 | // While the fields are built into the model; their usage is determined by the entry_type. 11 | // 12 | // For normal entries, the data field should contain the data change that should be applied. 13 | // The context field can be used for any contextual data that might be relevant to the 14 | // application of the data. 15 | // 16 | // For configuration changes, the data will contain the ConfChange message and the 17 | // context will provide anything needed to assist the configuration change. The context 18 | // if for the user to set and use in this case. 19 | message Entry { 20 | EntryType entry_type = 1; 21 | uint64 term = 2; 22 | uint64 index = 3; 23 | bytes data = 4; 24 | bytes context = 6; 25 | 26 | // Deprecated! It is kept for backward compatibility. 27 | // TODO: remove it in the next major release. 28 | bool sync_log = 5; 29 | } 30 | 31 | message SnapshotMetadata { 32 | ConfState conf_state = 1; 33 | ConfState pending_membership_change = 4; 34 | uint64 pending_membership_change_index = 5; 35 | uint64 index = 2; 36 | uint64 term = 3; 37 | } 38 | 39 | message Snapshot { 40 | bytes data = 1; 41 | SnapshotMetadata metadata = 2; 42 | } 43 | 44 | enum MessageType { 45 | MsgHup = 0; 46 | MsgBeat = 1; 47 | MsgPropose = 2; 48 | MsgAppend = 3; 49 | MsgAppendResponse = 4; 50 | MsgRequestVote = 5; 51 | MsgRequestVoteResponse = 6; 52 | MsgSnapshot = 7; 53 | MsgHeartbeat = 8; 54 | MsgHeartbeatResponse = 9; 55 | MsgUnreachable = 10; 56 | MsgSnapStatus = 11; 57 | MsgCheckQuorum = 12; 58 | MsgTransferLeader = 13; 59 | MsgTimeoutNow = 14; 60 | MsgReadIndex = 15; 61 | MsgReadIndexResp = 16; 62 | MsgRequestPreVote = 17; 63 | MsgRequestPreVoteResponse = 18; 64 | } 65 | 66 | message Message { 67 | MessageType msg_type = 1; 68 | uint64 to = 2; 69 | uint64 from = 3; 70 | uint64 term = 4; 71 | uint64 log_term = 5; 72 | uint64 index = 6; 73 | repeated Entry entries = 7; 74 | uint64 commit = 8; 75 | Snapshot snapshot = 9; 76 | uint64 request_snapshot = 13; 77 | bool reject = 10; 78 | uint64 reject_hint = 11; 79 | bytes context = 12; 80 | } 81 | 82 | message HardState { 83 | uint64 term = 1; 84 | uint64 vote = 2; 85 | uint64 commit = 3; 86 | } 87 | 88 | message ConfState { 89 | repeated uint64 nodes = 1; 90 | repeated uint64 learners = 2; 91 | } 92 | 93 | enum ConfChangeType { 94 | AddNode = 0; 95 | RemoveNode = 1; 96 | AddLearnerNode = 2; 97 | BeginMembershipChange = 3; 98 | FinalizeMembershipChange = 4; 99 | } 100 | 101 | message ConfChange { 102 | uint64 id = 1; 103 | ConfChangeType change_type = 2; 104 | // Used in `AddNode`, `RemoveNode`, and `AddLearnerNode`. 105 | uint64 node_id = 3; 106 | bytes context = 4; 107 | // Used in `BeginMembershipChange` and `FinalizeMembershipChange`. 108 | ConfState configuration = 5; 109 | // Used in `BeginMembershipChange` and `FinalizeMembershipChange`. 110 | // Because `RawNode::apply_conf_change` takes a `ConfChange` instead of an `Entry` we must 111 | // include this index so it can be known. 112 | uint64 start_index = 6; 113 | } 114 | -------------------------------------------------------------------------------- /src/pb.rs: -------------------------------------------------------------------------------- 1 | use raft::eraftpb; 2 | 3 | tonic::include_proto!("kv"); 4 | 5 | pub mod wal { 6 | tonic::include_proto!("wal"); 7 | } 8 | 9 | // Types suck 10 | 11 | impl From for Message { 12 | fn from(m: eraftpb::Message) -> Message { 13 | Message { 14 | msg_type: m.msg_type, 15 | to: m.to, 16 | from: m.from, 17 | term: m.term, 18 | log_term: m.log_term, 19 | index: m.index, 20 | entries: m.entries.into_iter().map(Entry::from).collect(), 21 | commit: m.commit, 22 | snapshot: m.snapshot.map(Snapshot::from), 23 | request_snapshot: m.request_snapshot, 24 | reject: m.reject, 25 | context: m.context, 26 | reject_hint: m.reject_hint, 27 | } 28 | } 29 | } 30 | 31 | impl From for Entry { 32 | fn from(e: eraftpb::Entry) -> Entry { 33 | Entry { 34 | entry_type: e.entry_type, 35 | term: e.term, 36 | index: e.index, 37 | data: e.data, 38 | context: e.context, 39 | sync_log: e.sync_log, 40 | } 41 | } 42 | } 43 | 44 | impl From for Snapshot { 45 | fn from(s: eraftpb::Snapshot) -> Snapshot { 46 | Snapshot { 47 | data: s.data, 48 | metadata: s.metadata.map(SnapshotMetadata::from), 49 | } 50 | } 51 | } 52 | 53 | impl From for SnapshotMetadata { 54 | fn from(sm: eraftpb::SnapshotMetadata) -> SnapshotMetadata { 55 | SnapshotMetadata { 56 | conf_state: sm.conf_state.map(ConfState::from), 57 | pending_membership_change: sm.pending_membership_change.map(ConfState::from), 58 | pending_membership_change_index: sm.pending_membership_change_index, 59 | index: sm.index, 60 | term: sm.term, 61 | } 62 | } 63 | } 64 | 65 | impl From for ConfState { 66 | fn from(c: eraftpb::ConfState) -> ConfState { 67 | ConfState { 68 | nodes: c.nodes, 69 | learners: c.learners, 70 | } 71 | } 72 | } 73 | 74 | // And now we do the reverse... 75 | 76 | impl From for eraftpb::Message { 77 | fn from(m: Message) -> eraftpb::Message { 78 | eraftpb::Message { 79 | msg_type: m.msg_type, 80 | to: m.to, 81 | from: m.from, 82 | term: m.term, 83 | log_term: m.log_term, 84 | index: m.index, 85 | entries: m.entries.into_iter().map(eraftpb::Entry::from).collect(), 86 | commit: m.commit, 87 | snapshot: m.snapshot.map(eraftpb::Snapshot::from), 88 | request_snapshot: m.request_snapshot, 89 | reject: m.reject, 90 | context: m.context, 91 | reject_hint: m.reject_hint, 92 | } 93 | } 94 | } 95 | 96 | impl From for eraftpb::Entry { 97 | fn from(e: Entry) -> eraftpb::Entry { 98 | eraftpb::Entry { 99 | entry_type: e.entry_type, 100 | term: e.term, 101 | index: e.index, 102 | data: e.data, 103 | context: e.context, 104 | sync_log: e.sync_log, 105 | } 106 | } 107 | } 108 | 109 | impl From for eraftpb::Snapshot { 110 | fn from(s: Snapshot) -> eraftpb::Snapshot { 111 | eraftpb::Snapshot { 112 | data: s.data, 113 | metadata: s.metadata.map(eraftpb::SnapshotMetadata::from), 114 | } 115 | } 116 | } 117 | 118 | impl From for eraftpb::SnapshotMetadata { 119 | fn from(sm: SnapshotMetadata) -> eraftpb::SnapshotMetadata { 120 | eraftpb::SnapshotMetadata { 121 | conf_state: sm.conf_state.map(eraftpb::ConfState::from), 122 | pending_membership_change: sm.pending_membership_change.map(eraftpb::ConfState::from), 123 | pending_membership_change_index: sm.pending_membership_change_index, 124 | index: sm.index, 125 | term: sm.term, 126 | } 127 | } 128 | } 129 | 130 | impl From for eraftpb::ConfState { 131 | fn from(c: ConfState) -> eraftpb::ConfState { 132 | eraftpb::ConfState { 133 | nodes: c.nodes, 134 | learners: c.learners, 135 | } 136 | } 137 | } 138 | -------------------------------------------------------------------------------- /src/storage.rs: -------------------------------------------------------------------------------- 1 | use bytes::BytesMut; 2 | use prost::Message; 3 | use raft::{ 4 | eraftpb::{ConfState, Entry, Snapshot}, 5 | RaftState, 6 | }; 7 | use sled::Db; 8 | use std::path::Path; 9 | 10 | pub use raft::storage::MemStorage; 11 | 12 | const CONF_STATE_KEY: &'static str = "conf_state"; 13 | const HIGH_INDEX: [u8; 8] = [255, 255, 255, 255, 255, 255, 255, 255]; 14 | const LOW_INDEX: [u8; 10] = [255, 255, 255, 255, 255, 255, 255, 255, 255, 0]; 15 | 16 | pub trait Storage: raft::storage::Storage { 17 | fn append(&self, entries: &[Entry]) -> Result<(), crate::Error>; 18 | fn apply_snapshot(&self, snapshot: Snapshot) -> Result<(), crate::Error>; 19 | fn set_conf_state(&self, cs: ConfState) -> Result<(), crate::Error>; 20 | fn set_hard_state(&self, commit: u64, term: u64) -> Result<(), crate::Error>; 21 | } 22 | 23 | impl Storage for MemStorage { 24 | fn append(&self, entries: &[Entry]) -> Result<(), crate::Error> { 25 | self.wl().append(entries)?; 26 | Ok(()) 27 | } 28 | 29 | fn apply_snapshot(&self, snapshot: Snapshot) -> Result<(), crate::Error> { 30 | self.wl().apply_snapshot(snapshot)?; 31 | Ok(()) 32 | } 33 | 34 | fn set_conf_state(&self, cs: ConfState) -> Result<(), crate::Error> { 35 | self.wl().set_conf_state(cs, None); 36 | Ok(()) 37 | } 38 | 39 | fn set_hard_state(&self, commit: u64, term: u64) -> Result<(), crate::Error> { 40 | let mut me = self.wl(); 41 | me.mut_hard_state().commit = commit; 42 | me.mut_hard_state().term = term; 43 | Ok(()) 44 | } 45 | } 46 | 47 | #[derive(Debug)] 48 | pub struct SledStorage { 49 | db: Db, 50 | } 51 | 52 | impl SledStorage { 53 | pub fn new(path: impl AsRef) -> Result { 54 | let db = Db::open(path)?; 55 | 56 | Ok(Self { db }) 57 | } 58 | 59 | pub async fn append(&self, entries: &[Entry]) -> Result<(), crate::Error> { 60 | for entry in entries { 61 | let key = format!("key-{}", entry.index); 62 | let mut buf = BytesMut::with_capacity(entry.encoded_len()); 63 | entry.encode(&mut buf)?; 64 | self.db.insert(&key[..], &buf[..])?; 65 | } 66 | 67 | self.db.flush_async().await?; 68 | 69 | Ok(()) 70 | } 71 | 72 | pub fn set_conf_state(&self, cs: ConfState) -> Result<(), crate::Error> { 73 | let mut buf = BytesMut::with_capacity(cs.encoded_len()); 74 | cs.encode(&mut buf)?; 75 | self.db.insert(CONF_STATE_KEY.as_bytes(), &buf[..])?; 76 | Ok(()) 77 | } 78 | } 79 | 80 | impl raft::storage::Storage for SledStorage { 81 | fn initial_state(&self) -> raft::Result { 82 | Ok(RaftState::default()) 83 | } 84 | 85 | fn entries( 86 | &self, 87 | low: u64, 88 | high: u64, 89 | max_size: impl Into>, 90 | ) -> raft::Result> { 91 | let low = make_log_key(low); 92 | let high = make_log_key(high); 93 | 94 | let entries = if let Some(max_size) = max_size.into() { 95 | self.db 96 | .range(low..high) 97 | .take(max_size as usize) 98 | .collect::, _>>() 99 | .unwrap() 100 | } else { 101 | self.db 102 | .range(low..high) 103 | .collect::, _>>() 104 | .unwrap() 105 | }; 106 | 107 | let entries = entries 108 | .into_iter() 109 | .map(|(_idx, buf)| Entry::decode(&buf[..])) 110 | .collect::, _>>() 111 | .unwrap(); 112 | 113 | Ok(entries) 114 | } 115 | 116 | fn term(&self, idx: u64) -> raft::Result { 117 | let key = make_log_key(idx); 118 | 119 | if let Some(value) = self.db.get(&key[..]).unwrap() { 120 | let entry = Entry::decode(&value[..]).unwrap(); 121 | Ok(entry.term) 122 | } else { 123 | Ok(1) 124 | } 125 | } 126 | 127 | fn first_index(&self) -> raft::Result { 128 | if let Some(value) = self.db.range(LOW_INDEX..).next() { 129 | let value = value.unwrap(); 130 | let entry = Entry::decode(&value.1[..]).unwrap(); 131 | Ok(entry.index) 132 | } else { 133 | Ok(1) 134 | } 135 | } 136 | 137 | fn last_index(&self) -> raft::Result { 138 | if let Some(value) = self.db.range(..HIGH_INDEX).next_back() { 139 | let value = value.unwrap(); 140 | let entry = Entry::decode(&value.1[..]).unwrap(); 141 | Ok(entry.index) 142 | } else { 143 | Ok(1) 144 | } 145 | } 146 | 147 | fn snapshot(&self, _request_index: u64) -> raft::Result { 148 | unimplemented!() 149 | } 150 | } 151 | 152 | fn make_log_key(idx: u64) -> [u8; 9] { 153 | use bytes::BufMut; 154 | use std::io::Cursor; 155 | let mut key = [0; 9]; 156 | 157 | { 158 | let mut key = Cursor::new(&mut key[..]); 159 | key.put_u8(1); 160 | key.put_u64_le(idx); 161 | } 162 | 163 | key 164 | } 165 | -------------------------------------------------------------------------------- /src/network.rs: -------------------------------------------------------------------------------- 1 | use crate::{ 2 | node, 3 | pb::{self, client, server}, 4 | }; 5 | use sled::Db; 6 | use std::{collections::HashMap, net::SocketAddr, sync::Arc}; 7 | use tokio::sync::{mpsc, oneshot}; 8 | use tonic::{transport, Code, Request, Response, Status, Streaming}; 9 | use tracing::{debug, error, info, info_span, warn}; 10 | 11 | type RaftInboundSender = mpsc::Sender; 12 | pub type Peers = Arc>; 13 | pub type RaftInboundEvents = mpsc::Receiver; 14 | pub type KvClient = client::KvClient; 15 | 16 | #[derive(Debug)] 17 | pub struct Server { 18 | bind: SocketAddr, 19 | peers: Peers, 20 | db: Db, 21 | // TODO: add shutdown handle here. 22 | } 23 | 24 | impl Server { 25 | pub fn new(bind: SocketAddr, peers: Peers, db: Db) -> Self { 26 | Self { bind, peers, db } 27 | } 28 | 29 | pub fn start(&mut self) -> Result { 30 | let server_span = info_span!("server", bind = %self.bind); 31 | let _enter = server_span.enter(); 32 | 33 | info!(message = "Starting server.", listening = %self.bind); 34 | 35 | let (raft_inbound_tx, raft_inbound_rx) = mpsc::channel(1024); 36 | 37 | let server = KvServer { 38 | inbound_tx: raft_inbound_tx, 39 | peers: self.peers.clone(), 40 | db: self.db.clone(), 41 | }; 42 | 43 | let bind = self.bind; 44 | 45 | crate::spawn( 46 | async move { 47 | if let Err(error) = transport::Server::builder() 48 | .serve(bind, server::KvServer::new(server)) 49 | .await 50 | { 51 | error!(message = "Transport server error.", %error); 52 | } 53 | }, 54 | server_span.clone(), 55 | ); 56 | 57 | Ok(raft_inbound_rx) 58 | } 59 | } 60 | 61 | #[derive(Clone, Debug)] 62 | pub struct RaftPeerRouter { 63 | sender: mpsc::Sender<(u64, pb::Message)>, 64 | } 65 | 66 | impl RaftPeerRouter { 67 | pub fn new(peers: Peers) -> Self { 68 | let (tx, rx) = mpsc::channel(1024); 69 | 70 | let mut bg = RaftPeerRouterTask::new(peers, rx); 71 | 72 | crate::spawn( 73 | async move { 74 | if let Err(error) = bg.run().await { 75 | error!(message = "Raft router background task error.", %error); 76 | } 77 | }, 78 | info_span!("background"), 79 | ); 80 | 81 | Self { sender: tx } 82 | } 83 | 84 | pub async fn send(&mut self, to: u64, msg: pb::Message) -> Result<(), crate::Error> { 85 | self.sender.send((to, msg)).await?; 86 | Ok(()) 87 | } 88 | } 89 | 90 | #[derive(Debug)] 91 | pub struct RaftPeerRouterTask { 92 | peers: Peers, 93 | raft_events: mpsc::Receiver<(u64, pb::Message)>, 94 | connections: HashMap>, 95 | } 96 | 97 | impl RaftPeerRouterTask { 98 | pub fn new(peers: Peers, raft_events: mpsc::Receiver<(u64, pb::Message)>) -> Self { 99 | Self { 100 | peers, 101 | raft_events, 102 | connections: HashMap::new(), 103 | } 104 | } 105 | 106 | pub async fn run(&mut self) -> Result<(), crate::Error> { 107 | while let Some((id, msg)) = self.raft_events.recv().await { 108 | if !self.connections.contains_key(&id) { 109 | if let Some(addr) = self.peers.get(&id) { 110 | let (tx, rx) = mpsc::channel(1024); 111 | 112 | self.connections.insert(id, tx); 113 | 114 | let addr = *addr; 115 | crate::spawn( 116 | async move { 117 | if let Err(error) = connect_to_peer(addr, rx).await { 118 | error!(message = "Error connecting to peer.", %error); 119 | } 120 | }, 121 | info_span!("connection", %id, %addr), 122 | ); 123 | } else { 124 | warn!(message = "Attempting to connection to peer that does not exist.", %id); 125 | } 126 | } 127 | 128 | if let Some(conn) = self.connections.get_mut(&id) { 129 | if let Err(_) = conn.send(msg).await { 130 | self.connections.remove(&id); 131 | } 132 | } 133 | } 134 | 135 | Ok(()) 136 | } 137 | } 138 | 139 | async fn connect_to_peer( 140 | peer: SocketAddr, 141 | mut rx: mpsc::Receiver, 142 | ) -> Result { 143 | let dst = format!("http://{}", peer); 144 | let mut client = KvClient::connect(dst)?; 145 | 146 | client.handshake(Request::new(())).await?; 147 | 148 | let stream = async_stream::try_stream! { 149 | while let Some(msg) = rx.recv().await { 150 | yield msg; 151 | } 152 | }; 153 | 154 | client.raft(Request::new(stream)).await?; 155 | 156 | Ok(client) 157 | } 158 | 159 | #[derive(Debug)] 160 | struct KvServer { 161 | inbound_tx: RaftInboundSender, 162 | peers: Peers, 163 | db: Db, 164 | } 165 | 166 | #[tonic::async_trait] 167 | impl pb::server::Kv for KvServer { 168 | async fn join( 169 | &self, 170 | req: Request, 171 | ) -> Result, Status> { 172 | let id = req.into_inner().id; 173 | 174 | info!(message = "Join request.", from = %id); 175 | 176 | if self.peers.contains_key(&id) { 177 | let (tx, rx) = oneshot::channel(); 178 | let proposal = node::Control::Propose(node::Proposal::AddNode { id }, tx); 179 | 180 | debug!(message = "Submitting proposal for join.", %id); 181 | self.inbound_tx 182 | .clone() 183 | .send(proposal) 184 | .await 185 | .map_err(|_| Status::new(Code::Internal, "Inbound sender channel dropped."))?; 186 | 187 | rx.await 188 | .map_err(|_| Status::new(Code::Internal, "Proposal sender dropped."))?; 189 | 190 | Ok(Response::new(pb::JoinResponse { joined: true })) 191 | } else { 192 | Ok(Response::new(pb::JoinResponse { joined: false })) 193 | } 194 | } 195 | 196 | async fn put(&self, req: Request) -> Result, Status> { 197 | let put = req.into_inner(); 198 | 199 | let req = pb::InternalRaftMessage { 200 | put: Some(put), 201 | ..Default::default() 202 | }; 203 | 204 | self.propose(req).await?; 205 | 206 | Ok(Response::new(pb::PutResponse {})) 207 | } 208 | 209 | async fn range( 210 | &self, 211 | req: Request, 212 | ) -> Result, Status> { 213 | let range = req.into_inner(); 214 | 215 | let kvs = if range.range_end.is_empty() { 216 | self.db 217 | .get(&range.key[..]) 218 | .map_err(|_| Status::new(Code::Internal, "db error"))? 219 | .into_iter() 220 | .map(|value| pb::KeyValue { 221 | key: range.key.clone(), 222 | value: Vec::from(&value[..]), 223 | }) 224 | .collect::>() 225 | } else { 226 | unimplemented!() 227 | }; 228 | 229 | Ok(Response::new(pb::RangeResponse { kvs })) 230 | } 231 | 232 | async fn raft(&self, mut req: Request>) -> Result, Status> { 233 | while let Some(msg) = req.get_mut().message().await? { 234 | let msg = node::Control::Raft(msg.into()); 235 | debug!(message = "Inbound message", ?msg); 236 | self.inbound_tx 237 | .clone() 238 | .send(msg) 239 | .await 240 | .map_err(|_| Status::new(Code::Internal, "Internal raft process shutdown"))?; 241 | } 242 | 243 | Ok(Response::new(())) 244 | } 245 | 246 | async fn handshake(&self, _: Request<()>) -> Result, Status> { 247 | Ok(Response::new(())) 248 | } 249 | } 250 | 251 | impl KvServer { 252 | async fn propose(&self, req: pb::InternalRaftMessage) -> Result<(), Status> { 253 | let (tx, rx) = oneshot::channel(); 254 | 255 | let proposal = node::Control::Propose(node::Proposal::InternalMessage(req), tx); 256 | 257 | debug!(message = "Submitting proposal for internal message."); 258 | 259 | self.inbound_tx 260 | .clone() 261 | .send(proposal) 262 | .await 263 | .map_err(|_| Status::new(Code::Internal, "Inbound sender channel dropped."))?; 264 | 265 | rx.await 266 | .map_err(|_| Status::new(Code::Internal, "Proposal sender dropped."))?; 267 | 268 | Ok(()) 269 | } 270 | } 271 | -------------------------------------------------------------------------------- /src/node.rs: -------------------------------------------------------------------------------- 1 | use crate::{network, pb, storage::Storage}; 2 | use bytes::BytesMut; 3 | use prost::Message; 4 | use raft::{ 5 | eraftpb::{ConfChange, ConfChangeType, EntryType, Snapshot}, 6 | RawNode, 7 | }; 8 | use sled::Db; 9 | use slog::{Drain, OwnedKVList, Record}; 10 | use std::{ 11 | collections::HashMap, 12 | time::{Duration, Instant}, 13 | }; 14 | use tokio::{future::FutureExt, sync::oneshot, timer}; 15 | use tracing::{debug, error, info, trace, warn}; 16 | 17 | #[derive(Debug)] 18 | pub enum Control { 19 | Propose(Proposal, oneshot::Sender<()>), 20 | Raft(pb::Message), 21 | } 22 | 23 | #[derive(Debug)] 24 | pub enum Proposal { 25 | AddNode { id: u64 }, 26 | InternalMessage(pb::InternalRaftMessage), 27 | } 28 | 29 | #[derive(Debug, PartialEq, Eq, Hash)] 30 | struct RequestId(u8); 31 | 32 | /// The kv node that contains all the state. 33 | pub struct Node { 34 | #[allow(dead_code)] 35 | id: u64, 36 | raft: RawNode, 37 | raft_inbound_events: network::RaftInboundEvents, 38 | router: network::RaftPeerRouter, 39 | next_request_id: RequestId, 40 | in_flight_proposals: HashMap>, 41 | db: Db, 42 | } 43 | 44 | impl Node { 45 | pub fn bootstrap( 46 | id: u64, 47 | peers: network::Peers, 48 | raft_inbound_events: network::RaftInboundEvents, 49 | storage: S, 50 | db: Db, 51 | ) -> Result { 52 | info!("Bootstraping raft."); 53 | let mut me = Self::new(id, peers, raft_inbound_events, storage, db)?; 54 | for _ in 0..10 { 55 | me.raft.tick(); 56 | } 57 | Ok(me) 58 | } 59 | 60 | pub async fn join( 61 | id: u64, 62 | target: u64, 63 | peers: network::Peers, 64 | mut raft_inbound_events: network::RaftInboundEvents, 65 | storage: S, 66 | db: Db, 67 | ) -> Result { 68 | let target = peers.get(&target).expect("Target id not in peer list!"); 69 | 70 | info!(message = "Attempting to join.", joining_via = %target); 71 | 72 | let dst = format!("http://{}", target); 73 | let mut client = network::KvClient::connect(dst)?; 74 | 75 | loop { 76 | let req = pb::JoinRequest { id }; 77 | match client.join(tonic::Request::new(req)).await { 78 | Ok(_) => { 79 | info!(message = "Connected to peer.", %target); 80 | break; 81 | } 82 | Err(error) => { 83 | warn!(message = "Unable to connect to client; Retrying in 3 sec.", %error); 84 | timer::delay_for(Duration::from_secs(3)).await; 85 | } 86 | } 87 | } 88 | 89 | info!("Waiting for next inbound raft message"); 90 | while let Some(msg) = raft_inbound_events.recv().await { 91 | if let Control::Raft(msg) = msg { 92 | use pb::MessageType::*; 93 | 94 | let msg_type = pb::MessageType::from_i32(msg.msg_type); 95 | 96 | if Some(MsgRequestVote) == msg_type 97 | || Some(MsgRequestPreVote) == msg_type 98 | || Some(MsgHeartbeat) == msg_type && msg.commit == 0 99 | { 100 | info!(message = "Recieved inbound raft message.", incoming_id = %msg.to, commit = %msg.commit); 101 | let mut me = Self::new(id, peers, raft_inbound_events, storage, db)?; 102 | me.raft.step(msg.into())?; 103 | return Ok(me); 104 | } 105 | } 106 | } 107 | 108 | panic!("Unable to join!"); 109 | } 110 | 111 | fn new( 112 | id: u64, 113 | peers: network::Peers, 114 | raft_inbound_events: network::RaftInboundEvents, 115 | storage: S, 116 | db: Db, 117 | ) -> Result { 118 | let config = raft::Config { 119 | id, 120 | election_tick: 10, 121 | heartbeat_tick: 3, 122 | ..Default::default() 123 | }; 124 | config.validate()?; 125 | 126 | // let storage = MemStorage::new_with_conf_state((vec![1], vec![])); 127 | // let temp_dir = std::env::temp_dir(); 128 | // let storage = Storage::new(format!("{:?}/{}.log", temp_dir, id))?; 129 | let logger = slog::Logger::root(slog::Fuse(SlogTracer::default()), slog::o!()); 130 | 131 | let raft = RawNode::new(&config, storage, &logger)?; 132 | 133 | let router = network::RaftPeerRouter::new(peers); 134 | 135 | Ok(Self { 136 | id, 137 | raft, 138 | raft_inbound_events, 139 | router, 140 | next_request_id: RequestId(0), 141 | in_flight_proposals: HashMap::new(), 142 | db, 143 | }) 144 | } 145 | 146 | pub async fn run(&mut self) -> Result<(), crate::Error> { 147 | let timeout = Duration::from_millis(100); 148 | let mut remaining_timeout = timeout; 149 | 150 | let debug_timeout = Duration::from_secs(15); 151 | let mut debug_remaining_timeout = timeout; 152 | 153 | loop { 154 | let now = Instant::now(); 155 | 156 | let msg = match self 157 | .raft_inbound_events 158 | .recv() 159 | .timeout(Duration::from_millis(100)) 160 | .await 161 | { 162 | Ok(Some(msg)) => Some(msg), 163 | Ok(None) => return Ok(()), 164 | Err(_) => None, 165 | }; 166 | 167 | if let Some(msg) = msg { 168 | match msg { 169 | Control::Propose(proposal, tx) => { 170 | debug!(message = "Inbound proposal.", ?proposal); 171 | self.handle_proposal(proposal, tx)?; 172 | } 173 | Control::Raft(m) => { 174 | trace!(message = "Inbound raft message.", message = ?m); 175 | self.raft.step(m.into())?; 176 | } 177 | } 178 | } 179 | 180 | let elapsed = now.elapsed(); 181 | if elapsed >= debug_remaining_timeout { 182 | debug_remaining_timeout = debug_timeout; 183 | 184 | let raft::Raft { 185 | id, 186 | term, 187 | state, 188 | leader_id, 189 | raft_log, 190 | .. 191 | } = &self.raft.raft; 192 | let raft::RaftLog { 193 | committed, applied, .. 194 | } = raft_log; 195 | 196 | let config = self.raft.raft.prs().configuration(); 197 | 198 | let voters = config.voters().len(); 199 | let learners = config.learners().len(); 200 | 201 | info!( 202 | %id, 203 | %term, 204 | ?state, 205 | %voters, 206 | %learners, 207 | %leader_id, 208 | %committed, 209 | %applied, 210 | ); 211 | } else { 212 | debug_remaining_timeout -= elapsed; 213 | } 214 | 215 | if elapsed >= remaining_timeout { 216 | remaining_timeout = timeout; 217 | 218 | // We drive Raft every 100ms. 219 | trace!("ticking raft node."); 220 | self.raft.tick(); 221 | } else { 222 | remaining_timeout -= elapsed; 223 | } 224 | 225 | if self.raft.has_ready() { 226 | trace!("Handling ready state"); 227 | self.handle_ready().await?; 228 | } 229 | } 230 | } 231 | 232 | fn handle_proposal( 233 | &mut self, 234 | proposal: Proposal, 235 | tx: oneshot::Sender<()>, 236 | ) -> Result<(), crate::Error> { 237 | let req_id = self.next_request_id.0.wrapping_add(1); 238 | self.in_flight_proposals.insert(RequestId(req_id), tx); 239 | let req_id_bytes = vec![req_id]; 240 | 241 | match proposal { 242 | Proposal::AddNode { id } => { 243 | let mut conf_change = ConfChange::default(); 244 | conf_change.node_id = id; 245 | conf_change.set_change_type(ConfChangeType::AddNode); 246 | self.raft.propose_conf_change(req_id_bytes, conf_change)?; 247 | } 248 | Proposal::InternalMessage(request) => { 249 | let mut buf = BytesMut::with_capacity(request.encoded_len()); 250 | if let Err(error) = request.encode(&mut buf) { 251 | error!(message = "Unable to propose request.", %error); 252 | return Ok(()); 253 | } 254 | 255 | self.raft.propose(req_id_bytes, buf.into_iter().collect())?; 256 | } 257 | } 258 | 259 | Ok(()) 260 | } 261 | 262 | fn notify_proposal(&mut self, context: Vec) { 263 | if let Some(notifier) = self.in_flight_proposals.remove(&RequestId(context[0])) { 264 | let _ = notifier.send(()); 265 | } 266 | } 267 | 268 | async fn handle_ready(&mut self) -> Result<(), crate::Error> { 269 | let mut ready = self.raft.ready(); 270 | let store = self.raft.mut_store(); 271 | 272 | if let Err(error) = store.append(ready.entries()) { 273 | error!(message = "Append entries failed.", %error); 274 | return Ok(()); 275 | } 276 | 277 | if *ready.snapshot() != Snapshot::default() { 278 | if let Err(error) = store.apply_snapshot(ready.snapshot().clone()) { 279 | error!(message = "Apply snapshot failed.", %error); 280 | return Ok(()); 281 | } 282 | } 283 | 284 | for msg in ready.messages.drain(..) { 285 | //if msg.to != self.id { 286 | self.router.send(msg.to, msg.into()).await?; 287 | //} 288 | } 289 | 290 | if let Some(committed_entries) = ready.committed_entries.take() { 291 | for entry in &committed_entries { 292 | if entry.data.is_empty() { 293 | // From new elected leaders. 294 | continue; 295 | } 296 | 297 | let entry_type = EntryType::from_i32(entry.entry_type); 298 | 299 | if let Some(EntryType::EntryConfChange) = entry_type { 300 | let mut cc = ConfChange::default(); 301 | cc.merge(&entry.data).unwrap(); 302 | 303 | let cs = self.raft.apply_conf_change(&cc)?; 304 | self.raft.mut_store().set_conf_state(cs)?; 305 | self.notify_proposal(entry.context.clone()); 306 | } 307 | 308 | if let Some(EntryType::EntryNormal) = entry_type { 309 | let mut internal_raft_message = pb::InternalRaftMessage::default(); 310 | internal_raft_message.merge(&entry.data).unwrap(); 311 | 312 | if let Err(error) = self.apply(internal_raft_message) { 313 | error!(message = "Unable to apply entry.", %error); 314 | // TODO: return an error to the user 315 | } 316 | 317 | self.notify_proposal(entry.context.clone()); 318 | } 319 | } 320 | 321 | if let Some(entry) = committed_entries.last() { 322 | self.raft 323 | .mut_store() 324 | .set_hard_state(entry.index, entry.term)?; 325 | } 326 | } 327 | 328 | self.raft.advance(ready); 329 | 330 | Ok(()) 331 | } 332 | 333 | fn apply(&mut self, req: pb::InternalRaftMessage) -> Result<(), crate::Error> { 334 | if let Some(put) = req.put { 335 | self.db.insert(put.key, put.value)?; 336 | } 337 | 338 | Ok(()) 339 | } 340 | } 341 | 342 | #[derive(Debug, Default)] 343 | pub(crate) struct SlogTracer; 344 | 345 | impl Drain for SlogTracer { 346 | type Ok = (); 347 | type Err = (); 348 | 349 | fn log(&self, record: &Record, _values: &OwnedKVList) -> Result { 350 | let span = tracing::info_span!("raft"); 351 | let _enter = span.enter(); 352 | 353 | match record.level() { 354 | slog::Level::Error | slog::Level::Critical => tracing::error!("{}", record.msg()), 355 | slog::Level::Warning => tracing::warn!("{}", record.msg()), 356 | slog::Level::Info => tracing::info!("{}", record.msg()), 357 | slog::Level::Debug => tracing::debug!("{}", record.msg()), 358 | slog::Level::Trace => tracing::debug!("{}", record.msg()), 359 | } 360 | 361 | Ok(()) 362 | } 363 | } 364 | -------------------------------------------------------------------------------- /Cargo.lock: -------------------------------------------------------------------------------- 1 | # This file is automatically @generated by Cargo. 2 | # It is not intended for manual editing. 3 | [[package]] 4 | name = "aho-corasick" 5 | version = "0.7.6" 6 | source = "registry+https://github.com/rust-lang/crates.io-index" 7 | dependencies = [ 8 | "memchr 2.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 9 | ] 10 | 11 | [[package]] 12 | name = "ansi_term" 13 | version = "0.11.0" 14 | source = "registry+https://github.com/rust-lang/crates.io-index" 15 | dependencies = [ 16 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 17 | ] 18 | 19 | [[package]] 20 | name = "arc-swap" 21 | version = "0.4.3" 22 | source = "registry+https://github.com/rust-lang/crates.io-index" 23 | 24 | [[package]] 25 | name = "arrayvec" 26 | version = "0.4.11" 27 | source = "registry+https://github.com/rust-lang/crates.io-index" 28 | dependencies = [ 29 | "nodrop 0.1.13 (registry+https://github.com/rust-lang/crates.io-index)", 30 | ] 31 | 32 | [[package]] 33 | name = "async-stream" 34 | version = "0.1.2" 35 | source = "registry+https://github.com/rust-lang/crates.io-index" 36 | dependencies = [ 37 | "async-stream-impl 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 38 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 39 | ] 40 | 41 | [[package]] 42 | name = "async-stream-impl" 43 | version = "0.1.1" 44 | source = "registry+https://github.com/rust-lang/crates.io-index" 45 | dependencies = [ 46 | "proc-macro2 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 47 | "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 48 | "syn 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 49 | ] 50 | 51 | [[package]] 52 | name = "async-trait" 53 | version = "0.1.13" 54 | source = "registry+https://github.com/rust-lang/crates.io-index" 55 | dependencies = [ 56 | "proc-macro2 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 57 | "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 58 | "syn 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 59 | ] 60 | 61 | [[package]] 62 | name = "atty" 63 | version = "0.2.13" 64 | source = "registry+https://github.com/rust-lang/crates.io-index" 65 | dependencies = [ 66 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 67 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 68 | ] 69 | 70 | [[package]] 71 | name = "autocfg" 72 | version = "0.1.6" 73 | source = "registry+https://github.com/rust-lang/crates.io-index" 74 | 75 | [[package]] 76 | name = "backtrace" 77 | version = "0.3.38" 78 | source = "registry+https://github.com/rust-lang/crates.io-index" 79 | dependencies = [ 80 | "backtrace-sys 0.1.31 (registry+https://github.com/rust-lang/crates.io-index)", 81 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 82 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 83 | "rustc-demangle 0.1.16 (registry+https://github.com/rust-lang/crates.io-index)", 84 | ] 85 | 86 | [[package]] 87 | name = "backtrace-sys" 88 | version = "0.1.31" 89 | source = "registry+https://github.com/rust-lang/crates.io-index" 90 | dependencies = [ 91 | "cc 1.0.45 (registry+https://github.com/rust-lang/crates.io-index)", 92 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 93 | ] 94 | 95 | [[package]] 96 | name = "base64" 97 | version = "0.10.1" 98 | source = "registry+https://github.com/rust-lang/crates.io-index" 99 | dependencies = [ 100 | "byteorder 1.3.2 (registry+https://github.com/rust-lang/crates.io-index)", 101 | ] 102 | 103 | [[package]] 104 | name = "bincode" 105 | version = "1.2.0" 106 | source = "registry+https://github.com/rust-lang/crates.io-index" 107 | dependencies = [ 108 | "autocfg 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 109 | "byteorder 1.3.2 (registry+https://github.com/rust-lang/crates.io-index)", 110 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 111 | ] 112 | 113 | [[package]] 114 | name = "bitflags" 115 | version = "1.2.0" 116 | source = "registry+https://github.com/rust-lang/crates.io-index" 117 | 118 | [[package]] 119 | name = "bstr" 120 | version = "0.2.8" 121 | source = "registry+https://github.com/rust-lang/crates.io-index" 122 | dependencies = [ 123 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 124 | "memchr 2.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 125 | "regex-automata 0.1.8 (registry+https://github.com/rust-lang/crates.io-index)", 126 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 127 | ] 128 | 129 | [[package]] 130 | name = "byteorder" 131 | version = "1.3.2" 132 | source = "registry+https://github.com/rust-lang/crates.io-index" 133 | 134 | [[package]] 135 | name = "bytes" 136 | version = "0.4.12" 137 | source = "registry+https://github.com/rust-lang/crates.io-index" 138 | dependencies = [ 139 | "byteorder 1.3.2 (registry+https://github.com/rust-lang/crates.io-index)", 140 | "iovec 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 141 | ] 142 | 143 | [[package]] 144 | name = "c2-chacha" 145 | version = "0.2.2" 146 | source = "registry+https://github.com/rust-lang/crates.io-index" 147 | dependencies = [ 148 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 149 | "ppv-lite86 0.2.5 (registry+https://github.com/rust-lang/crates.io-index)", 150 | ] 151 | 152 | [[package]] 153 | name = "cast" 154 | version = "0.2.2" 155 | source = "registry+https://github.com/rust-lang/crates.io-index" 156 | 157 | [[package]] 158 | name = "cc" 159 | version = "1.0.45" 160 | source = "registry+https://github.com/rust-lang/crates.io-index" 161 | 162 | [[package]] 163 | name = "cfg-if" 164 | version = "0.1.10" 165 | source = "registry+https://github.com/rust-lang/crates.io-index" 166 | 167 | [[package]] 168 | name = "chrono" 169 | version = "0.4.9" 170 | source = "registry+https://github.com/rust-lang/crates.io-index" 171 | dependencies = [ 172 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 173 | "num-integer 0.1.41 (registry+https://github.com/rust-lang/crates.io-index)", 174 | "num-traits 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 175 | "time 0.1.42 (registry+https://github.com/rust-lang/crates.io-index)", 176 | ] 177 | 178 | [[package]] 179 | name = "clap" 180 | version = "2.33.0" 181 | source = "registry+https://github.com/rust-lang/crates.io-index" 182 | dependencies = [ 183 | "ansi_term 0.11.0 (registry+https://github.com/rust-lang/crates.io-index)", 184 | "atty 0.2.13 (registry+https://github.com/rust-lang/crates.io-index)", 185 | "bitflags 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 186 | "strsim 0.8.0 (registry+https://github.com/rust-lang/crates.io-index)", 187 | "textwrap 0.11.0 (registry+https://github.com/rust-lang/crates.io-index)", 188 | "unicode-width 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 189 | "vec_map 0.8.1 (registry+https://github.com/rust-lang/crates.io-index)", 190 | ] 191 | 192 | [[package]] 193 | name = "cloudabi" 194 | version = "0.0.3" 195 | source = "registry+https://github.com/rust-lang/crates.io-index" 196 | dependencies = [ 197 | "bitflags 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 198 | ] 199 | 200 | [[package]] 201 | name = "crc32fast" 202 | version = "1.2.0" 203 | source = "registry+https://github.com/rust-lang/crates.io-index" 204 | dependencies = [ 205 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 206 | ] 207 | 208 | [[package]] 209 | name = "criterion" 210 | version = "0.3.0" 211 | source = "registry+https://github.com/rust-lang/crates.io-index" 212 | dependencies = [ 213 | "atty 0.2.13 (registry+https://github.com/rust-lang/crates.io-index)", 214 | "cast 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)", 215 | "clap 2.33.0 (registry+https://github.com/rust-lang/crates.io-index)", 216 | "criterion-plot 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 217 | "csv 1.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 218 | "itertools 0.8.0 (registry+https://github.com/rust-lang/crates.io-index)", 219 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 220 | "num-traits 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 221 | "rand_core 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)", 222 | "rand_os 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)", 223 | "rand_xoshiro 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 224 | "rayon 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 225 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 226 | "serde_derive 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 227 | "serde_json 1.0.41 (registry+https://github.com/rust-lang/crates.io-index)", 228 | "tinytemplate 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 229 | "walkdir 2.2.9 (registry+https://github.com/rust-lang/crates.io-index)", 230 | ] 231 | 232 | [[package]] 233 | name = "criterion-plot" 234 | version = "0.4.0" 235 | source = "registry+https://github.com/rust-lang/crates.io-index" 236 | dependencies = [ 237 | "cast 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)", 238 | "itertools 0.8.0 (registry+https://github.com/rust-lang/crates.io-index)", 239 | ] 240 | 241 | [[package]] 242 | name = "crossbeam-channel" 243 | version = "0.3.9" 244 | source = "registry+https://github.com/rust-lang/crates.io-index" 245 | dependencies = [ 246 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 247 | ] 248 | 249 | [[package]] 250 | name = "crossbeam-deque" 251 | version = "0.7.1" 252 | source = "registry+https://github.com/rust-lang/crates.io-index" 253 | dependencies = [ 254 | "crossbeam-epoch 0.7.2 (registry+https://github.com/rust-lang/crates.io-index)", 255 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 256 | ] 257 | 258 | [[package]] 259 | name = "crossbeam-epoch" 260 | version = "0.7.2" 261 | source = "registry+https://github.com/rust-lang/crates.io-index" 262 | dependencies = [ 263 | "arrayvec 0.4.11 (registry+https://github.com/rust-lang/crates.io-index)", 264 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 265 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 266 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 267 | "memoffset 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)", 268 | "scopeguard 1.0.0 (registry+https://github.com/rust-lang/crates.io-index)", 269 | ] 270 | 271 | [[package]] 272 | name = "crossbeam-queue" 273 | version = "0.1.2" 274 | source = "registry+https://github.com/rust-lang/crates.io-index" 275 | dependencies = [ 276 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 277 | ] 278 | 279 | [[package]] 280 | name = "crossbeam-utils" 281 | version = "0.6.6" 282 | source = "registry+https://github.com/rust-lang/crates.io-index" 283 | dependencies = [ 284 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 285 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 286 | ] 287 | 288 | [[package]] 289 | name = "csv" 290 | version = "1.1.1" 291 | source = "registry+https://github.com/rust-lang/crates.io-index" 292 | dependencies = [ 293 | "bstr 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 294 | "csv-core 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 295 | "itoa 0.4.4 (registry+https://github.com/rust-lang/crates.io-index)", 296 | "ryu 1.0.1 (registry+https://github.com/rust-lang/crates.io-index)", 297 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 298 | ] 299 | 300 | [[package]] 301 | name = "csv-core" 302 | version = "0.1.6" 303 | source = "registry+https://github.com/rust-lang/crates.io-index" 304 | dependencies = [ 305 | "memchr 2.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 306 | ] 307 | 308 | [[package]] 309 | name = "either" 310 | version = "1.5.3" 311 | source = "registry+https://github.com/rust-lang/crates.io-index" 312 | 313 | [[package]] 314 | name = "failure" 315 | version = "0.1.5" 316 | source = "registry+https://github.com/rust-lang/crates.io-index" 317 | dependencies = [ 318 | "backtrace 0.3.38 (registry+https://github.com/rust-lang/crates.io-index)", 319 | ] 320 | 321 | [[package]] 322 | name = "fixedbitset" 323 | version = "0.1.9" 324 | source = "registry+https://github.com/rust-lang/crates.io-index" 325 | 326 | [[package]] 327 | name = "fnv" 328 | version = "1.0.6" 329 | source = "registry+https://github.com/rust-lang/crates.io-index" 330 | 331 | [[package]] 332 | name = "fs2" 333 | version = "0.4.3" 334 | source = "registry+https://github.com/rust-lang/crates.io-index" 335 | dependencies = [ 336 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 337 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 338 | ] 339 | 340 | [[package]] 341 | name = "fuchsia-cprng" 342 | version = "0.1.1" 343 | source = "registry+https://github.com/rust-lang/crates.io-index" 344 | 345 | [[package]] 346 | name = "fuchsia-zircon" 347 | version = "0.3.3" 348 | source = "registry+https://github.com/rust-lang/crates.io-index" 349 | dependencies = [ 350 | "bitflags 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 351 | "fuchsia-zircon-sys 0.3.3 (registry+https://github.com/rust-lang/crates.io-index)", 352 | ] 353 | 354 | [[package]] 355 | name = "fuchsia-zircon-sys" 356 | version = "0.3.3" 357 | source = "registry+https://github.com/rust-lang/crates.io-index" 358 | 359 | [[package]] 360 | name = "futures-channel-preview" 361 | version = "0.3.0-alpha.19" 362 | source = "registry+https://github.com/rust-lang/crates.io-index" 363 | dependencies = [ 364 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 365 | ] 366 | 367 | [[package]] 368 | name = "futures-core-preview" 369 | version = "0.3.0-alpha.19" 370 | source = "registry+https://github.com/rust-lang/crates.io-index" 371 | 372 | [[package]] 373 | name = "futures-sink-preview" 374 | version = "0.3.0-alpha.19" 375 | source = "registry+https://github.com/rust-lang/crates.io-index" 376 | 377 | [[package]] 378 | name = "futures-util-preview" 379 | version = "0.3.0-alpha.19" 380 | source = "registry+https://github.com/rust-lang/crates.io-index" 381 | dependencies = [ 382 | "futures-channel-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 383 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 384 | "futures-sink-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 385 | "pin-utils 0.1.0-alpha.4 (registry+https://github.com/rust-lang/crates.io-index)", 386 | "slab 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 387 | ] 388 | 389 | [[package]] 390 | name = "fxhash" 391 | version = "0.2.1" 392 | source = "registry+https://github.com/rust-lang/crates.io-index" 393 | dependencies = [ 394 | "byteorder 1.3.2 (registry+https://github.com/rust-lang/crates.io-index)", 395 | ] 396 | 397 | [[package]] 398 | name = "getrandom" 399 | version = "0.1.12" 400 | source = "registry+https://github.com/rust-lang/crates.io-index" 401 | dependencies = [ 402 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 403 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 404 | "wasi 0.7.0 (registry+https://github.com/rust-lang/crates.io-index)", 405 | ] 406 | 407 | [[package]] 408 | name = "getset" 409 | version = "0.0.7" 410 | source = "registry+https://github.com/rust-lang/crates.io-index" 411 | dependencies = [ 412 | "proc-macro2 0.4.30 (registry+https://github.com/rust-lang/crates.io-index)", 413 | "quote 0.6.13 (registry+https://github.com/rust-lang/crates.io-index)", 414 | "syn 0.15.44 (registry+https://github.com/rust-lang/crates.io-index)", 415 | ] 416 | 417 | [[package]] 418 | name = "h2" 419 | version = "0.2.0-alpha.3" 420 | source = "registry+https://github.com/rust-lang/crates.io-index" 421 | dependencies = [ 422 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 423 | "fnv 1.0.6 (registry+https://github.com/rust-lang/crates.io-index)", 424 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 425 | "futures-sink-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 426 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 427 | "http 0.1.18 (registry+https://github.com/rust-lang/crates.io-index)", 428 | "indexmap 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 429 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 430 | "slab 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 431 | "string 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 432 | "tokio-codec 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 433 | "tokio-io 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 434 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 435 | ] 436 | 437 | [[package]] 438 | name = "hashbrown" 439 | version = "0.5.0" 440 | source = "registry+https://github.com/rust-lang/crates.io-index" 441 | 442 | [[package]] 443 | name = "heck" 444 | version = "0.3.1" 445 | source = "registry+https://github.com/rust-lang/crates.io-index" 446 | dependencies = [ 447 | "unicode-segmentation 1.3.0 (registry+https://github.com/rust-lang/crates.io-index)", 448 | ] 449 | 450 | [[package]] 451 | name = "http" 452 | version = "0.1.18" 453 | source = "registry+https://github.com/rust-lang/crates.io-index" 454 | dependencies = [ 455 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 456 | "fnv 1.0.6 (registry+https://github.com/rust-lang/crates.io-index)", 457 | "itoa 0.4.4 (registry+https://github.com/rust-lang/crates.io-index)", 458 | ] 459 | 460 | [[package]] 461 | name = "http-body" 462 | version = "0.2.0-alpha.2" 463 | source = "registry+https://github.com/rust-lang/crates.io-index" 464 | dependencies = [ 465 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 466 | "http 0.1.18 (registry+https://github.com/rust-lang/crates.io-index)", 467 | ] 468 | 469 | [[package]] 470 | name = "httparse" 471 | version = "1.3.4" 472 | source = "registry+https://github.com/rust-lang/crates.io-index" 473 | 474 | [[package]] 475 | name = "hyper" 476 | version = "0.13.0-alpha.3" 477 | source = "registry+https://github.com/rust-lang/crates.io-index" 478 | dependencies = [ 479 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 480 | "futures-channel-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 481 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 482 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 483 | "h2 0.2.0-alpha.3 (registry+https://github.com/rust-lang/crates.io-index)", 484 | "http 0.1.18 (registry+https://github.com/rust-lang/crates.io-index)", 485 | "http-body 0.2.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 486 | "httparse 1.3.4 (registry+https://github.com/rust-lang/crates.io-index)", 487 | "iovec 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 488 | "itoa 0.4.4 (registry+https://github.com/rust-lang/crates.io-index)", 489 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 490 | "net2 0.2.33 (registry+https://github.com/rust-lang/crates.io-index)", 491 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 492 | "time 0.1.42 (registry+https://github.com/rust-lang/crates.io-index)", 493 | "tokio 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 494 | "tokio-executor 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 495 | "tokio-io 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 496 | "tokio-net 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 497 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 498 | "tokio-timer 0.3.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 499 | "tower-make 0.3.0-alpha.2a (registry+https://github.com/rust-lang/crates.io-index)", 500 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 501 | "want 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)", 502 | ] 503 | 504 | [[package]] 505 | name = "indexmap" 506 | version = "1.2.0" 507 | source = "registry+https://github.com/rust-lang/crates.io-index" 508 | 509 | [[package]] 510 | name = "iovec" 511 | version = "0.1.2" 512 | source = "registry+https://github.com/rust-lang/crates.io-index" 513 | dependencies = [ 514 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 515 | "winapi 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 516 | ] 517 | 518 | [[package]] 519 | name = "itertools" 520 | version = "0.8.0" 521 | source = "registry+https://github.com/rust-lang/crates.io-index" 522 | dependencies = [ 523 | "either 1.5.3 (registry+https://github.com/rust-lang/crates.io-index)", 524 | ] 525 | 526 | [[package]] 527 | name = "itoa" 528 | version = "0.4.4" 529 | source = "registry+https://github.com/rust-lang/crates.io-index" 530 | 531 | [[package]] 532 | name = "kernel32-sys" 533 | version = "0.2.2" 534 | source = "registry+https://github.com/rust-lang/crates.io-index" 535 | dependencies = [ 536 | "winapi 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 537 | "winapi-build 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 538 | ] 539 | 540 | [[package]] 541 | name = "kv" 542 | version = "0.1.0" 543 | dependencies = [ 544 | "async-stream 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 545 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 546 | "criterion 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)", 547 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 548 | "prost 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 549 | "prost-build 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 550 | "raft 0.6.0-alpha (git+https://github.com/pingcap/raft-rs)", 551 | "sled 0.28.0 (registry+https://github.com/rust-lang/crates.io-index)", 552 | "slog 2.5.2 (registry+https://github.com/rust-lang/crates.io-index)", 553 | "structopt 0.2.18 (registry+https://github.com/rust-lang/crates.io-index)", 554 | "tokio 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 555 | "tonic 0.1.0-alpha.1 (git+https://github.com/hyperium/tonic)", 556 | "tonic-build 0.1.0-alpha.1 (git+https://github.com/hyperium/tonic)", 557 | "tracing 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", 558 | "tracing-futures 0.0.1-alpha.1 (registry+https://github.com/rust-lang/crates.io-index)", 559 | "tracing-log 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)", 560 | "tracing-subscriber 0.1.4 (registry+https://github.com/rust-lang/crates.io-index)", 561 | "uuid 0.7.4 (registry+https://github.com/rust-lang/crates.io-index)", 562 | ] 563 | 564 | [[package]] 565 | name = "lazy_static" 566 | version = "1.4.0" 567 | source = "registry+https://github.com/rust-lang/crates.io-index" 568 | dependencies = [ 569 | "spin 0.5.2 (registry+https://github.com/rust-lang/crates.io-index)", 570 | ] 571 | 572 | [[package]] 573 | name = "libc" 574 | version = "0.2.62" 575 | source = "registry+https://github.com/rust-lang/crates.io-index" 576 | 577 | [[package]] 578 | name = "lock_api" 579 | version = "0.3.1" 580 | source = "registry+https://github.com/rust-lang/crates.io-index" 581 | dependencies = [ 582 | "scopeguard 1.0.0 (registry+https://github.com/rust-lang/crates.io-index)", 583 | ] 584 | 585 | [[package]] 586 | name = "log" 587 | version = "0.4.8" 588 | source = "registry+https://github.com/rust-lang/crates.io-index" 589 | dependencies = [ 590 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 591 | ] 592 | 593 | [[package]] 594 | name = "matchers" 595 | version = "0.0.1" 596 | source = "registry+https://github.com/rust-lang/crates.io-index" 597 | dependencies = [ 598 | "regex-automata 0.1.8 (registry+https://github.com/rust-lang/crates.io-index)", 599 | ] 600 | 601 | [[package]] 602 | name = "memchr" 603 | version = "2.2.1" 604 | source = "registry+https://github.com/rust-lang/crates.io-index" 605 | dependencies = [ 606 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 607 | ] 608 | 609 | [[package]] 610 | name = "memoffset" 611 | version = "0.5.1" 612 | source = "registry+https://github.com/rust-lang/crates.io-index" 613 | dependencies = [ 614 | "rustc_version 0.2.3 (registry+https://github.com/rust-lang/crates.io-index)", 615 | ] 616 | 617 | [[package]] 618 | name = "mio" 619 | version = "0.6.19" 620 | source = "registry+https://github.com/rust-lang/crates.io-index" 621 | dependencies = [ 622 | "fuchsia-zircon 0.3.3 (registry+https://github.com/rust-lang/crates.io-index)", 623 | "fuchsia-zircon-sys 0.3.3 (registry+https://github.com/rust-lang/crates.io-index)", 624 | "iovec 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 625 | "kernel32-sys 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)", 626 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 627 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 628 | "miow 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 629 | "net2 0.2.33 (registry+https://github.com/rust-lang/crates.io-index)", 630 | "slab 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 631 | "winapi 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 632 | ] 633 | 634 | [[package]] 635 | name = "mio-uds" 636 | version = "0.6.7" 637 | source = "registry+https://github.com/rust-lang/crates.io-index" 638 | dependencies = [ 639 | "iovec 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 640 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 641 | "mio 0.6.19 (registry+https://github.com/rust-lang/crates.io-index)", 642 | ] 643 | 644 | [[package]] 645 | name = "miow" 646 | version = "0.2.1" 647 | source = "registry+https://github.com/rust-lang/crates.io-index" 648 | dependencies = [ 649 | "kernel32-sys 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)", 650 | "net2 0.2.33 (registry+https://github.com/rust-lang/crates.io-index)", 651 | "winapi 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 652 | "ws2_32-sys 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 653 | ] 654 | 655 | [[package]] 656 | name = "multimap" 657 | version = "0.4.0" 658 | source = "registry+https://github.com/rust-lang/crates.io-index" 659 | 660 | [[package]] 661 | name = "net2" 662 | version = "0.2.33" 663 | source = "registry+https://github.com/rust-lang/crates.io-index" 664 | dependencies = [ 665 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 666 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 667 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 668 | ] 669 | 670 | [[package]] 671 | name = "nodrop" 672 | version = "0.1.13" 673 | source = "registry+https://github.com/rust-lang/crates.io-index" 674 | 675 | [[package]] 676 | name = "num-integer" 677 | version = "0.1.41" 678 | source = "registry+https://github.com/rust-lang/crates.io-index" 679 | dependencies = [ 680 | "autocfg 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 681 | "num-traits 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 682 | ] 683 | 684 | [[package]] 685 | name = "num-traits" 686 | version = "0.2.8" 687 | source = "registry+https://github.com/rust-lang/crates.io-index" 688 | dependencies = [ 689 | "autocfg 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 690 | ] 691 | 692 | [[package]] 693 | name = "num_cpus" 694 | version = "1.10.1" 695 | source = "registry+https://github.com/rust-lang/crates.io-index" 696 | dependencies = [ 697 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 698 | ] 699 | 700 | [[package]] 701 | name = "owning_ref" 702 | version = "0.4.0" 703 | source = "registry+https://github.com/rust-lang/crates.io-index" 704 | dependencies = [ 705 | "stable_deref_trait 1.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 706 | ] 707 | 708 | [[package]] 709 | name = "pagecache" 710 | version = "0.19.4" 711 | source = "registry+https://github.com/rust-lang/crates.io-index" 712 | dependencies = [ 713 | "bincode 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 714 | "crc32fast 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 715 | "crossbeam-channel 0.3.9 (registry+https://github.com/rust-lang/crates.io-index)", 716 | "crossbeam-epoch 0.7.2 (registry+https://github.com/rust-lang/crates.io-index)", 717 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 718 | "fs2 0.4.3 (registry+https://github.com/rust-lang/crates.io-index)", 719 | "fxhash 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 720 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 721 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 722 | "parking_lot 0.9.0 (registry+https://github.com/rust-lang/crates.io-index)", 723 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 724 | ] 725 | 726 | [[package]] 727 | name = "parking_lot" 728 | version = "0.9.0" 729 | source = "registry+https://github.com/rust-lang/crates.io-index" 730 | dependencies = [ 731 | "lock_api 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 732 | "parking_lot_core 0.6.2 (registry+https://github.com/rust-lang/crates.io-index)", 733 | "rustc_version 0.2.3 (registry+https://github.com/rust-lang/crates.io-index)", 734 | ] 735 | 736 | [[package]] 737 | name = "parking_lot_core" 738 | version = "0.6.2" 739 | source = "registry+https://github.com/rust-lang/crates.io-index" 740 | dependencies = [ 741 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 742 | "cloudabi 0.0.3 (registry+https://github.com/rust-lang/crates.io-index)", 743 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 744 | "redox_syscall 0.1.56 (registry+https://github.com/rust-lang/crates.io-index)", 745 | "rustc_version 0.2.3 (registry+https://github.com/rust-lang/crates.io-index)", 746 | "smallvec 0.6.10 (registry+https://github.com/rust-lang/crates.io-index)", 747 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 748 | ] 749 | 750 | [[package]] 751 | name = "percent-encoding" 752 | version = "1.0.1" 753 | source = "registry+https://github.com/rust-lang/crates.io-index" 754 | 755 | [[package]] 756 | name = "petgraph" 757 | version = "0.4.13" 758 | source = "registry+https://github.com/rust-lang/crates.io-index" 759 | dependencies = [ 760 | "fixedbitset 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", 761 | ] 762 | 763 | [[package]] 764 | name = "pin-project" 765 | version = "0.4.2" 766 | source = "registry+https://github.com/rust-lang/crates.io-index" 767 | dependencies = [ 768 | "pin-project-internal 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 769 | ] 770 | 771 | [[package]] 772 | name = "pin-project-internal" 773 | version = "0.4.2" 774 | source = "registry+https://github.com/rust-lang/crates.io-index" 775 | dependencies = [ 776 | "proc-macro2 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 777 | "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 778 | "syn 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 779 | ] 780 | 781 | [[package]] 782 | name = "pin-utils" 783 | version = "0.1.0-alpha.4" 784 | source = "registry+https://github.com/rust-lang/crates.io-index" 785 | 786 | [[package]] 787 | name = "ppv-lite86" 788 | version = "0.2.5" 789 | source = "registry+https://github.com/rust-lang/crates.io-index" 790 | 791 | [[package]] 792 | name = "proc-macro2" 793 | version = "0.4.30" 794 | source = "registry+https://github.com/rust-lang/crates.io-index" 795 | dependencies = [ 796 | "unicode-xid 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)", 797 | ] 798 | 799 | [[package]] 800 | name = "proc-macro2" 801 | version = "1.0.5" 802 | source = "registry+https://github.com/rust-lang/crates.io-index" 803 | dependencies = [ 804 | "unicode-xid 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 805 | ] 806 | 807 | [[package]] 808 | name = "prost" 809 | version = "0.5.0" 810 | source = "registry+https://github.com/rust-lang/crates.io-index" 811 | dependencies = [ 812 | "byteorder 1.3.2 (registry+https://github.com/rust-lang/crates.io-index)", 813 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 814 | "prost-derive 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 815 | ] 816 | 817 | [[package]] 818 | name = "prost-build" 819 | version = "0.5.0" 820 | source = "registry+https://github.com/rust-lang/crates.io-index" 821 | dependencies = [ 822 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 823 | "heck 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 824 | "itertools 0.8.0 (registry+https://github.com/rust-lang/crates.io-index)", 825 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 826 | "multimap 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 827 | "petgraph 0.4.13 (registry+https://github.com/rust-lang/crates.io-index)", 828 | "prost 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 829 | "prost-types 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 830 | "tempfile 3.1.0 (registry+https://github.com/rust-lang/crates.io-index)", 831 | "which 2.0.1 (registry+https://github.com/rust-lang/crates.io-index)", 832 | ] 833 | 834 | [[package]] 835 | name = "prost-derive" 836 | version = "0.5.0" 837 | source = "registry+https://github.com/rust-lang/crates.io-index" 838 | dependencies = [ 839 | "failure 0.1.5 (registry+https://github.com/rust-lang/crates.io-index)", 840 | "itertools 0.8.0 (registry+https://github.com/rust-lang/crates.io-index)", 841 | "proc-macro2 0.4.30 (registry+https://github.com/rust-lang/crates.io-index)", 842 | "quote 0.6.13 (registry+https://github.com/rust-lang/crates.io-index)", 843 | "syn 0.15.44 (registry+https://github.com/rust-lang/crates.io-index)", 844 | ] 845 | 846 | [[package]] 847 | name = "prost-types" 848 | version = "0.5.0" 849 | source = "registry+https://github.com/rust-lang/crates.io-index" 850 | dependencies = [ 851 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 852 | "prost 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 853 | ] 854 | 855 | [[package]] 856 | name = "protobuf" 857 | version = "2.8.1" 858 | source = "registry+https://github.com/rust-lang/crates.io-index" 859 | 860 | [[package]] 861 | name = "protobuf-build" 862 | version = "0.10.0" 863 | source = "registry+https://github.com/rust-lang/crates.io-index" 864 | dependencies = [ 865 | "bitflags 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 866 | "prost-build 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 867 | "quote 0.6.13 (registry+https://github.com/rust-lang/crates.io-index)", 868 | "regex 1.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 869 | "syn 0.15.44 (registry+https://github.com/rust-lang/crates.io-index)", 870 | ] 871 | 872 | [[package]] 873 | name = "quick-error" 874 | version = "1.2.2" 875 | source = "registry+https://github.com/rust-lang/crates.io-index" 876 | 877 | [[package]] 878 | name = "quote" 879 | version = "0.6.13" 880 | source = "registry+https://github.com/rust-lang/crates.io-index" 881 | dependencies = [ 882 | "proc-macro2 0.4.30 (registry+https://github.com/rust-lang/crates.io-index)", 883 | ] 884 | 885 | [[package]] 886 | name = "quote" 887 | version = "1.0.2" 888 | source = "registry+https://github.com/rust-lang/crates.io-index" 889 | dependencies = [ 890 | "proc-macro2 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 891 | ] 892 | 893 | [[package]] 894 | name = "raft" 895 | version = "0.6.0-alpha" 896 | source = "git+https://github.com/pingcap/raft-rs#71d379559dd4e1c2e02cefec0b617814ec0cce7e" 897 | dependencies = [ 898 | "getset 0.0.7 (registry+https://github.com/rust-lang/crates.io-index)", 899 | "hashbrown 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 900 | "protobuf 2.8.1 (registry+https://github.com/rust-lang/crates.io-index)", 901 | "quick-error 1.2.2 (registry+https://github.com/rust-lang/crates.io-index)", 902 | "raft-proto 0.6.0-alpha (git+https://github.com/pingcap/raft-rs)", 903 | "rand 0.6.5 (registry+https://github.com/rust-lang/crates.io-index)", 904 | "slog 2.5.2 (registry+https://github.com/rust-lang/crates.io-index)", 905 | ] 906 | 907 | [[package]] 908 | name = "raft-proto" 909 | version = "0.6.0-alpha" 910 | source = "git+https://github.com/pingcap/raft-rs#71d379559dd4e1c2e02cefec0b617814ec0cce7e" 911 | dependencies = [ 912 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 913 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 914 | "prost 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 915 | "prost-derive 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 916 | "protobuf 2.8.1 (registry+https://github.com/rust-lang/crates.io-index)", 917 | "protobuf-build 0.10.0 (registry+https://github.com/rust-lang/crates.io-index)", 918 | ] 919 | 920 | [[package]] 921 | name = "rand" 922 | version = "0.6.5" 923 | source = "registry+https://github.com/rust-lang/crates.io-index" 924 | dependencies = [ 925 | "autocfg 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 926 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 927 | "rand_chacha 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 928 | "rand_core 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 929 | "rand_hc 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)", 930 | "rand_isaac 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 931 | "rand_jitter 0.1.4 (registry+https://github.com/rust-lang/crates.io-index)", 932 | "rand_os 0.1.3 (registry+https://github.com/rust-lang/crates.io-index)", 933 | "rand_pcg 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 934 | "rand_xorshift 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 935 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 936 | ] 937 | 938 | [[package]] 939 | name = "rand" 940 | version = "0.7.2" 941 | source = "registry+https://github.com/rust-lang/crates.io-index" 942 | dependencies = [ 943 | "getrandom 0.1.12 (registry+https://github.com/rust-lang/crates.io-index)", 944 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 945 | "rand_chacha 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 946 | "rand_core 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)", 947 | "rand_hc 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 948 | ] 949 | 950 | [[package]] 951 | name = "rand_chacha" 952 | version = "0.1.1" 953 | source = "registry+https://github.com/rust-lang/crates.io-index" 954 | dependencies = [ 955 | "autocfg 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 956 | "rand_core 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 957 | ] 958 | 959 | [[package]] 960 | name = "rand_chacha" 961 | version = "0.2.1" 962 | source = "registry+https://github.com/rust-lang/crates.io-index" 963 | dependencies = [ 964 | "c2-chacha 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)", 965 | "rand_core 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)", 966 | ] 967 | 968 | [[package]] 969 | name = "rand_core" 970 | version = "0.3.1" 971 | source = "registry+https://github.com/rust-lang/crates.io-index" 972 | dependencies = [ 973 | "rand_core 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 974 | ] 975 | 976 | [[package]] 977 | name = "rand_core" 978 | version = "0.4.2" 979 | source = "registry+https://github.com/rust-lang/crates.io-index" 980 | 981 | [[package]] 982 | name = "rand_core" 983 | version = "0.5.1" 984 | source = "registry+https://github.com/rust-lang/crates.io-index" 985 | dependencies = [ 986 | "getrandom 0.1.12 (registry+https://github.com/rust-lang/crates.io-index)", 987 | ] 988 | 989 | [[package]] 990 | name = "rand_hc" 991 | version = "0.1.0" 992 | source = "registry+https://github.com/rust-lang/crates.io-index" 993 | dependencies = [ 994 | "rand_core 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 995 | ] 996 | 997 | [[package]] 998 | name = "rand_hc" 999 | version = "0.2.0" 1000 | source = "registry+https://github.com/rust-lang/crates.io-index" 1001 | dependencies = [ 1002 | "rand_core 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)", 1003 | ] 1004 | 1005 | [[package]] 1006 | name = "rand_isaac" 1007 | version = "0.1.1" 1008 | source = "registry+https://github.com/rust-lang/crates.io-index" 1009 | dependencies = [ 1010 | "rand_core 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 1011 | ] 1012 | 1013 | [[package]] 1014 | name = "rand_jitter" 1015 | version = "0.1.4" 1016 | source = "registry+https://github.com/rust-lang/crates.io-index" 1017 | dependencies = [ 1018 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 1019 | "rand_core 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1020 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 1021 | ] 1022 | 1023 | [[package]] 1024 | name = "rand_os" 1025 | version = "0.1.3" 1026 | source = "registry+https://github.com/rust-lang/crates.io-index" 1027 | dependencies = [ 1028 | "cloudabi 0.0.3 (registry+https://github.com/rust-lang/crates.io-index)", 1029 | "fuchsia-cprng 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 1030 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 1031 | "rand_core 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1032 | "rdrand 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1033 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 1034 | ] 1035 | 1036 | [[package]] 1037 | name = "rand_os" 1038 | version = "0.2.2" 1039 | source = "registry+https://github.com/rust-lang/crates.io-index" 1040 | dependencies = [ 1041 | "getrandom 0.1.12 (registry+https://github.com/rust-lang/crates.io-index)", 1042 | "rand_core 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)", 1043 | ] 1044 | 1045 | [[package]] 1046 | name = "rand_pcg" 1047 | version = "0.1.2" 1048 | source = "registry+https://github.com/rust-lang/crates.io-index" 1049 | dependencies = [ 1050 | "autocfg 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 1051 | "rand_core 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1052 | ] 1053 | 1054 | [[package]] 1055 | name = "rand_xorshift" 1056 | version = "0.1.1" 1057 | source = "registry+https://github.com/rust-lang/crates.io-index" 1058 | dependencies = [ 1059 | "rand_core 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 1060 | ] 1061 | 1062 | [[package]] 1063 | name = "rand_xoshiro" 1064 | version = "0.3.1" 1065 | source = "registry+https://github.com/rust-lang/crates.io-index" 1066 | dependencies = [ 1067 | "rand_core 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)", 1068 | ] 1069 | 1070 | [[package]] 1071 | name = "rayon" 1072 | version = "1.2.0" 1073 | source = "registry+https://github.com/rust-lang/crates.io-index" 1074 | dependencies = [ 1075 | "crossbeam-deque 0.7.1 (registry+https://github.com/rust-lang/crates.io-index)", 1076 | "either 1.5.3 (registry+https://github.com/rust-lang/crates.io-index)", 1077 | "rayon-core 1.6.0 (registry+https://github.com/rust-lang/crates.io-index)", 1078 | ] 1079 | 1080 | [[package]] 1081 | name = "rayon-core" 1082 | version = "1.6.0" 1083 | source = "registry+https://github.com/rust-lang/crates.io-index" 1084 | dependencies = [ 1085 | "crossbeam-deque 0.7.1 (registry+https://github.com/rust-lang/crates.io-index)", 1086 | "crossbeam-queue 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 1087 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 1088 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1089 | "num_cpus 1.10.1 (registry+https://github.com/rust-lang/crates.io-index)", 1090 | ] 1091 | 1092 | [[package]] 1093 | name = "rdrand" 1094 | version = "0.4.0" 1095 | source = "registry+https://github.com/rust-lang/crates.io-index" 1096 | dependencies = [ 1097 | "rand_core 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 1098 | ] 1099 | 1100 | [[package]] 1101 | name = "redox_syscall" 1102 | version = "0.1.56" 1103 | source = "registry+https://github.com/rust-lang/crates.io-index" 1104 | 1105 | [[package]] 1106 | name = "regex" 1107 | version = "1.3.1" 1108 | source = "registry+https://github.com/rust-lang/crates.io-index" 1109 | dependencies = [ 1110 | "aho-corasick 0.7.6 (registry+https://github.com/rust-lang/crates.io-index)", 1111 | "memchr 2.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 1112 | "regex-syntax 0.6.12 (registry+https://github.com/rust-lang/crates.io-index)", 1113 | "thread_local 0.3.6 (registry+https://github.com/rust-lang/crates.io-index)", 1114 | ] 1115 | 1116 | [[package]] 1117 | name = "regex-automata" 1118 | version = "0.1.8" 1119 | source = "registry+https://github.com/rust-lang/crates.io-index" 1120 | dependencies = [ 1121 | "byteorder 1.3.2 (registry+https://github.com/rust-lang/crates.io-index)", 1122 | "regex-syntax 0.6.12 (registry+https://github.com/rust-lang/crates.io-index)", 1123 | "utf8-ranges 1.0.4 (registry+https://github.com/rust-lang/crates.io-index)", 1124 | ] 1125 | 1126 | [[package]] 1127 | name = "regex-syntax" 1128 | version = "0.6.12" 1129 | source = "registry+https://github.com/rust-lang/crates.io-index" 1130 | 1131 | [[package]] 1132 | name = "remove_dir_all" 1133 | version = "0.5.2" 1134 | source = "registry+https://github.com/rust-lang/crates.io-index" 1135 | dependencies = [ 1136 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 1137 | ] 1138 | 1139 | [[package]] 1140 | name = "rustc-demangle" 1141 | version = "0.1.16" 1142 | source = "registry+https://github.com/rust-lang/crates.io-index" 1143 | 1144 | [[package]] 1145 | name = "rustc_version" 1146 | version = "0.2.3" 1147 | source = "registry+https://github.com/rust-lang/crates.io-index" 1148 | dependencies = [ 1149 | "semver 0.9.0 (registry+https://github.com/rust-lang/crates.io-index)", 1150 | ] 1151 | 1152 | [[package]] 1153 | name = "ryu" 1154 | version = "1.0.1" 1155 | source = "registry+https://github.com/rust-lang/crates.io-index" 1156 | 1157 | [[package]] 1158 | name = "same-file" 1159 | version = "1.0.5" 1160 | source = "registry+https://github.com/rust-lang/crates.io-index" 1161 | dependencies = [ 1162 | "winapi-util 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 1163 | ] 1164 | 1165 | [[package]] 1166 | name = "scopeguard" 1167 | version = "1.0.0" 1168 | source = "registry+https://github.com/rust-lang/crates.io-index" 1169 | 1170 | [[package]] 1171 | name = "semver" 1172 | version = "0.9.0" 1173 | source = "registry+https://github.com/rust-lang/crates.io-index" 1174 | dependencies = [ 1175 | "semver-parser 0.7.0 (registry+https://github.com/rust-lang/crates.io-index)", 1176 | ] 1177 | 1178 | [[package]] 1179 | name = "semver-parser" 1180 | version = "0.7.0" 1181 | source = "registry+https://github.com/rust-lang/crates.io-index" 1182 | 1183 | [[package]] 1184 | name = "serde" 1185 | version = "1.0.101" 1186 | source = "registry+https://github.com/rust-lang/crates.io-index" 1187 | dependencies = [ 1188 | "serde_derive 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 1189 | ] 1190 | 1191 | [[package]] 1192 | name = "serde_bytes" 1193 | version = "0.11.2" 1194 | source = "registry+https://github.com/rust-lang/crates.io-index" 1195 | dependencies = [ 1196 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 1197 | ] 1198 | 1199 | [[package]] 1200 | name = "serde_derive" 1201 | version = "1.0.101" 1202 | source = "registry+https://github.com/rust-lang/crates.io-index" 1203 | dependencies = [ 1204 | "proc-macro2 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 1205 | "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 1206 | "syn 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 1207 | ] 1208 | 1209 | [[package]] 1210 | name = "serde_json" 1211 | version = "1.0.41" 1212 | source = "registry+https://github.com/rust-lang/crates.io-index" 1213 | dependencies = [ 1214 | "itoa 0.4.4 (registry+https://github.com/rust-lang/crates.io-index)", 1215 | "ryu 1.0.1 (registry+https://github.com/rust-lang/crates.io-index)", 1216 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 1217 | ] 1218 | 1219 | [[package]] 1220 | name = "signal-hook-registry" 1221 | version = "1.1.1" 1222 | source = "registry+https://github.com/rust-lang/crates.io-index" 1223 | dependencies = [ 1224 | "arc-swap 0.4.3 (registry+https://github.com/rust-lang/crates.io-index)", 1225 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 1226 | ] 1227 | 1228 | [[package]] 1229 | name = "slab" 1230 | version = "0.4.2" 1231 | source = "registry+https://github.com/rust-lang/crates.io-index" 1232 | 1233 | [[package]] 1234 | name = "sled" 1235 | version = "0.28.0" 1236 | source = "registry+https://github.com/rust-lang/crates.io-index" 1237 | dependencies = [ 1238 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 1239 | "pagecache 0.19.4 (registry+https://github.com/rust-lang/crates.io-index)", 1240 | "parking_lot 0.9.0 (registry+https://github.com/rust-lang/crates.io-index)", 1241 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 1242 | "serde_bytes 0.11.2 (registry+https://github.com/rust-lang/crates.io-index)", 1243 | ] 1244 | 1245 | [[package]] 1246 | name = "slog" 1247 | version = "2.5.2" 1248 | source = "registry+https://github.com/rust-lang/crates.io-index" 1249 | 1250 | [[package]] 1251 | name = "smallvec" 1252 | version = "0.6.10" 1253 | source = "registry+https://github.com/rust-lang/crates.io-index" 1254 | 1255 | [[package]] 1256 | name = "spin" 1257 | version = "0.5.2" 1258 | source = "registry+https://github.com/rust-lang/crates.io-index" 1259 | 1260 | [[package]] 1261 | name = "stable_deref_trait" 1262 | version = "1.1.1" 1263 | source = "registry+https://github.com/rust-lang/crates.io-index" 1264 | 1265 | [[package]] 1266 | name = "string" 1267 | version = "0.2.1" 1268 | source = "registry+https://github.com/rust-lang/crates.io-index" 1269 | dependencies = [ 1270 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 1271 | ] 1272 | 1273 | [[package]] 1274 | name = "strsim" 1275 | version = "0.8.0" 1276 | source = "registry+https://github.com/rust-lang/crates.io-index" 1277 | 1278 | [[package]] 1279 | name = "structopt" 1280 | version = "0.2.18" 1281 | source = "registry+https://github.com/rust-lang/crates.io-index" 1282 | dependencies = [ 1283 | "clap 2.33.0 (registry+https://github.com/rust-lang/crates.io-index)", 1284 | "structopt-derive 0.2.18 (registry+https://github.com/rust-lang/crates.io-index)", 1285 | ] 1286 | 1287 | [[package]] 1288 | name = "structopt-derive" 1289 | version = "0.2.18" 1290 | source = "registry+https://github.com/rust-lang/crates.io-index" 1291 | dependencies = [ 1292 | "heck 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 1293 | "proc-macro2 0.4.30 (registry+https://github.com/rust-lang/crates.io-index)", 1294 | "quote 0.6.13 (registry+https://github.com/rust-lang/crates.io-index)", 1295 | "syn 0.15.44 (registry+https://github.com/rust-lang/crates.io-index)", 1296 | ] 1297 | 1298 | [[package]] 1299 | name = "syn" 1300 | version = "0.15.44" 1301 | source = "registry+https://github.com/rust-lang/crates.io-index" 1302 | dependencies = [ 1303 | "proc-macro2 0.4.30 (registry+https://github.com/rust-lang/crates.io-index)", 1304 | "quote 0.6.13 (registry+https://github.com/rust-lang/crates.io-index)", 1305 | "unicode-xid 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)", 1306 | ] 1307 | 1308 | [[package]] 1309 | name = "syn" 1310 | version = "1.0.5" 1311 | source = "registry+https://github.com/rust-lang/crates.io-index" 1312 | dependencies = [ 1313 | "proc-macro2 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 1314 | "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 1315 | "unicode-xid 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 1316 | ] 1317 | 1318 | [[package]] 1319 | name = "tempfile" 1320 | version = "3.1.0" 1321 | source = "registry+https://github.com/rust-lang/crates.io-index" 1322 | dependencies = [ 1323 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 1324 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 1325 | "rand 0.7.2 (registry+https://github.com/rust-lang/crates.io-index)", 1326 | "redox_syscall 0.1.56 (registry+https://github.com/rust-lang/crates.io-index)", 1327 | "remove_dir_all 0.5.2 (registry+https://github.com/rust-lang/crates.io-index)", 1328 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 1329 | ] 1330 | 1331 | [[package]] 1332 | name = "textwrap" 1333 | version = "0.11.0" 1334 | source = "registry+https://github.com/rust-lang/crates.io-index" 1335 | dependencies = [ 1336 | "unicode-width 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 1337 | ] 1338 | 1339 | [[package]] 1340 | name = "thread_local" 1341 | version = "0.3.6" 1342 | source = "registry+https://github.com/rust-lang/crates.io-index" 1343 | dependencies = [ 1344 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1345 | ] 1346 | 1347 | [[package]] 1348 | name = "time" 1349 | version = "0.1.42" 1350 | source = "registry+https://github.com/rust-lang/crates.io-index" 1351 | dependencies = [ 1352 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 1353 | "redox_syscall 0.1.56 (registry+https://github.com/rust-lang/crates.io-index)", 1354 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 1355 | ] 1356 | 1357 | [[package]] 1358 | name = "tinytemplate" 1359 | version = "1.0.2" 1360 | source = "registry+https://github.com/rust-lang/crates.io-index" 1361 | dependencies = [ 1362 | "serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)", 1363 | "serde_json 1.0.41 (registry+https://github.com/rust-lang/crates.io-index)", 1364 | ] 1365 | 1366 | [[package]] 1367 | name = "tokio" 1368 | version = "0.2.0-alpha.6" 1369 | source = "registry+https://github.com/rust-lang/crates.io-index" 1370 | dependencies = [ 1371 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 1372 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1373 | "futures-sink-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1374 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1375 | "num_cpus 1.10.1 (registry+https://github.com/rust-lang/crates.io-index)", 1376 | "tokio-codec 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1377 | "tokio-executor 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1378 | "tokio-fs 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1379 | "tokio-io 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1380 | "tokio-macros 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1381 | "tokio-net 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1382 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1383 | "tokio-timer 0.3.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1384 | "tracing-core 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 1385 | ] 1386 | 1387 | [[package]] 1388 | name = "tokio-codec" 1389 | version = "0.2.0-alpha.6" 1390 | source = "registry+https://github.com/rust-lang/crates.io-index" 1391 | dependencies = [ 1392 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 1393 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1394 | "futures-sink-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1395 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 1396 | "tokio-io 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1397 | ] 1398 | 1399 | [[package]] 1400 | name = "tokio-executor" 1401 | version = "0.2.0-alpha.6" 1402 | source = "registry+https://github.com/rust-lang/crates.io-index" 1403 | dependencies = [ 1404 | "crossbeam-channel 0.3.9 (registry+https://github.com/rust-lang/crates.io-index)", 1405 | "crossbeam-deque 0.7.1 (registry+https://github.com/rust-lang/crates.io-index)", 1406 | "crossbeam-queue 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 1407 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 1408 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1409 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1410 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1411 | "num_cpus 1.10.1 (registry+https://github.com/rust-lang/crates.io-index)", 1412 | "slab 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1413 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1414 | "tracing 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", 1415 | ] 1416 | 1417 | [[package]] 1418 | name = "tokio-fs" 1419 | version = "0.2.0-alpha.6" 1420 | source = "registry+https://github.com/rust-lang/crates.io-index" 1421 | dependencies = [ 1422 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1423 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1424 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1425 | "tokio-executor 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1426 | "tokio-io 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1427 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1428 | ] 1429 | 1430 | [[package]] 1431 | name = "tokio-io" 1432 | version = "0.2.0-alpha.6" 1433 | source = "registry+https://github.com/rust-lang/crates.io-index" 1434 | dependencies = [ 1435 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 1436 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1437 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 1438 | "memchr 2.2.1 (registry+https://github.com/rust-lang/crates.io-index)", 1439 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1440 | ] 1441 | 1442 | [[package]] 1443 | name = "tokio-macros" 1444 | version = "0.2.0-alpha.6" 1445 | source = "registry+https://github.com/rust-lang/crates.io-index" 1446 | dependencies = [ 1447 | "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 1448 | "syn 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 1449 | ] 1450 | 1451 | [[package]] 1452 | name = "tokio-net" 1453 | version = "0.2.0-alpha.6" 1454 | source = "registry+https://github.com/rust-lang/crates.io-index" 1455 | dependencies = [ 1456 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 1457 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 1458 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1459 | "futures-sink-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1460 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1461 | "iovec 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 1462 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1463 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 1464 | "mio 0.6.19 (registry+https://github.com/rust-lang/crates.io-index)", 1465 | "mio-uds 0.6.7 (registry+https://github.com/rust-lang/crates.io-index)", 1466 | "num_cpus 1.10.1 (registry+https://github.com/rust-lang/crates.io-index)", 1467 | "parking_lot 0.9.0 (registry+https://github.com/rust-lang/crates.io-index)", 1468 | "signal-hook-registry 1.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 1469 | "slab 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1470 | "tokio-codec 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1471 | "tokio-executor 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1472 | "tokio-io 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1473 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1474 | "tracing 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", 1475 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 1476 | ] 1477 | 1478 | [[package]] 1479 | name = "tokio-sync" 1480 | version = "0.2.0-alpha.6" 1481 | source = "registry+https://github.com/rust-lang/crates.io-index" 1482 | dependencies = [ 1483 | "fnv 1.0.6 (registry+https://github.com/rust-lang/crates.io-index)", 1484 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1485 | "futures-sink-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1486 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1487 | ] 1488 | 1489 | [[package]] 1490 | name = "tokio-timer" 1491 | version = "0.3.0-alpha.6" 1492 | source = "registry+https://github.com/rust-lang/crates.io-index" 1493 | dependencies = [ 1494 | "crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)", 1495 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1496 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1497 | "slab 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1498 | "tokio-executor 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1499 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1500 | ] 1501 | 1502 | [[package]] 1503 | name = "tonic" 1504 | version = "0.1.0-alpha.1" 1505 | source = "git+https://github.com/hyperium/tonic#01e72d9a95c2dad6bf03db25ddb597b3c972981e" 1506 | dependencies = [ 1507 | "async-stream 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 1508 | "async-trait 0.1.13 (registry+https://github.com/rust-lang/crates.io-index)", 1509 | "base64 0.10.1 (registry+https://github.com/rust-lang/crates.io-index)", 1510 | "bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)", 1511 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1512 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1513 | "http 0.1.18 (registry+https://github.com/rust-lang/crates.io-index)", 1514 | "http-body 0.2.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1515 | "hyper 0.13.0-alpha.3 (registry+https://github.com/rust-lang/crates.io-index)", 1516 | "percent-encoding 1.0.1 (registry+https://github.com/rust-lang/crates.io-index)", 1517 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1518 | "prost 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 1519 | "prost-derive 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 1520 | "tokio 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1521 | "tokio-codec 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1522 | "tower 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1523 | "tower-balance 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1524 | "tower-load 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1525 | "tower-make 0.3.0-alpha.2a (registry+https://github.com/rust-lang/crates.io-index)", 1526 | "tower-reconnect 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1527 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1528 | "tracing 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", 1529 | ] 1530 | 1531 | [[package]] 1532 | name = "tonic-build" 1533 | version = "0.1.0-alpha.1" 1534 | source = "git+https://github.com/hyperium/tonic#01e72d9a95c2dad6bf03db25ddb597b3c972981e" 1535 | dependencies = [ 1536 | "proc-macro2 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 1537 | "prost-build 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)", 1538 | "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 1539 | "syn 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 1540 | ] 1541 | 1542 | [[package]] 1543 | name = "tower" 1544 | version = "0.3.0-alpha.2" 1545 | source = "registry+https://github.com/rust-lang/crates.io-index" 1546 | dependencies = [ 1547 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1548 | "tower-buffer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1549 | "tower-discover 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1550 | "tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1551 | "tower-limit 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1552 | "tower-load-shed 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1553 | "tower-retry 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1554 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1555 | "tower-timeout 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1556 | "tower-util 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1557 | ] 1558 | 1559 | [[package]] 1560 | name = "tower-balance" 1561 | version = "0.3.0-alpha.2" 1562 | source = "registry+https://github.com/rust-lang/crates.io-index" 1563 | dependencies = [ 1564 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1565 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1566 | "indexmap 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)", 1567 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1568 | "rand 0.6.5 (registry+https://github.com/rust-lang/crates.io-index)", 1569 | "slab 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1570 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1571 | "tokio-timer 0.3.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1572 | "tower-discover 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1573 | "tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1574 | "tower-load 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1575 | "tower-make 0.3.0-alpha.2a (registry+https://github.com/rust-lang/crates.io-index)", 1576 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1577 | "tracing 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", 1578 | ] 1579 | 1580 | [[package]] 1581 | name = "tower-buffer" 1582 | version = "0.3.0-alpha.2" 1583 | source = "registry+https://github.com/rust-lang/crates.io-index" 1584 | dependencies = [ 1585 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1586 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1587 | "tokio-executor 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1588 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1589 | "tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1590 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1591 | "tracing 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", 1592 | ] 1593 | 1594 | [[package]] 1595 | name = "tower-discover" 1596 | version = "0.3.0-alpha.2" 1597 | source = "registry+https://github.com/rust-lang/crates.io-index" 1598 | dependencies = [ 1599 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1600 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1601 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1602 | ] 1603 | 1604 | [[package]] 1605 | name = "tower-layer" 1606 | version = "0.3.0-alpha.2" 1607 | source = "registry+https://github.com/rust-lang/crates.io-index" 1608 | 1609 | [[package]] 1610 | name = "tower-limit" 1611 | version = "0.3.0-alpha.2" 1612 | source = "registry+https://github.com/rust-lang/crates.io-index" 1613 | dependencies = [ 1614 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1615 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1616 | "tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1617 | "tokio-timer 0.3.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1618 | "tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1619 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1620 | ] 1621 | 1622 | [[package]] 1623 | name = "tower-load" 1624 | version = "0.3.0-alpha.2" 1625 | source = "registry+https://github.com/rust-lang/crates.io-index" 1626 | dependencies = [ 1627 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1628 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 1629 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1630 | "tokio-timer 0.3.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1631 | "tower-discover 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1632 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1633 | ] 1634 | 1635 | [[package]] 1636 | name = "tower-load-shed" 1637 | version = "0.3.0-alpha.2" 1638 | source = "registry+https://github.com/rust-lang/crates.io-index" 1639 | dependencies = [ 1640 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1641 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1642 | "tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1643 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1644 | ] 1645 | 1646 | [[package]] 1647 | name = "tower-make" 1648 | version = "0.3.0-alpha.2a" 1649 | source = "registry+https://github.com/rust-lang/crates.io-index" 1650 | dependencies = [ 1651 | "tokio-io 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1652 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1653 | ] 1654 | 1655 | [[package]] 1656 | name = "tower-reconnect" 1657 | version = "0.3.0-alpha.2" 1658 | source = "registry+https://github.com/rust-lang/crates.io-index" 1659 | dependencies = [ 1660 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 1661 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1662 | "tower-make 0.3.0-alpha.2a (registry+https://github.com/rust-lang/crates.io-index)", 1663 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1664 | ] 1665 | 1666 | [[package]] 1667 | name = "tower-retry" 1668 | version = "0.3.0-alpha.2" 1669 | source = "registry+https://github.com/rust-lang/crates.io-index" 1670 | dependencies = [ 1671 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1672 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1673 | "tokio-timer 0.3.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1674 | "tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1675 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1676 | ] 1677 | 1678 | [[package]] 1679 | name = "tower-service" 1680 | version = "0.3.0-alpha.2" 1681 | source = "registry+https://github.com/rust-lang/crates.io-index" 1682 | 1683 | [[package]] 1684 | name = "tower-timeout" 1685 | version = "0.3.0-alpha.2" 1686 | source = "registry+https://github.com/rust-lang/crates.io-index" 1687 | dependencies = [ 1688 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1689 | "tokio-timer 0.3.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)", 1690 | "tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1691 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1692 | ] 1693 | 1694 | [[package]] 1695 | name = "tower-util" 1696 | version = "0.3.0-alpha.2" 1697 | source = "registry+https://github.com/rust-lang/crates.io-index" 1698 | dependencies = [ 1699 | "futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1700 | "futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)", 1701 | "pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)", 1702 | "tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1703 | "tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)", 1704 | ] 1705 | 1706 | [[package]] 1707 | name = "tracing" 1708 | version = "0.1.9" 1709 | source = "registry+https://github.com/rust-lang/crates.io-index" 1710 | dependencies = [ 1711 | "cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)", 1712 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 1713 | "spin 0.5.2 (registry+https://github.com/rust-lang/crates.io-index)", 1714 | "tracing-attributes 0.1.4 (registry+https://github.com/rust-lang/crates.io-index)", 1715 | "tracing-core 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 1716 | ] 1717 | 1718 | [[package]] 1719 | name = "tracing-attributes" 1720 | version = "0.1.4" 1721 | source = "registry+https://github.com/rust-lang/crates.io-index" 1722 | dependencies = [ 1723 | "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", 1724 | "syn 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 1725 | ] 1726 | 1727 | [[package]] 1728 | name = "tracing-core" 1729 | version = "0.1.6" 1730 | source = "registry+https://github.com/rust-lang/crates.io-index" 1731 | dependencies = [ 1732 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1733 | "spin 0.5.2 (registry+https://github.com/rust-lang/crates.io-index)", 1734 | ] 1735 | 1736 | [[package]] 1737 | name = "tracing-futures" 1738 | version = "0.0.1-alpha.1" 1739 | source = "registry+https://github.com/rust-lang/crates.io-index" 1740 | dependencies = [ 1741 | "pin-utils 0.1.0-alpha.4 (registry+https://github.com/rust-lang/crates.io-index)", 1742 | "tracing 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", 1743 | ] 1744 | 1745 | [[package]] 1746 | name = "tracing-log" 1747 | version = "0.1.0" 1748 | source = "registry+https://github.com/rust-lang/crates.io-index" 1749 | dependencies = [ 1750 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1751 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 1752 | "tracing-core 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 1753 | ] 1754 | 1755 | [[package]] 1756 | name = "tracing-subscriber" 1757 | version = "0.1.4" 1758 | source = "registry+https://github.com/rust-lang/crates.io-index" 1759 | dependencies = [ 1760 | "ansi_term 0.11.0 (registry+https://github.com/rust-lang/crates.io-index)", 1761 | "chrono 0.4.9 (registry+https://github.com/rust-lang/crates.io-index)", 1762 | "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1763 | "matchers 0.0.1 (registry+https://github.com/rust-lang/crates.io-index)", 1764 | "owning_ref 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1765 | "regex 1.3.1 (registry+https://github.com/rust-lang/crates.io-index)", 1766 | "smallvec 0.6.10 (registry+https://github.com/rust-lang/crates.io-index)", 1767 | "tracing-core 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)", 1768 | "tracing-log 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)", 1769 | ] 1770 | 1771 | [[package]] 1772 | name = "try-lock" 1773 | version = "0.2.2" 1774 | source = "registry+https://github.com/rust-lang/crates.io-index" 1775 | 1776 | [[package]] 1777 | name = "unicode-segmentation" 1778 | version = "1.3.0" 1779 | source = "registry+https://github.com/rust-lang/crates.io-index" 1780 | 1781 | [[package]] 1782 | name = "unicode-width" 1783 | version = "0.1.6" 1784 | source = "registry+https://github.com/rust-lang/crates.io-index" 1785 | 1786 | [[package]] 1787 | name = "unicode-xid" 1788 | version = "0.1.0" 1789 | source = "registry+https://github.com/rust-lang/crates.io-index" 1790 | 1791 | [[package]] 1792 | name = "unicode-xid" 1793 | version = "0.2.0" 1794 | source = "registry+https://github.com/rust-lang/crates.io-index" 1795 | 1796 | [[package]] 1797 | name = "utf8-ranges" 1798 | version = "1.0.4" 1799 | source = "registry+https://github.com/rust-lang/crates.io-index" 1800 | 1801 | [[package]] 1802 | name = "uuid" 1803 | version = "0.7.4" 1804 | source = "registry+https://github.com/rust-lang/crates.io-index" 1805 | dependencies = [ 1806 | "rand 0.6.5 (registry+https://github.com/rust-lang/crates.io-index)", 1807 | ] 1808 | 1809 | [[package]] 1810 | name = "vec_map" 1811 | version = "0.8.1" 1812 | source = "registry+https://github.com/rust-lang/crates.io-index" 1813 | 1814 | [[package]] 1815 | name = "walkdir" 1816 | version = "2.2.9" 1817 | source = "registry+https://github.com/rust-lang/crates.io-index" 1818 | dependencies = [ 1819 | "same-file 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)", 1820 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 1821 | "winapi-util 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", 1822 | ] 1823 | 1824 | [[package]] 1825 | name = "want" 1826 | version = "0.3.0" 1827 | source = "registry+https://github.com/rust-lang/crates.io-index" 1828 | dependencies = [ 1829 | "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", 1830 | "try-lock 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)", 1831 | ] 1832 | 1833 | [[package]] 1834 | name = "wasi" 1835 | version = "0.7.0" 1836 | source = "registry+https://github.com/rust-lang/crates.io-index" 1837 | 1838 | [[package]] 1839 | name = "which" 1840 | version = "2.0.1" 1841 | source = "registry+https://github.com/rust-lang/crates.io-index" 1842 | dependencies = [ 1843 | "failure 0.1.5 (registry+https://github.com/rust-lang/crates.io-index)", 1844 | "libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)", 1845 | ] 1846 | 1847 | [[package]] 1848 | name = "winapi" 1849 | version = "0.2.8" 1850 | source = "registry+https://github.com/rust-lang/crates.io-index" 1851 | 1852 | [[package]] 1853 | name = "winapi" 1854 | version = "0.3.8" 1855 | source = "registry+https://github.com/rust-lang/crates.io-index" 1856 | dependencies = [ 1857 | "winapi-i686-pc-windows-gnu 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1858 | "winapi-x86_64-pc-windows-gnu 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)", 1859 | ] 1860 | 1861 | [[package]] 1862 | name = "winapi-build" 1863 | version = "0.1.1" 1864 | source = "registry+https://github.com/rust-lang/crates.io-index" 1865 | 1866 | [[package]] 1867 | name = "winapi-i686-pc-windows-gnu" 1868 | version = "0.4.0" 1869 | source = "registry+https://github.com/rust-lang/crates.io-index" 1870 | 1871 | [[package]] 1872 | name = "winapi-util" 1873 | version = "0.1.2" 1874 | source = "registry+https://github.com/rust-lang/crates.io-index" 1875 | dependencies = [ 1876 | "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", 1877 | ] 1878 | 1879 | [[package]] 1880 | name = "winapi-x86_64-pc-windows-gnu" 1881 | version = "0.4.0" 1882 | source = "registry+https://github.com/rust-lang/crates.io-index" 1883 | 1884 | [[package]] 1885 | name = "ws2_32-sys" 1886 | version = "0.2.1" 1887 | source = "registry+https://github.com/rust-lang/crates.io-index" 1888 | dependencies = [ 1889 | "winapi 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)", 1890 | "winapi-build 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", 1891 | ] 1892 | 1893 | [metadata] 1894 | "checksum aho-corasick 0.7.6 (registry+https://github.com/rust-lang/crates.io-index)" = "58fb5e95d83b38284460a5fda7d6470aa0b8844d283a0b614b8535e880800d2d" 1895 | "checksum ansi_term 0.11.0 (registry+https://github.com/rust-lang/crates.io-index)" = "ee49baf6cb617b853aa8d93bf420db2383fab46d314482ca2803b40d5fde979b" 1896 | "checksum arc-swap 0.4.3 (registry+https://github.com/rust-lang/crates.io-index)" = "f1a1eca3195b729bbd64e292ef2f5fff6b1c28504fed762ce2b1013dde4d8e92" 1897 | "checksum arrayvec 0.4.11 (registry+https://github.com/rust-lang/crates.io-index)" = "b8d73f9beda665eaa98ab9e4f7442bd4e7de6652587de55b2525e52e29c1b0ba" 1898 | "checksum async-stream 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)" = "fb6fa015ebe961e9908ca4c1854e7dc7aabd4417da77b6a0466e4dfb4c8f6f69" 1899 | "checksum async-stream-impl 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "4f0d8c5b411e36dcfb04388bacfec54795726b1f0148adcb0f377a96d6747e0e" 1900 | "checksum async-trait 0.1.13 (registry+https://github.com/rust-lang/crates.io-index)" = "6f1418cdc602e31e3ffeea57d8481b8c5db895e733028c2552a06d5cf36beb91" 1901 | "checksum atty 0.2.13 (registry+https://github.com/rust-lang/crates.io-index)" = "1803c647a3ec87095e7ae7acfca019e98de5ec9a7d01343f611cf3152ed71a90" 1902 | "checksum autocfg 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)" = "b671c8fb71b457dd4ae18c4ba1e59aa81793daacc361d82fcd410cef0d491875" 1903 | "checksum backtrace 0.3.38 (registry+https://github.com/rust-lang/crates.io-index)" = "690a62be8920ccf773ee00ef0968649b0e724cda8bd5b12286302b4ae955fdf5" 1904 | "checksum backtrace-sys 0.1.31 (registry+https://github.com/rust-lang/crates.io-index)" = "82a830b4ef2d1124a711c71d263c5abdc710ef8e907bd508c88be475cebc422b" 1905 | "checksum base64 0.10.1 (registry+https://github.com/rust-lang/crates.io-index)" = "0b25d992356d2eb0ed82172f5248873db5560c4721f564b13cb5193bda5e668e" 1906 | "checksum bincode 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)" = "b8ab639324e3ee8774d296864fbc0dbbb256cf1a41c490b94cba90c082915f92" 1907 | "checksum bitflags 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)" = "8a606a02debe2813760609f57a64a2ffd27d9fdf5b2f133eaca0b248dd92cdd2" 1908 | "checksum bstr 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)" = "8d6c2c5b58ab920a4f5aeaaca34b4488074e8cc7596af94e6f8c6ff247c60245" 1909 | "checksum byteorder 1.3.2 (registry+https://github.com/rust-lang/crates.io-index)" = "a7c3dd8985a7111efc5c80b44e23ecdd8c007de8ade3b96595387e812b957cf5" 1910 | "checksum bytes 0.4.12 (registry+https://github.com/rust-lang/crates.io-index)" = "206fdffcfa2df7cbe15601ef46c813fce0965eb3286db6b56c583b814b51c81c" 1911 | "checksum c2-chacha 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)" = "7d64d04786e0f528460fc884753cf8dddcc466be308f6026f8e355c41a0e4101" 1912 | "checksum cast 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)" = "926013f2860c46252efceabb19f4a6b308197505082c609025aa6706c011d427" 1913 | "checksum cc 1.0.45 (registry+https://github.com/rust-lang/crates.io-index)" = "4fc9a35e1f4290eb9e5fc54ba6cf40671ed2a2514c3eeb2b2a908dda2ea5a1be" 1914 | "checksum cfg-if 0.1.10 (registry+https://github.com/rust-lang/crates.io-index)" = "4785bdd1c96b2a846b2bd7cc02e86b6b3dbf14e7e53446c4f54c92a361040822" 1915 | "checksum chrono 0.4.9 (registry+https://github.com/rust-lang/crates.io-index)" = "e8493056968583b0193c1bb04d6f7684586f3726992d6c573261941a895dbd68" 1916 | "checksum clap 2.33.0 (registry+https://github.com/rust-lang/crates.io-index)" = "5067f5bb2d80ef5d68b4c87db81601f0b75bca627bc2ef76b141d7b846a3c6d9" 1917 | "checksum cloudabi 0.0.3 (registry+https://github.com/rust-lang/crates.io-index)" = "ddfc5b9aa5d4507acaf872de71051dfd0e309860e88966e1051e462a077aac4f" 1918 | "checksum crc32fast 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)" = "ba125de2af0df55319f41944744ad91c71113bf74a4646efff39afe1f6842db1" 1919 | "checksum criterion 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)" = "938703e165481c8d612ea3479ac8342e5615185db37765162e762ec3523e2fc6" 1920 | "checksum criterion-plot 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)" = "eccdc6ce8bbe352ca89025bee672aa6d24f4eb8c53e3a8b5d1bc58011da072a2" 1921 | "checksum crossbeam-channel 0.3.9 (registry+https://github.com/rust-lang/crates.io-index)" = "c8ec7fcd21571dc78f96cc96243cab8d8f035247c3efd16c687be154c3fa9efa" 1922 | "checksum crossbeam-deque 0.7.1 (registry+https://github.com/rust-lang/crates.io-index)" = "b18cd2e169ad86297e6bc0ad9aa679aee9daa4f19e8163860faf7c164e4f5a71" 1923 | "checksum crossbeam-epoch 0.7.2 (registry+https://github.com/rust-lang/crates.io-index)" = "fedcd6772e37f3da2a9af9bf12ebe046c0dfe657992377b4df982a2b54cd37a9" 1924 | "checksum crossbeam-queue 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)" = "7c979cd6cfe72335896575c6b5688da489e420d36a27a0b9eb0c73db574b4a4b" 1925 | "checksum crossbeam-utils 0.6.6 (registry+https://github.com/rust-lang/crates.io-index)" = "04973fa96e96579258a5091af6003abde64af786b860f18622b82e026cca60e6" 1926 | "checksum csv 1.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "37519ccdfd73a75821cac9319d4fce15a81b9fcf75f951df5b9988aa3a0af87d" 1927 | "checksum csv-core 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)" = "9b5cadb6b25c77aeff80ba701712494213f4a8418fcda2ee11b6560c3ad0bf4c" 1928 | "checksum either 1.5.3 (registry+https://github.com/rust-lang/crates.io-index)" = "bb1f6b1ce1c140482ea30ddd3335fc0024ac7ee112895426e0a629a6c20adfe3" 1929 | "checksum failure 0.1.5 (registry+https://github.com/rust-lang/crates.io-index)" = "795bd83d3abeb9220f257e597aa0080a508b27533824adf336529648f6abf7e2" 1930 | "checksum fixedbitset 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)" = "86d4de0081402f5e88cdac65c8dcdcc73118c1a7a465e2a05f0da05843a8ea33" 1931 | "checksum fnv 1.0.6 (registry+https://github.com/rust-lang/crates.io-index)" = "2fad85553e09a6f881f739c29f0b00b0f01357c743266d478b68951ce23285f3" 1932 | "checksum fs2 0.4.3 (registry+https://github.com/rust-lang/crates.io-index)" = "9564fc758e15025b46aa6643b1b77d047d1a56a1aea6e01002ac0c7026876213" 1933 | "checksum fuchsia-cprng 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "a06f77d526c1a601b7c4cdd98f54b5eaabffc14d5f2f0296febdc7f357c6d3ba" 1934 | "checksum fuchsia-zircon 0.3.3 (registry+https://github.com/rust-lang/crates.io-index)" = "2e9763c69ebaae630ba35f74888db465e49e259ba1bc0eda7d06f4a067615d82" 1935 | "checksum fuchsia-zircon-sys 0.3.3 (registry+https://github.com/rust-lang/crates.io-index)" = "3dcaa9ae7725d12cdb85b3ad99a434db70b468c09ded17e012d86b5c1010f7a7" 1936 | "checksum futures-channel-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)" = "d5e5f4df964fa9c1c2f8bddeb5c3611631cacd93baf810fc8bb2fb4b495c263a" 1937 | "checksum futures-core-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)" = "b35b6263fb1ef523c3056565fa67b1d16f0a8604ff12b11b08c25f28a734c60a" 1938 | "checksum futures-sink-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)" = "86f148ef6b69f75bb610d4f9a2336d4fc88c4b5b67129d1a340dd0fd362efeec" 1939 | "checksum futures-util-preview 0.3.0-alpha.19 (registry+https://github.com/rust-lang/crates.io-index)" = "5ce968633c17e5f97936bd2797b6e38fb56cf16a7422319f7ec2e30d3c470e8d" 1940 | "checksum fxhash 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)" = "c31b6d751ae2c7f11320402d34e41349dd1016f8d5d45e48c4312bc8625af50c" 1941 | "checksum getrandom 0.1.12 (registry+https://github.com/rust-lang/crates.io-index)" = "473a1265acc8ff1e808cd0a1af8cee3c2ee5200916058a2ca113c29f2d903571" 1942 | "checksum getset 0.0.7 (registry+https://github.com/rust-lang/crates.io-index)" = "19fbde0fad0c1c1f9474694b1f5c9ba22b09f2f74f74e6d2bd19c43f6656e2cb" 1943 | "checksum h2 0.2.0-alpha.3 (registry+https://github.com/rust-lang/crates.io-index)" = "0f107db1419ef8271686187b1a5d47c6431af4a7f4d98b495e7b7fc249bb0a78" 1944 | "checksum hashbrown 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)" = "e1de41fb8dba9714efd92241565cdff73f78508c95697dd56787d3cba27e2353" 1945 | "checksum heck 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)" = "20564e78d53d2bb135c343b3f47714a56af2061f1c928fdb541dc7b9fdd94205" 1946 | "checksum http 0.1.18 (registry+https://github.com/rust-lang/crates.io-index)" = "372bcb56f939e449117fb0869c2e8fd8753a8223d92a172c6e808cf123a5b6e4" 1947 | "checksum http-body 0.2.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "a679027a9151a6bde1aeefc9c78ff02970ff3f03f82749063e9fd9127142d5d7" 1948 | "checksum httparse 1.3.4 (registry+https://github.com/rust-lang/crates.io-index)" = "cd179ae861f0c2e53da70d892f5f3029f9594be0c41dc5269cd371691b1dc2f9" 1949 | "checksum hyper 0.13.0-alpha.3 (registry+https://github.com/rust-lang/crates.io-index)" = "61b1ad878f0787e12d5f2a4a46633ee9a3ab17ed4dc1d317bd4a06981a2d1fdc" 1950 | "checksum indexmap 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)" = "a61202fbe46c4a951e9404a720a0180bcf3212c750d735cb5c4ba4dc551299f3" 1951 | "checksum iovec 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)" = "dbe6e417e7d0975db6512b90796e8ce223145ac4e33c377e4a42882a0e88bb08" 1952 | "checksum itertools 0.8.0 (registry+https://github.com/rust-lang/crates.io-index)" = "5b8467d9c1cebe26feb08c640139247fac215782d35371ade9a2136ed6085358" 1953 | "checksum itoa 0.4.4 (registry+https://github.com/rust-lang/crates.io-index)" = "501266b7edd0174f8530248f87f99c88fbe60ca4ef3dd486835b8d8d53136f7f" 1954 | "checksum kernel32-sys 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)" = "7507624b29483431c0ba2d82aece8ca6cdba9382bff4ddd0f7490560c056098d" 1955 | "checksum lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)" = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" 1956 | "checksum libc 0.2.62 (registry+https://github.com/rust-lang/crates.io-index)" = "34fcd2c08d2f832f376f4173a231990fa5aef4e99fb569867318a227ef4c06ba" 1957 | "checksum lock_api 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)" = "f8912e782533a93a167888781b836336a6ca5da6175c05944c86cf28c31104dc" 1958 | "checksum log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)" = "14b6052be84e6b71ab17edffc2eeabf5c2c3ae1fdb464aae35ac50c67a44e1f7" 1959 | "checksum matchers 0.0.1 (registry+https://github.com/rust-lang/crates.io-index)" = "f099785f7595cc4b4553a174ce30dd7589ef93391ff414dbb67f62392b9e0ce1" 1960 | "checksum memchr 2.2.1 (registry+https://github.com/rust-lang/crates.io-index)" = "88579771288728879b57485cc7d6b07d648c9f0141eb955f8ab7f9d45394468e" 1961 | "checksum memoffset 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)" = "ce6075db033bbbb7ee5a0bbd3a3186bbae616f57fb001c485c7ff77955f8177f" 1962 | "checksum mio 0.6.19 (registry+https://github.com/rust-lang/crates.io-index)" = "83f51996a3ed004ef184e16818edc51fadffe8e7ca68be67f9dee67d84d0ff23" 1963 | "checksum mio-uds 0.6.7 (registry+https://github.com/rust-lang/crates.io-index)" = "966257a94e196b11bb43aca423754d87429960a768de9414f3691d6957abf125" 1964 | "checksum miow 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)" = "8c1f2f3b1cf331de6896aabf6e9d55dca90356cc9960cca7eaaf408a355ae919" 1965 | "checksum multimap 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)" = "2eb04b9f127583ed176e163fb9ec6f3e793b87e21deedd5734a69386a18a0151" 1966 | "checksum net2 0.2.33 (registry+https://github.com/rust-lang/crates.io-index)" = "42550d9fb7b6684a6d404d9fa7250c2eb2646df731d1c06afc06dcee9e1bcf88" 1967 | "checksum nodrop 0.1.13 (registry+https://github.com/rust-lang/crates.io-index)" = "2f9667ddcc6cc8a43afc9b7917599d7216aa09c463919ea32c59ed6cac8bc945" 1968 | "checksum num-integer 0.1.41 (registry+https://github.com/rust-lang/crates.io-index)" = "b85e541ef8255f6cf42bbfe4ef361305c6c135d10919ecc26126c4e5ae94bc09" 1969 | "checksum num-traits 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)" = "6ba9a427cfca2be13aa6f6403b0b7e7368fe982bfa16fccc450ce74c46cd9b32" 1970 | "checksum num_cpus 1.10.1 (registry+https://github.com/rust-lang/crates.io-index)" = "bcef43580c035376c0705c42792c294b66974abbfd2789b511784023f71f3273" 1971 | "checksum owning_ref 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)" = "49a4b8ea2179e6a2e27411d3bca09ca6dd630821cf6894c6c7c8467a8ee7ef13" 1972 | "checksum pagecache 0.19.4 (registry+https://github.com/rust-lang/crates.io-index)" = "3a3cb2f80e81d64e087e9de0dedd9d95c4360aae2822917d5e4994bf1b2e0070" 1973 | "checksum parking_lot 0.9.0 (registry+https://github.com/rust-lang/crates.io-index)" = "f842b1982eb6c2fe34036a4fbfb06dd185a3f5c8edfaacdf7d1ea10b07de6252" 1974 | "checksum parking_lot_core 0.6.2 (registry+https://github.com/rust-lang/crates.io-index)" = "b876b1b9e7ac6e1a74a6da34d25c42e17e8862aa409cbbbdcfc8d86c6f3bc62b" 1975 | "checksum percent-encoding 1.0.1 (registry+https://github.com/rust-lang/crates.io-index)" = "31010dd2e1ac33d5b46a5b413495239882813e0369f8ed8a5e266f173602f831" 1976 | "checksum petgraph 0.4.13 (registry+https://github.com/rust-lang/crates.io-index)" = "9c3659d1ee90221741f65dd128d9998311b0e40c5d3c23a62445938214abce4f" 1977 | "checksum pin-project 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)" = "3d9156ea5979ae30ecc0460cd848738daf24cfb89eb11a41e0c369ba1f0e6aeb" 1978 | "checksum pin-project-internal 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)" = "1a375fffcd7bf53d8302fb95c1e2f3e0a1a92bd57edcab796f26f9e527c2f3da" 1979 | "checksum pin-utils 0.1.0-alpha.4 (registry+https://github.com/rust-lang/crates.io-index)" = "5894c618ce612a3fa23881b152b608bafb8c56cfc22f434a3ba3120b40f7b587" 1980 | "checksum ppv-lite86 0.2.5 (registry+https://github.com/rust-lang/crates.io-index)" = "e3cbf9f658cdb5000fcf6f362b8ea2ba154b9f146a61c7a20d647034c6b6561b" 1981 | "checksum proc-macro2 0.4.30 (registry+https://github.com/rust-lang/crates.io-index)" = "cf3d2011ab5c909338f7887f4fc896d35932e29146c12c8d01da6b22a80ba759" 1982 | "checksum proc-macro2 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)" = "90cf5f418035b98e655e9cdb225047638296b862b42411c4e45bb88d700f7fc0" 1983 | "checksum prost 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)" = "96d14b1c185652833d24aaad41c5832b0be5616a590227c1fbff57c616754b23" 1984 | "checksum prost-build 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)" = "eb788126ea840817128183f8f603dce02cb7aea25c2a0b764359d8e20010702e" 1985 | "checksum prost-derive 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)" = "5e7dc378b94ac374644181a2247cebf59a6ec1c88b49ac77f3a94b86b79d0e11" 1986 | "checksum prost-types 0.5.0 (registry+https://github.com/rust-lang/crates.io-index)" = "1de482a366941c8d56d19b650fac09ca08508f2a696119ee7513ad590c8bac6f" 1987 | "checksum protobuf 2.8.1 (registry+https://github.com/rust-lang/crates.io-index)" = "40361836defdd5871ff7e84096c6f6444af7fc157f8ef1789f54f147687caa20" 1988 | "checksum protobuf-build 0.10.0 (registry+https://github.com/rust-lang/crates.io-index)" = "211897988382ac761fa302992638ec837b544df73e6fb2ffc022c8515a0563f2" 1989 | "checksum quick-error 1.2.2 (registry+https://github.com/rust-lang/crates.io-index)" = "9274b940887ce9addde99c4eee6b5c44cc494b182b97e73dc8ffdcb3397fd3f0" 1990 | "checksum quote 0.6.13 (registry+https://github.com/rust-lang/crates.io-index)" = "6ce23b6b870e8f94f81fb0a363d65d86675884b34a09043c81e5562f11c1f8e1" 1991 | "checksum quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)" = "053a8c8bcc71fcce321828dc897a98ab9760bef03a4fc36693c231e5b3216cfe" 1992 | "checksum raft 0.6.0-alpha (git+https://github.com/pingcap/raft-rs)" = "" 1993 | "checksum raft-proto 0.6.0-alpha (git+https://github.com/pingcap/raft-rs)" = "" 1994 | "checksum rand 0.6.5 (registry+https://github.com/rust-lang/crates.io-index)" = "6d71dacdc3c88c1fde3885a3be3fbab9f35724e6ce99467f7d9c5026132184ca" 1995 | "checksum rand 0.7.2 (registry+https://github.com/rust-lang/crates.io-index)" = "3ae1b169243eaf61759b8475a998f0a385e42042370f3a7dbaf35246eacc8412" 1996 | "checksum rand_chacha 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "556d3a1ca6600bfcbab7c7c91ccb085ac7fbbcd70e008a98742e7847f4f7bcef" 1997 | "checksum rand_chacha 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)" = "03a2a90da8c7523f554344f921aa97283eadf6ac484a6d2a7d0212fa7f8d6853" 1998 | "checksum rand_core 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)" = "7a6fdeb83b075e8266dcc8762c22776f6877a63111121f5f8c7411e5be7eed4b" 1999 | "checksum rand_core 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)" = "9c33a3c44ca05fa6f1807d8e6743f3824e8509beca625669633be0acbdf509dc" 2000 | "checksum rand_core 0.5.1 (registry+https://github.com/rust-lang/crates.io-index)" = "90bde5296fc891b0cef12a6d03ddccc162ce7b2aff54160af9338f8d40df6d19" 2001 | "checksum rand_hc 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)" = "7b40677c7be09ae76218dc623efbf7b18e34bced3f38883af07bb75630a21bc4" 2002 | "checksum rand_hc 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)" = "ca3129af7b92a17112d59ad498c6f81eaf463253766b90396d39ea7a39d6613c" 2003 | "checksum rand_isaac 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "ded997c9d5f13925be2a6fd7e66bf1872597f759fd9dd93513dd7e92e5a5ee08" 2004 | "checksum rand_jitter 0.1.4 (registry+https://github.com/rust-lang/crates.io-index)" = "1166d5c91dc97b88d1decc3285bb0a99ed84b05cfd0bc2341bdf2d43fc41e39b" 2005 | "checksum rand_os 0.1.3 (registry+https://github.com/rust-lang/crates.io-index)" = "7b75f676a1e053fc562eafbb47838d67c84801e38fc1ba459e8f180deabd5071" 2006 | "checksum rand_os 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)" = "a788ae3edb696cfcba1c19bfd388cc4b8c21f8a408432b199c072825084da58a" 2007 | "checksum rand_pcg 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)" = "abf9b09b01790cfe0364f52bf32995ea3c39f4d2dd011eac241d2914146d0b44" 2008 | "checksum rand_xorshift 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "cbf7e9e623549b0e21f6e97cf8ecf247c1a8fd2e8a992ae265314300b2455d5c" 2009 | "checksum rand_xoshiro 0.3.1 (registry+https://github.com/rust-lang/crates.io-index)" = "0e18c91676f670f6f0312764c759405f13afb98d5d73819840cf72a518487bff" 2010 | "checksum rayon 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)" = "83a27732a533a1be0a0035a111fe76db89ad312f6f0347004c220c57f209a123" 2011 | "checksum rayon-core 1.6.0 (registry+https://github.com/rust-lang/crates.io-index)" = "98dcf634205083b17d0861252431eb2acbfb698ab7478a2d20de07954f47ec7b" 2012 | "checksum rdrand 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)" = "678054eb77286b51581ba43620cc911abf02758c91f93f479767aed0f90458b2" 2013 | "checksum redox_syscall 0.1.56 (registry+https://github.com/rust-lang/crates.io-index)" = "2439c63f3f6139d1b57529d16bc3b8bb855230c8efcc5d3a896c8bea7c3b1e84" 2014 | "checksum regex 1.3.1 (registry+https://github.com/rust-lang/crates.io-index)" = "dc220bd33bdce8f093101afe22a037b8eb0e5af33592e6a9caafff0d4cb81cbd" 2015 | "checksum regex-automata 0.1.8 (registry+https://github.com/rust-lang/crates.io-index)" = "92b73c2a1770c255c240eaa4ee600df1704a38dc3feaa6e949e7fcd4f8dc09f9" 2016 | "checksum regex-syntax 0.6.12 (registry+https://github.com/rust-lang/crates.io-index)" = "11a7e20d1cce64ef2fed88b66d347f88bd9babb82845b2b858f3edbf59a4f716" 2017 | "checksum remove_dir_all 0.5.2 (registry+https://github.com/rust-lang/crates.io-index)" = "4a83fa3702a688b9359eccba92d153ac33fd2e8462f9e0e3fdf155239ea7792e" 2018 | "checksum rustc-demangle 0.1.16 (registry+https://github.com/rust-lang/crates.io-index)" = "4c691c0e608126e00913e33f0ccf3727d5fc84573623b8d65b2df340b5201783" 2019 | "checksum rustc_version 0.2.3 (registry+https://github.com/rust-lang/crates.io-index)" = "138e3e0acb6c9fb258b19b67cb8abd63c00679d2851805ea151465464fe9030a" 2020 | "checksum ryu 1.0.1 (registry+https://github.com/rust-lang/crates.io-index)" = "19d2271fa48eaf61e53cc88b4ad9adcbafa2d512c531e7fadb6dc11a4d3656c5" 2021 | "checksum same-file 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)" = "585e8ddcedc187886a30fa705c47985c3fa88d06624095856b36ca0b82ff4421" 2022 | "checksum scopeguard 1.0.0 (registry+https://github.com/rust-lang/crates.io-index)" = "b42e15e59b18a828bbf5c58ea01debb36b9b096346de35d941dcb89009f24a0d" 2023 | "checksum semver 0.9.0 (registry+https://github.com/rust-lang/crates.io-index)" = "1d7eb9ef2c18661902cc47e535f9bc51b78acd254da71d375c2f6720d9a40403" 2024 | "checksum semver-parser 0.7.0 (registry+https://github.com/rust-lang/crates.io-index)" = "388a1df253eca08550bef6c72392cfe7c30914bf41df5269b68cbd6ff8f570a3" 2025 | "checksum serde 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)" = "9796c9b7ba2ffe7a9ce53c2287dfc48080f4b2b362fcc245a259b3a7201119dd" 2026 | "checksum serde_bytes 0.11.2 (registry+https://github.com/rust-lang/crates.io-index)" = "45af0182ff64abaeea290235eb67da3825a576c5d53e642c4d5b652e12e6effc" 2027 | "checksum serde_derive 1.0.101 (registry+https://github.com/rust-lang/crates.io-index)" = "4b133a43a1ecd55d4086bd5b4dc6c1751c68b1bfbeba7a5040442022c7e7c02e" 2028 | "checksum serde_json 1.0.41 (registry+https://github.com/rust-lang/crates.io-index)" = "2f72eb2a68a7dc3f9a691bfda9305a1c017a6215e5a4545c258500d2099a37c2" 2029 | "checksum signal-hook-registry 1.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "1797d48f38f91643908bb14e35e79928f9f4b3cefb2420a564dde0991b4358dc" 2030 | "checksum slab 0.4.2 (registry+https://github.com/rust-lang/crates.io-index)" = "c111b5bd5695e56cffe5129854aa230b39c93a305372fdbb2668ca2394eea9f8" 2031 | "checksum sled 0.28.0 (registry+https://github.com/rust-lang/crates.io-index)" = "9c3d0ffdcb80d10203517641c514b205d89541b001149df43ae6e902bc19809d" 2032 | "checksum slog 2.5.2 (registry+https://github.com/rust-lang/crates.io-index)" = "1cc9c640a4adbfbcc11ffb95efe5aa7af7309e002adab54b185507dbf2377b99" 2033 | "checksum smallvec 0.6.10 (registry+https://github.com/rust-lang/crates.io-index)" = "ab606a9c5e214920bb66c458cd7be8ef094f813f20fe77a54cc7dbfff220d4b7" 2034 | "checksum spin 0.5.2 (registry+https://github.com/rust-lang/crates.io-index)" = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" 2035 | "checksum stable_deref_trait 1.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "dba1a27d3efae4351c8051072d619e3ade2820635c3958d826bfea39d59b54c8" 2036 | "checksum string 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)" = "d24114bfcceb867ca7f71a0d3fe45d45619ec47a6fbfa98cb14e14250bfa5d6d" 2037 | "checksum strsim 0.8.0 (registry+https://github.com/rust-lang/crates.io-index)" = "8ea5119cdb4c55b55d432abb513a0429384878c15dde60cc77b1c99de1a95a6a" 2038 | "checksum structopt 0.2.18 (registry+https://github.com/rust-lang/crates.io-index)" = "16c2cdbf9cc375f15d1b4141bc48aeef444806655cd0e904207edc8d68d86ed7" 2039 | "checksum structopt-derive 0.2.18 (registry+https://github.com/rust-lang/crates.io-index)" = "53010261a84b37689f9ed7d395165029f9cc7abb9f56bbfe86bee2597ed25107" 2040 | "checksum syn 0.15.44 (registry+https://github.com/rust-lang/crates.io-index)" = "9ca4b3b69a77cbe1ffc9e198781b7acb0c7365a883670e8f1c1bc66fba79a5c5" 2041 | "checksum syn 1.0.5 (registry+https://github.com/rust-lang/crates.io-index)" = "66850e97125af79138385e9b88339cbcd037e3f28ceab8c5ad98e64f0f1f80bf" 2042 | "checksum tempfile 3.1.0 (registry+https://github.com/rust-lang/crates.io-index)" = "7a6e24d9338a0a5be79593e2fa15a648add6138caa803e2d5bc782c371732ca9" 2043 | "checksum textwrap 0.11.0 (registry+https://github.com/rust-lang/crates.io-index)" = "d326610f408c7a4eb6f51c37c330e496b08506c9457c9d34287ecc38809fb060" 2044 | "checksum thread_local 0.3.6 (registry+https://github.com/rust-lang/crates.io-index)" = "c6b53e329000edc2b34dbe8545fd20e55a333362d0a321909685a19bd28c3f1b" 2045 | "checksum time 0.1.42 (registry+https://github.com/rust-lang/crates.io-index)" = "db8dcfca086c1143c9270ac42a2bbd8a7ee477b78ac8e45b19abfb0cbede4b6f" 2046 | "checksum tinytemplate 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)" = "4574b75faccaacddb9b284faecdf0b544b80b6b294f3d062d325c5726a209c20" 2047 | "checksum tokio 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "1f17f5d6ab0f35c1506678b28fb1798bdf74fcb737e9843c7b17b73e426eba38" 2048 | "checksum tokio-codec 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "9f5d22fd1e84bd4045d28813491cb7d7caae34d45c80517c2213f09a85e8787a" 2049 | "checksum tokio-executor 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "9ee9ceecf69145923834ea73f32ba40c790fd877b74a7817dd0b089f1eb9c7c8" 2050 | "checksum tokio-fs 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "0bf85e16971e06e680c622e0c1b455be94b086275c5ddcd6d4a83a2bfbb83cda" 2051 | "checksum tokio-io 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "112784d5543df30660b04a72ca423bfbd90e8bb32f94dcf610f15401218b22c5" 2052 | "checksum tokio-macros 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "86b616374bcdadd95974e1f0dfca07dc913f1163c53840c0d664aca35114964e" 2053 | "checksum tokio-net 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "a441682cd32f3559383112c4a7f372f5c9fa1950c5cf8c8dd05274a2ce8c2654" 2054 | "checksum tokio-sync 0.2.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "4f1aaeb685540f7407ea0e27f1c9757d258c7c6bf4e3eb19da6fc59b747239d2" 2055 | "checksum tokio-timer 0.3.0-alpha.6 (registry+https://github.com/rust-lang/crates.io-index)" = "b97c1587fe71018eb245a4a9daa13a5a3b681bbc1f7fdadfe24720e141472c13" 2056 | "checksum tonic 0.1.0-alpha.1 (git+https://github.com/hyperium/tonic)" = "" 2057 | "checksum tonic-build 0.1.0-alpha.1 (git+https://github.com/hyperium/tonic)" = "" 2058 | "checksum tower 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "b05ce6a5913c351c65d5538545302eceaf3f84e8257654370d298c3ea1a97935" 2059 | "checksum tower-balance 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "38dce028671196b8e89965a7b057b8e4bc99206185c0418ce8d164fe674c9d72" 2060 | "checksum tower-buffer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "eb66caab441682c3eac396617cbc1c2bd8962589468d08be3b05cb6875200694" 2061 | "checksum tower-discover 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "fba86e21286ce7aa7735c9a12801384a34493ae0fae6a9767fafc22b3968a2e9" 2062 | "checksum tower-layer 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "44458625be64d57b07579ddf7408ba5111853474893980ad81c2e9c56ad2d2e2" 2063 | "checksum tower-limit 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "f630ef814d59b52ea79f37ad78d9d88669d11ff29b92eb72d36b3a32af80681a" 2064 | "checksum tower-load 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "87335a79d9928ea6353ff6cc12470584ff16229783c738fbb169c3a13698a5bc" 2065 | "checksum tower-load-shed 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "a2da88d27937d6960520fdfcfbc52c35f7535e65bcb2ddeda3f2246b70eb2de1" 2066 | "checksum tower-make 0.3.0-alpha.2a (registry+https://github.com/rust-lang/crates.io-index)" = "316d47dd40cde4ac5d88110eaf9a10a4e2a68612d9c056cd2aa24e37dcb484cd" 2067 | "checksum tower-reconnect 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "ec3b0ff785d951558016852c0d92f274296f34e7b7fa6e15fd3b8c55f43f1907" 2068 | "checksum tower-retry 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "8bb37b898570d5a50c10c50ee254563853f8445fc23e261cffc747beb1a84618" 2069 | "checksum tower-service 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "63ff37396cd966ce43bea418bfa339f802857495f797dafa00bea5b7221ebdfa" 2070 | "checksum tower-timeout 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "a43f7de8a757264c7949c3cb109a89e4ec9f2a3dd938804d3a0cb521862b35ff" 2071 | "checksum tower-util 0.3.0-alpha.2 (registry+https://github.com/rust-lang/crates.io-index)" = "aef810d487f4d67df42064a69f58572413b9cb37b177d363921762ac5a9eb76b" 2072 | "checksum tracing 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)" = "c21ff9457accc293386c20e8f754d0b059e67e325edf2284f04230d125d7e5ff" 2073 | "checksum tracing-attributes 0.1.4 (registry+https://github.com/rust-lang/crates.io-index)" = "3ff978fd9c9afe2cc9c671e247713421c6406b3422305cbdce5de695d3ab4c3c" 2074 | "checksum tracing-core 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)" = "528c8ebaaa16cdac34795180b046c031775b0d56402704d98c096788f33d646a" 2075 | "checksum tracing-futures 0.0.1-alpha.1 (registry+https://github.com/rust-lang/crates.io-index)" = "08c7446f4fb35df7ba2c537b7e2f812f91b20a58aa2b846f028342c4d2429be0" 2076 | "checksum tracing-log 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)" = "652bc99e1286541d6ccc42d5fb37213d1cdde544f88b19fac3d94e3117b55163" 2077 | "checksum tracing-subscriber 0.1.4 (registry+https://github.com/rust-lang/crates.io-index)" = "c63b1771a75314374703309107e685805628c04643e6dd2c7a5cf8f94348c62e" 2078 | "checksum try-lock 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)" = "e604eb7b43c06650e854be16a2a03155743d3752dd1c943f6829e26b7a36e382" 2079 | "checksum unicode-segmentation 1.3.0 (registry+https://github.com/rust-lang/crates.io-index)" = "1967f4cdfc355b37fd76d2a954fb2ed3871034eb4f26d60537d88795cfc332a9" 2080 | "checksum unicode-width 0.1.6 (registry+https://github.com/rust-lang/crates.io-index)" = "7007dbd421b92cc6e28410fe7362e2e0a2503394908f417b68ec8d1c364c4e20" 2081 | "checksum unicode-xid 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)" = "fc72304796d0818e357ead4e000d19c9c174ab23dc11093ac919054d20a6a7fc" 2082 | "checksum unicode-xid 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)" = "826e7639553986605ec5979c7dd957c7895e93eabed50ab2ffa7f6128a75097c" 2083 | "checksum utf8-ranges 1.0.4 (registry+https://github.com/rust-lang/crates.io-index)" = "b4ae116fef2b7fea257ed6440d3cfcff7f190865f170cdad00bb6465bf18ecba" 2084 | "checksum uuid 0.7.4 (registry+https://github.com/rust-lang/crates.io-index)" = "90dbc611eb48397705a6b0f6e917da23ae517e4d127123d2cf7674206627d32a" 2085 | "checksum vec_map 0.8.1 (registry+https://github.com/rust-lang/crates.io-index)" = "05c78687fb1a80548ae3250346c3db86a80a7cdd77bda190189f2d0a0987c81a" 2086 | "checksum walkdir 2.2.9 (registry+https://github.com/rust-lang/crates.io-index)" = "9658c94fa8b940eab2250bd5a457f9c48b748420d71293b165c8cdbe2f55f71e" 2087 | "checksum want 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)" = "1ce8a968cb1cd110d136ff8b819a556d6fb6d919363c61534f6860c7eb172ba0" 2088 | "checksum wasi 0.7.0 (registry+https://github.com/rust-lang/crates.io-index)" = "b89c3ce4ce14bdc6fb6beaf9ec7928ca331de5df7e5ea278375642a2f478570d" 2089 | "checksum which 2.0.1 (registry+https://github.com/rust-lang/crates.io-index)" = "b57acb10231b9493c8472b20cb57317d0679a49e0bdbee44b3b803a6473af164" 2090 | "checksum winapi 0.2.8 (registry+https://github.com/rust-lang/crates.io-index)" = "167dc9d6949a9b857f3451275e911c3f44255842c1f7a76f33c55103a909087a" 2091 | "checksum winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)" = "8093091eeb260906a183e6ae1abdba2ef5ef2257a21801128899c3fc699229c6" 2092 | "checksum winapi-build 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)" = "2d315eee3b34aca4797b2da6b13ed88266e6d612562a0c46390af8299fc699bc" 2093 | "checksum winapi-i686-pc-windows-gnu 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)" = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" 2094 | "checksum winapi-util 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)" = "7168bab6e1daee33b4557efd0e95d5ca70a03706d39fa5f3fe7a236f584b03c9" 2095 | "checksum winapi-x86_64-pc-windows-gnu 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)" = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" 2096 | "checksum ws2_32-sys 0.2.1 (registry+https://github.com/rust-lang/crates.io-index)" = "d59cefebd0c892fa2dd6de581e937301d8552cb44489cdff035c6187cb63fa5e" 2097 | --------------------------------------------------------------------------------