From 99756696e35e7b3beef100e23d6afd8a84fbf496 Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Sat, 2 Jun 2018 16:49:04 +0300 Subject: [PATCH 01/24] Zer-copy parsing, migrate to capnp --- Cargo.toml | 11 +- build.rs | 8 + schema/protocol.capnp | 109 +++++++++++ src/main.rs | 99 +++++----- src/metric.rs | 165 +++++++++++++++-- src/parser.rs | 245 +++++++++++------------- src/peer.rs | 419 +++++++++++++++++++++++++++++------------- src/task.rs | 247 ++++++++++++++----------- src/util.rs | 110 +++++------ 9 files changed, 932 insertions(+), 481 deletions(-) create mode 100644 build.rs create mode 100644 schema/protocol.capnp diff --git a/Cargo.toml b/Cargo.toml index 84c4162..c069822 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "bioyino" -version = "0.3.3" +version = "0.4.0" authors = ["Sergey Noskov aka Albibek "] description = "StatsD-compatible, high-performance, fault-tolerant metric aggregator" @@ -14,10 +14,10 @@ futures="^0.1" tokio="^0.1" tokio-core="^0.1" tokio-io="^0.1" -bytes="^0.4" +bytes = { version = "^0.4", features = [ "serde" ] } resolve="^0.1" net2="^0.2" -combine="^2.5" +combine="^3.3" hyper="^0.11" serde="^1.0" serde_derive="^1.0" @@ -29,3 +29,8 @@ slog-async="^2.3" slog-scope="^4.0" toml="^0.4" ftoa = "^0.1" +capnp = "^0.8" +capnp-futures = "^0.1" + +[build-dependencies] +capnpc = "^0.8" diff --git a/build.rs b/build.rs new file mode 100644 index 0000000..e1b8436 --- /dev/null +++ b/build.rs @@ -0,0 +1,8 @@ +extern crate capnpc; + +fn main() { + capnpc::CompilerCommand::new() + .file("schema/protocol.capnp") + .run() + .expect("Failed compiling messages schema"); +} diff --git a/schema/protocol.capnp b/schema/protocol.capnp new file mode 100644 index 0000000..448d220 --- /dev/null +++ b/schema/protocol.capnp @@ -0,0 +1,109 @@ +@0xd87a49a1c493df22; + +# This schema defines a way to deliver metrics in both ways: +# as pre-aggregated shapshots and as new metrics +# Please note, that capnproto allows to skip sending any fields +# if they are separate types, so there is almost no need to integrate +# option-like type into schema type system. +# Bio will try to accept unspecified fields with some defaults, +# but may fail if it cannot get ones it needs + +# A message type for using in network interactions when metrics are involved +# the difference between snapshot and multi is that snapshot will be sent +# only to backend not to other nodes in the network +struct Message { + union { + single @0 :Metric; + multi @1 :List(Metric); + snapshot @2 :List(Metric); + } +} + +# A message type for internal messaging, should not be used by clients +# WARNING: This is reserved for future, only some commands may work +struct PeerCommand { + union { + # pause consensus leadership changes, see description below + pauseConsensus @0 :PauseConsensusCommand; + + # Resume consensus. Leader will be set to one from consensus + resumeConsensus @1 :Void; + + # server will answer with ServerStatus message + status @2 :Void; + } +} + +# Turn consensus off for time(in milliseconds). The consensus module will still work, but signals +# on leadership changes will be ignored +# Leader state will be unchanged if setLeader is 0 +# Leader will be disabled if setLeader is < 0 +# Leader will be enabled if setLeader is > 0 +struct PauseConsensusCommand { + pause @0 :UInt64; + setLeader @1 :Int8; +} + +struct ServerStatus { + leaderStatus @0 :Bool; + consensusPaused @1 :UInt64; +} + +struct Metric { + + # everyone should have a name, even metrics + name @0 :Text; + + # each metric has a value when it's sent + value @1 :Float64; + + # some types also imply additional internal values depending if metric type + type @2 :MetricType; + + # a timesamp can optionally be sent, i.e. for historic reasons + timestamp @3 :Timestamp; + + # additional useful data about metric + meta @4 :MetricMeta; +} + +struct Timestamp { + ts @0 :UInt64; +} + +struct MetricType { + union { + # counter value is stored inside it's value + counter @0 :Void; + + # for diff counter the metric value stores current counter value + # the internal value stores last received counter change fr differentiating + diffCounter @1 :Float64; + + # timer holds all values for further stats counting + timer @2 :List(Float64); + + # gauge can work as a counter too when `+value` or `-value` is received + gauge @3 :Gauge; + + # someday we will support this... conributions are welcomed if you need any of those + # histogram @4 :... + # set @5 :... + } +} + +struct Gauge { + union { + unsigned @0 :Void; + signed @1 :Int8; + } +} + +struct MetricMeta { + sampling @0 :Sampling; + updateCounter @1 :UInt32; +} + +struct Sampling { + sampling @0 :Float32; +} diff --git a/src/main.rs b/src/main.rs index 3defd2f..7eca2b8 100644 --- a/src/main.rs +++ b/src/main.rs @@ -17,6 +17,8 @@ extern crate toml; extern crate bytes; #[macro_use] extern crate futures; +extern crate capnp; +extern crate capnp_futures; extern crate hyper; extern crate libc; extern crate net2; @@ -47,20 +49,24 @@ pub mod server; pub mod task; pub mod util; +pub mod protocol_capnp { + include!(concat!(env!("OUT_DIR"), "/schema/protocol_capnp.rs")); +} + use std::cell::RefCell; use std::collections::HashMap; use std::io; use std::net::SocketAddr; use std::str::FromStr; -use std::sync::Arc; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering, ATOMIC_BOOL_INIT, ATOMIC_USIZE_INIT}; +use std::sync::Arc; use std::thread; use std::time::{self, Duration, Instant, SystemTime}; use slog::{Drain, Level}; use bytes::{Bytes, BytesMut}; -use futures::future::{empty, lazy, ok}; +use futures::future::{empty, ok}; use futures::sync::mpsc; use futures::{Future, IntoFuture, Stream}; @@ -69,8 +75,8 @@ use tokio::runtime::current_thread::Runtime; use tokio::timer::Interval; use tokio_core::reactor::Core; -use net2::UdpBuilder; use net2::unix::UnixUdpBuilderExt; +use net2::UdpBuilder; use resolve::resolver; use carbon::CarbonBackend; @@ -78,15 +84,15 @@ use config::{Command, Consul, Metrics, Network, System}; use consul::ConsulConsensus; use errors::GeneralError; use metric::Metric; -use peer::{PeerCommandClient, PeerServer, PeerSnapshotClient}; +use peer::{NativeProtocolServer, NativeProtocolSnapshot, PeerCommandClient}; use server::StatsdServer; use task::Task; use util::{AggregateOptions, Aggregator, BackoffRetryBuilder, OwnStats, UpdateCounterOptions}; pub type Float = f64; -pub type Cache = HashMap>; -thread_local!(static LONG_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); -thread_local!(static SHORT_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); +pub type Cache = HashMap>; +thread_local!(static LONG_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); +thread_local!(static SHORT_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); pub static PARSE_ERRORS: AtomicUsize = ATOMIC_USIZE_INIT; pub static AGG_ERRORS: AtomicUsize = ATOMIC_USIZE_INIT; @@ -133,28 +139,28 @@ fn main() { nodes, snapshot_interval, }, - consul: - Consul { - start_disabled: consul_disable, - agent, - session_ttl: consul_session_ttl, - renew_time: consul_renew_time, - key_name: consul_key, - }, - metrics: - Metrics { - // max_metrics, - mut count_updates, - update_counter_prefix, - update_counter_suffix, - update_counter_threshold, - }, - carbon, - n_threads, - w_threads, - stats_interval: s_interval, - task_queue_size, - stats_prefix, + consul: + Consul { + start_disabled: consul_disable, + agent, + session_ttl: consul_session_ttl, + renew_time: consul_renew_time, + key_name: consul_key, + }, + metrics: + Metrics { + // max_metrics, + mut count_updates, + update_counter_prefix, + update_counter_suffix, + update_counter_threshold, + }, + carbon, + n_threads, + w_threads, + stats_interval: s_interval, + task_queue_size, + stats_prefix, } = system.clone(); let verbosity = Level::from_str(&verbosity).expect("bad verbosity"); @@ -207,10 +213,10 @@ fn main() { .name(format!("bioyino_cnt{}", i).into()) .spawn(move || { let mut runtime = Runtime::new().expect("creating runtime for counting worker"); - let future = rx.for_each(move |task: Task| lazy(|| ok(task.run()))); + let future = rx.for_each(move |task: Task| ok(task.run())); runtime.block_on(future).expect("worker thread failed"); }) - .expect("starting counting worker thread"); + .expect("starting counting worker thread"); } let stats_prefix = stats_prefix.trim_right_matches(".").to_string(); @@ -225,19 +231,19 @@ fn main() { info!(log, "starting snapshot sender"); let snap_log = rlog.clone(); let snap_err_log = rlog.clone(); - let snapshot = PeerSnapshotClient::new( + let snapshot = NativeProtocolSnapshot::new( &snap_log, nodes.clone(), Duration::from_millis(snapshot_interval as u64), &chans, - ).into_future() + ).into_future() .map_err(move |e| { PEER_ERRORS.fetch_add(1, Ordering::Relaxed); info!(snap_err_log, "error sending snapshot";"error"=>format!("{}", e)); }); runtime.spawn(snapshot); - // settings afe for asap restart + // settings safe for asap restart info!(log, "starting snapshot receiver"); let peer_server_ret = BackoffRetryBuilder { delay: 1, @@ -246,7 +252,8 @@ fn main() { retries: ::std::usize::MAX, }; let serv_log = rlog.clone(); - let peer_server = PeerServer::new(rlog.clone(), peer_listen, chans.clone(), nodes.clone()); + + let peer_server = NativeProtocolServer::new(rlog.clone(), peer_listen, chans.clone()); let peer_server = peer_server_ret.spawn(peer_server).map_err(move |e| { warn!(serv_log, "shot server gone with error: {:?}", e); }); @@ -275,7 +282,7 @@ fn main() { core.run(consensus.into_future().map_err(|_| ())) .expect("running core for Consul consensus"); }) - .expect("starting thread for running consul"); + .expect("starting thread for running consul"); } else { info!(log, "consul is diabled, starting as leader"); IS_LEADER.store(true, Ordering::SeqCst); @@ -339,7 +346,7 @@ fn main() { .inspect(|_| { EGRESS.fetch_add(1, Ordering::Relaxed); }) - .collect() + .collect() .and_then(|metrics| { let backend = CarbonBackend::new(backend_addr, ts, Arc::new(metrics)); @@ -366,10 +373,10 @@ fn main() { .block_on(aggregator.then(|_| Ok::<(), ()>(()))) .unwrap_or_else( |e| error!(carbon_log, "Failed to join aggregated metrics"; "error"=>e), - ); + ); } }) - .expect("starting thread for sending to graphite"); + .expect("starting thread for sending to graphite"); Ok(()) }); @@ -425,7 +432,7 @@ fn main() { iov_base: buf.as_mut_ptr() as *mut c_void, iov_len: bufsize as size_t, }, - ); + ); let m = mmsghdr { msg_hdr: msghdr { msg_name: null_mut(), @@ -467,7 +474,7 @@ fn main() { vlen as c_uint, MSG_WAITFORONE, null_mut(), - ) + ) }; use bytes::BufMut; @@ -504,7 +511,7 @@ fn main() { warn!(log, "error sending buffer(queue full?)"); DROPS.fetch_add(res as usize, Ordering::Relaxed); }) - .unwrap_or(()); + .unwrap_or(()); chunks = task_queue_size as isize; } } else { @@ -520,7 +527,7 @@ fn main() { } } }) - .expect("starting multimsg thread"); + .expect("starting multimsg thread"); } } else { info!(log, "multimessage is disabled, starting in async UDP mode"); @@ -561,7 +568,7 @@ fn main() { let socket = socket.try_clone().expect("cloning socket"); let socket = UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()) - .expect("adding socket to event loop"); + .expect("adding socket to event loop"); let server = StatsdServer::new( socket, @@ -572,7 +579,7 @@ fn main() { i, readbuf, task_queue_size * bufsize, - ); + ); runtime.spawn(server.into_future()); } @@ -582,7 +589,7 @@ fn main() { .block_on(empty::<(), ()>()) .expect("starting runtime for async UDP"); }) - .expect("creating UDP reader thread"); + .expect("creating UDP reader thread"); } } diff --git a/src/metric.rs b/src/metric.rs index 65e5c97..cc39679 100644 --- a/src/metric.rs +++ b/src/metric.rs @@ -1,9 +1,15 @@ -//use errors::*; -use Float; -use failure::Error; use std::fmt::Debug; use std::ops::{Add, AddAssign, Div, Mul, Sub, SubAssign}; +use bytes::Bytes; +use capnp; +use capnp::message::{Allocator, Builder, HeapAllocator, Reader, ReaderSegments}; +//use capnp::message::ReaderSegments; +use failure::Error; + +use protocol_capnp::metric as cmetric; +use Float; + #[derive(Fail, Debug)] pub enum MetricError { #[fail(display = "float conversion")] @@ -14,6 +20,12 @@ pub enum MetricError { #[fail(display = "aggregating metrics of different types")] Aggregating, + + #[fail(display = "decoding error: {}", _0)] + Capnp(capnp::Error), + + #[fail(display = "schema error: {}", _0)] + CapnpSchema(capnp::NotInSchema), } // Percentile counter. Not safe. Requires at least two elements in vector @@ -52,18 +64,19 @@ where Counter, DiffCounter(F), Timer(Vec), - // Histogram, Gauge(Option), + // Histogram, // Set(HashSet), } -#[derive(Debug, Clone, Serialize, Deserialize)] +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct Metric where F: Copy + PartialEq + Debug, { pub value: F, pub mtype: MetricType, + pub timestamp: Option, pub update_counter: u32, pub sampling: Option, } @@ -79,21 +92,27 @@ where + Mul + PartialOrd + PartialEq - + Into + + Into + + From + Copy + Debug + Sync, { - pub fn new(value: F, mtype: MetricType, sampling: Option) -> Result { + pub fn new( + value: F, + mtype: MetricType, + timestamp: Option, + sampling: Option, + ) -> Result { let mut metric = Metric { - value: value, - mtype: mtype, - sampling: sampling, + value, + mtype, + timestamp, + sampling, update_counter: 1, }; if let MetricType::Timer(ref mut agg) = metric.mtype { - // ckms.insert(metric.value); agg.push(metric.value) }; // if let MetricType::Set(ref mut set) = metric.mtype { @@ -146,6 +165,130 @@ where }; Ok(()) } + + pub fn from_capnp<'a>( + reader: cmetric::Reader<'a>, + ) -> Result<(Bytes, Metric), MetricError> { + let name = reader.get_name().map_err(MetricError::Capnp)?.into(); + let value = reader.get_value(); + + let mtype = reader.get_type().map_err(MetricError::Capnp)?; + use protocol_capnp::metric_type; + let mtype = match mtype.which().map_err(MetricError::CapnpSchema)? { + metric_type::Which::Counter(()) => MetricType::Counter, + metric_type::Which::DiffCounter(c) => MetricType::DiffCounter(c), + metric_type::Which::Gauge(reader) => { + use protocol_capnp::gauge; + let reader = reader.map_err(MetricError::Capnp)?; + match reader.which().map_err(MetricError::CapnpSchema)? { + gauge::Which::Unsigned(()) => MetricType::Gauge(None), + gauge::Which::Signed(sign) => MetricType::Gauge(Some(sign)), + } + } + metric_type::Which::Timer(reader) => { + let reader = reader.map_err(MetricError::Capnp)?; + let mut v = Vec::new(); + v.reserve_exact(reader.len() as usize); + reader.iter().map(|ms| v.push(ms)).last(); + MetricType::Timer(v) + } + }; + + let timestamp = if reader.has_timestamp() { + Some(reader.get_timestamp().map_err(MetricError::Capnp)?.get_ts()) + } else { + None + }; + + let (sampling, up_counter) = match reader.get_meta() { + Ok(reader) => ( + if reader.has_sampling() { + reader + .get_sampling() + .ok() + .map(|reader| reader.get_sampling()) + } else { + None + }, + // .ok() + Some(reader.get_update_counter()), + ), + Err(_) => (None, None), + }; + + let mut metric = Metric::new(value.into(), mtype, timestamp, sampling)?; + + if let Some(c) = up_counter { + metric.update_counter = c; + } + + Ok((name, metric)) + } + + pub fn fill_capnp<'a>(&self, builder: &mut cmetric::Builder<'a>) { + // no name is known at this stage + // value + builder.set_value(self.value.into()); + // mtype + { + let mut t_builder = builder.reborrow().init_type(); + match self.mtype { + MetricType::Counter => t_builder.set_counter(()), + MetricType::DiffCounter(v) => t_builder.set_diff_counter(v.into()), + MetricType::Gauge(sign) => { + let mut g_builder = t_builder.init_gauge(); + match sign { + Some(v) => g_builder.set_signed(v), + None => g_builder.set_unsigned(()), + } + } + MetricType::Timer(ref v) => { + let mut timer_builder = t_builder.init_timer(v.len() as u32); + v.iter() + .enumerate() + .map(|(idx, value)| { + let value: f64 = (*value).into(); + timer_builder.set(idx as u32, value); + }) + .last(); + } + } + } + + // timestamp + { + if let Some(timestamp) = self.timestamp { + builder.reborrow().init_timestamp().set_ts(timestamp); + } + } + + // meta + let mut m_builder = builder.reborrow().init_meta(); + if let Some(sampling) = self.sampling { + m_builder.reborrow().init_sampling().set_sampling(sampling) + } + m_builder.set_update_counter(self.update_counter); + } + + // may be useful in future somehow + pub fn as_capnp(&self, allocator: A) -> Builder { + let mut builder = Builder::new(allocator); + { + let mut root = builder.init_root::(); + self.fill_capnp(&mut root); + } + builder + } + // may be useful in future somehow + pub fn as_capnp_heap(&self) -> Builder { + let allocator = HeapAllocator::new(); + let mut builder = Builder::new(allocator); + { + let mut root = builder.init_root::(); + self.fill_capnp(&mut root); + } + builder + } } impl IntoIterator for Metric diff --git a/src/parser.rs b/src/parser.rs index 50e73aa..f8b0d77 100644 --- a/src/parser.rs +++ b/src/parser.rs @@ -1,27 +1,20 @@ -use combine::byte::{byte, bytes, newline}; -use combine::combinator::{eof, skip_many}; -use combine::range::{take_while, take_while1}; -use combine::{optional, Parser}; -use metric::{Metric, MetricType}; use std::fmt::Debug; -use std::ops::{Add, AddAssign, Div, Mul, Sub, SubAssign}; -use std::str::FromStr; +use std::ops::{Add, AddAssign, Div, Mul, Neg, Sub, SubAssign}; use std::str::from_utf8; -//use quantiles::ckms::CKMS; -//use quantiles::greenwald_khanna::Stream as GK; - -use failure::ResultExt; +use std::str::FromStr; -#[derive(Fail, Debug)] -enum ParseError { - #[fail(display = "parsing UTF-8 data")] - Utf8(#[cause] ::std::str::Utf8Error), +use combine::byte::{byte, bytes, newline}; +use combine::combinator::{eof, skip_many}; +use combine::error::UnexpectedParse; +use combine::parser::range::{take_while, take_while1}; +use combine::{optional, Parser}; - #[fail(display = "parsing float")] - Float(String), -} +use metric::MetricType; -pub fn metric_parser<'a, F>() -> impl Parser), Input = &'a [u8]> +// to make his zero-copy and get better errors, parser only recognizes parts +// of the metric: (name, value, type, sampling) +pub fn metric_parser<'a, F>( +) -> impl Parser, Option), Input = &'a [u8]> where F: FromStr + Add @@ -30,8 +23,10 @@ where + SubAssign + Div + Mul + + Neg + PartialOrd + Into + + From + Debug + Default + Clone @@ -40,10 +35,7 @@ where + Sync, { // This will parse metric name and separator - let name = take_while1(|c: u8| c != b':' && c != b'\n') - .skip(byte(b':')) - .and_then(|name| from_utf8(name).map(|name| name.to_string())); - + let name = take_while1(|c: u8| c != b':' && c != b'\n').skip(byte(b':')); let sign = byte(b'+').map(|_| 1i8).or(byte(b'-').map(|_| -1i8)); // This should parse metric value and separator @@ -51,20 +43,12 @@ where .skip(byte(b'|')) .and_then(|value| { from_utf8(value) - .map_err(|e| ParseError::Utf8(e)) - .compat()? - .parse::() - .map_err(|_| { - ParseError::Float( - format!("parsing {:?} as float metric value", value).to_string(), - ) - }) - .compat() + .map_err(|_e| UnexpectedParse::Unexpected) + .map(|v| v.parse::().map_err(|_e| UnexpectedParse::Unexpected))? }); // This parses metric type let mtype = bytes(b"ms") - //.map(|_| MetricType::Timer(CKMS::::new(EPSILON))) .map(|_| MetricType::Timer(Vec::::new())) .or(byte(b'g').map(|_| MetricType::Gauge(None))) .or(byte(b'C').map(|_| MetricType::DiffCounter(F::default()))) @@ -73,113 +57,104 @@ where // we can add more types here // .or(byte(b'h').map(|_| MetricType::Histrogram)) ; - let sampling = (bytes(b"|@"), take_while(|c: u8| c != b'\n')).and_then(|(_, value)| { from_utf8(value) - .map_err(|e| ParseError::Utf8(e)) - .compat()? - .parse::() - .map_err(|_| { - ParseError::Float( - format!("parsing {:?} as float sampling value", value).to_string(), - ) - }) - .compat() + .map_err(|_e| UnexpectedParse::Unexpected) + .map(|v| v.parse::().map_err(|_e| UnexpectedParse::Unexpected))? }); - - let metric = name.and( - ( - optional(sign), - value, - mtype, - optional(sampling), - skip_many(newline()).or(eof()), - ).and_then(|(sign, value, mtype, sampling, _)| { - let mtype = if let MetricType::Gauge(_) = mtype { - MetricType::Gauge(sign) - } else { - mtype - }; - - Metric::::new(value, mtype, sampling).compat() - }), - ); - - metric + ( + name, + optional(sign), + value, + mtype, + optional(sampling), + skip_many(newline()).or(eof()), + ).and_then(|(name, sign, mut value, mtype, sampling, _)| { + let mtype = if let MetricType::Gauge(_) = mtype { + MetricType::Gauge(sign) + } else { + if sign == Some(-1) { + // get negative values back + value = value.neg() + } + mtype + }; + + Ok::<_, UnexpectedParse>((name, value, mtype, sampling)) + }) } -/* #[cfg(test)] mod tests { -// WARNING: these tests most probably don't work as of now -// FIXME: tests -use super::*; -use num::rational::Ratio; - -// TODO; parse bad and tricky metrics -// Questioned cases: -// * non-integer counters -// * negative counters - -#[test] -fn parse_good_counter() { -let data = b"gorets:1|c|@1"; -let v = parse_metrics(data).unwrap(); -assert_eq!(v[0].0, "gorets".to_string()); -assert_eq!(v.len(), 1); -assert_eq!(v[0].1.mtype, MetricType::Counter); -assert_eq!(v[0].1.value, 0); -} - -#[test] -fn parse_multi_metric_one() { -let data = b"complex.bioyino.test:1|g\n"; -let v = parse_metrics(data).unwrap(); -assert_eq!(v[0].0, "complex.bioyino.test".to_string()); -assert_eq!(v.len(), 1); -assert_eq!(v[0].1.mtype, MetricType::Gauge(None)); -assert_eq!(v[0].1.value, Ratio::new(1.into(), 1.into())); -} - -#[test] -fn parse_multi_metric_many() { -let data = b"complex.bioyino.test.1:1|g\ncomplex.bioyino.test.2:2|g"; -let v = parse_metrics(data).unwrap(); -assert_eq!(v.len(), 2); -assert_eq!(v[0].0, "complex.bioyino.test.1".to_string()); -assert_eq!(v[0].1.mtype, MetricType::Gauge(None)); -assert_eq!(v[0].1.value, Ratio::new(1.into(), 1.into())); -assert_eq!(v[1].0, "complex.bioyino.test.2".to_string()); -assert_eq!(v[1].1.mtype, MetricType::Gauge(None)); -assert_eq!(v[1].1.value, Ratio::new(2.into(), 1.into())); -} - -#[test] -fn parse_short_metric() { -let data = b"gorets:1|c"; -let d = parse_metrics(data); -let v = d.unwrap(); -assert_eq!(v[0].1.mtype, MetricType::Counter); -assert_eq!(v[0].1.value, Ratio::new(1.into(), 1.into())); -} - -#[test] -fn parse_gauge() { -let data = b"gorets:-75e-2|g|@1"; -let v = parse_metrics(data).unwrap(); -assert_eq!(v[0].1.mtype, MetricType::Gauge(Some(-1)), "bad type"); -// 0.75 should parse to 3/4 -assert_eq!(v[0].1.value, Ratio::new(3.into(), 4.into()), "bad value"); -} - -#[test] -fn parse_complex_gauges() { -let data = b"gorets:+1000|g\ngorets:-1000|g|@0.5"; -let v = parse_metrics(data).unwrap(); -assert_eq!(v[0].1.mtype, MetricType::Gauge(Some(1))); -assert_eq!(v[0].1.value, Ratio::new(1000.into(), 1.into())); -assert_eq!(v[1].1.mtype, MetricType::Gauge(Some(-1)), ""); -assert_eq!(v[1].1.value, Ratio::new(1000.into(), 1.into())); -} + // WARNING: these tests most probably don't work as of now + use super::*; + + // TODO: Questioned cases: + // * negative counters + // * diff counters + + #[test] + fn parse_metric_good_counter() { + let data = b"gorets:1|c|@1"; + let mut parser = metric_parser::(); + let (v, rest) = parser.parse(data).unwrap(); + assert_eq!(v.0, b"gorets"); + assert_eq!(v.1, 1f64); + assert_eq!(v.2, MetricType::Counter); + assert_eq!(v.3, Some(1f32)); + assert_eq!(rest.len(), 0); + } + + #[test] + fn parse_metric_good_counter_float() { + let data = b"gorets:12.65|c|@0.001"; + let mut parser = metric_parser::(); + let (v, rest) = parser.parse(data).unwrap(); + assert_eq!(v.0, b"gorets"); + assert_eq!(v.1, 12.65f64); + assert_eq!(v.2, MetricType::Counter); + assert_eq!(v.3, Some(1e-3f32)); + assert_eq!(rest.len(), 0); + } + + #[test] + fn parse_metric_with_newline() { + let data = b"complex.bioyino.test:-1e10|g\n"; + let mut parser = metric_parser::(); + let (v, rest) = parser.parse(data).unwrap(); + assert_eq!(v.0, b"complex.bioyino.test"); + assert_eq!(v.1, 1e10f64); + assert_eq!(v.2, MetricType::Gauge(Some(-1i8))); + assert_eq!(rest.len(), 0); + } + + #[test] + fn parse_metric_short() { + let data = b"gorets:1|c"; + let mut parser = metric_parser::(); + let (v, rest) = parser.parse(data).unwrap(); + assert_eq!(v.0, b"gorets"); + assert_eq!(v.1, 1f64); + assert_eq!(v.2, MetricType::Counter); + assert_eq!(v.3, None); + assert_eq!(rest.len(), 0) + } + + #[test] + fn parse_metric_many() { + let data = b"gorets:+1000|g\ngorets:-1000|g|@0.5"; + let mut parser = metric_parser::(); + let (v, rest) = parser.parse(data).unwrap(); + assert_eq!(v.0, b"gorets"); + assert_eq!(v.1, 1000f64); + assert_eq!(v.2, MetricType::Gauge(Some(1i8))); + assert_eq!(v.3, None); + //assert_neq!(rest.len(), 0) + let (v, rest) = parser.parse(rest).unwrap(); + assert_eq!(v.0, b"gorets"); + assert_eq!(v.1, 1000f64); + assert_eq!(v.2, MetricType::Gauge(Some(-1i8))); + assert_eq!(v.3, Some(0.5f32)); + assert_eq!(rest.len(), 0) + } } -*/ diff --git a/src/peer.rs b/src/peer.rs index 8554329..d752a7f 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -4,6 +4,9 @@ use std::sync::atomic::Ordering; use std::time::{Duration, Instant}; use bincode; +use capnp; +use capnp::message::{Builder, ReaderOptions}; +use capnp_futures::ReadStream; use futures::future::{join_all, ok, Future, IntoFuture}; use futures::sync::mpsc::Sender; use futures::sync::oneshot; @@ -17,8 +20,12 @@ use tokio_io::codec::length_delimited; use tokio_io::codec::length_delimited::Framed; use tokio_io::{AsyncRead, AsyncWrite}; +use metric::{Metric, MetricError}; +use protocol_capnp::message as cmsg; + +use protocol_capnp::peer_command; use task::Task; -use {Cache, CAN_LEADER, FORCE_LEADER, IS_LEADER, PEER_ERRORS}; +use {Cache, Float, CAN_LEADER, FORCE_LEADER, IS_LEADER, PEER_ERRORS}; #[derive(Fail, Debug)] pub enum PeerError { @@ -45,6 +52,15 @@ pub enum PeerError { #[fail(display = "response not sent")] Response, + + #[fail(display = "decoding capnp failed: {}", _0)] + Capnp(capnp::Error), + + #[fail(display = "decoding capnp schema failed: {}", _0)] + CapnpSchema(capnp::NotInSchema), + + #[fail(display = "decoding metric failed: {}", _0)] + Metric(MetricError), } #[derive(Debug, Serialize, Deserialize)] @@ -149,41 +165,29 @@ where } #[derive(Clone, Debug)] -pub struct PeerServer { +pub struct NativeProtocolServer { log: Logger, listen: SocketAddr, - nodes: Vec, chans: Vec>, } -impl PeerServer { - pub fn new( - log: Logger, - listen: SocketAddr, - chans: Vec>, - nodes: Vec, - ) -> Self { +impl NativeProtocolServer { + pub fn new(log: Logger, listen: SocketAddr, chans: Vec>) -> Self { Self { - log: log.new(o!("source"=>"peer-server", "ip"=>format!("{}", listen.clone()))), + log: log.new(o!("source"=>"canproto-peer-server", "ip"=>format!("{}", listen.clone()))), listen, - nodes: nodes, chans: chans, } } } -impl IntoFuture for PeerServer { +impl IntoFuture for NativeProtocolServer { type Item = (); type Error = PeerError; type Future = Box>; fn into_future(self) -> Self::Future { - let Self { - log, - listen, - nodes, - chans, - } = self; + let Self { log, listen, chans } = self; let future = TcpListener::bind(&listen) .expect("listening peer port") .incoming() @@ -192,104 +196,104 @@ impl IntoFuture for PeerServer { let peer_addr = conn.peer_addr() .map(|addr| addr.to_string()) .unwrap_or("[UNCONNECTED]".into()); - let transport = PeerCodec::new(conn); + let transport = ReadStream::new(conn, ReaderOptions::default()); let log = log.new(o!("remote"=>peer_addr)); - let nodes = nodes.clone(); let chans = chans.clone(); let mut chans = chans.into_iter().cycle(); - let (writer, reader) = transport.split(); - - let err_log = log.clone(); - reader - .map(move |m| { - match m { - PeerMessage::Snapshot(shot) => { - let next_chan = chans.next().unwrap(); - let future = next_chan - .send(Task::JoinSnapshot(shot)) - .map(|_| ()) // drop next sender - .map_err(|_| PeerError::TaskSend); - let elog = log.clone(); - spawn(future.map_err(move |e| { - warn!(elog, "error joining snapshot: {:?}", e); - })); - None - } - PeerMessage::Command(PeerCommand::LeaderEnable) => { - info!(log, "enabling leader"; "command"=>"leader_enable"); - CAN_LEADER.store(true, Ordering::SeqCst); - FORCE_LEADER.store(false, Ordering::SeqCst); - None - } - PeerMessage::Command(PeerCommand::LeaderDisable) => { - info!(log, "disabling leader"; "command"=>"leader_disable"); - CAN_LEADER.store(false, Ordering::SeqCst); - //IS_LEADER.store(false, Ordering::SeqCst); - None - } - PeerMessage::Command(PeerCommand::ForceLeader) => { - info!(log, "enforcing leader"; "command"=>"force_leader"); - CAN_LEADER.store(false, Ordering::SeqCst); - IS_LEADER.store(true, Ordering::SeqCst); - FORCE_LEADER.store(true, Ordering::SeqCst); - nodes - .clone() - .into_iter() - .map(|node| { - let elog = log.clone(); - let command = PeerCommandClient::new( - log.clone(), - node.clone(), - PeerCommand::LeaderDisable, - ).into_future() - .map_err(move |e| { - warn!( - elog, - "could not send command to {:?}: {:?}", node, e - ); - }) - .then(|_| Ok(())); - spawn(command) - }) - .last(); - None - } - PeerMessage::Command(PeerCommand::Status) => { - let is_leader = IS_LEADER.load(Ordering::Relaxed); - let can_leader = CAN_LEADER.load(Ordering::Relaxed); - let force_leader = FORCE_LEADER.load(Ordering::Relaxed); - let status = PeerStatus { - is_leader, - can_leader, - force_leader, - }; - Some(PeerMessage::Status(status)) - } - PeerMessage::Status(_) => { - // TODO: log bad error or response with BadMessage to client - None - } - } + transport + .then(move |reader| { + // decode incoming capnp data into message + // FIXME unwraps + let reader = reader.map_err(PeerError::Capnp)?; + let reader = reader.get_root::().map_err(PeerError::Capnp)?; + let next_chan = chans.next().unwrap(); + parse_and_send(reader, next_chan, log.clone()).map_err(|e| { + warn!(log, "bad incoming message"; "error" => e.to_string()); + PeerError::Metric(e) + }) + }) + .for_each(|_| { + // + Ok(()) }) - .forward(writer) - .map_err(move |e| info!(err_log, "peer command error"; "error"=>e.to_string())) - .then(|_| Ok(())) // don't let send errors fail the server }); Box::new(future) } } -pub struct PeerSnapshotClient { +fn parse_and_send( + reader: cmsg::Reader, + next_chan: Sender, + log: Logger, +) -> Result<(), MetricError> { + match reader.which().map_err(MetricError::CapnpSchema)? { + cmsg::Single(reader) => { + let reader = reader.map_err(MetricError::Capnp)?; + let (name, metric) = Metric::::from_capnp(reader)?; + let future = next_chan + .send(Task::AddMetric(name, metric)) + .map(|_| ()) // drop next sender + .map_err(|_| PeerError::TaskSend); + let elog = log.clone(); + spawn(future.map_err(move |e| { + warn!(elog, "error joining snapshot: {:?}", e); + })); + Ok(()) + } + cmsg::Multi(reader) => { + let reader = reader.map_err(MetricError::Capnp)?; + let mut metrics = Vec::new(); + reader + .iter() + .map(|reader| { + Metric::::from_capnp(reader) + .map(|(name, metric)| metrics.push((name, metric))) + }) + .last(); + let future = next_chan + .send(Task::AddMetrics(metrics)) + .map(|_| ()) // drop next sender + .map_err(|_| PeerError::TaskSend); + let elog = log.clone(); + spawn(future.map_err(move |e| { + warn!(elog, "error joining snapshot: {:?}", e); + })); + Ok(()) + } + cmsg::Snapshot(reader) => { + let reader = reader.map_err(MetricError::Capnp)?; + let mut metrics = Vec::new(); + reader + .iter() + .map(|reader| { + Metric::::from_capnp(reader) + .map(|(name, metric)| metrics.push((name, metric))) + }) + .last(); + let future = next_chan + .send(Task::AddSnapshot(metrics)) + .map(|_| ()) // drop next sender + .map_err(|_| PeerError::TaskSend); + let elog = log.clone(); + spawn(future.map_err(move |e| { + warn!(elog, "error joining snapshot: {:?}", e); + })); + Ok(()) + } + } +} + +pub struct NativeProtocolSnapshot { nodes: Vec, interval: Duration, chans: Vec>, log: Logger, } -impl PeerSnapshotClient { +impl NativeProtocolSnapshot { pub fn new( log: &Logger, nodes: Vec, @@ -305,7 +309,7 @@ impl PeerSnapshotClient { } } -impl IntoFuture for PeerSnapshotClient { +impl IntoFuture for NativeProtocolSnapshot { type Item = (); type Error = PeerError; type Future = Box>; @@ -346,22 +350,51 @@ impl IntoFuture for PeerSnapshotClient { // so we don't parallel connections and metrics fetching // TODO: we probably clne a lots of bytes here, // could've changed them to Arc - let log = log.clone(); + let dlog = log.clone(); + let elog = log.clone(); get_metrics.and_then(move |metrics| { let clients = nodes .into_iter() .map(|address| { let metrics = metrics.clone(); - let log = log.clone(); + let elog = elog.clone(); + let dlog = dlog.clone(); TcpStream::connect(&address) .map_err(|e| PeerError::Io(e)) .and_then(move |conn| { - let codec = PeerCodec::new(conn); - codec.send(Some(PeerMessage::Snapshot(metrics))).map(|_| ()) + let codec = ::capnp_futures::serialize::Transport::new( + conn, + ReaderOptions::default(), + ); + + let mut snapshot_message = Builder::new_default(); + { + let builder = snapshot_message + .init_root::<::protocol_capnp::message::Builder>(); + let mut multi_metric = + builder.init_snapshot(metrics.len() as u32); + metrics + .into_iter() + .flat_map(|hmap| hmap.into_iter()) + .enumerate() + .map(|(idx, (name, metric))| { + let mut c_metric = + multi_metric.reborrow().get(idx as u32); + let name = + unsafe { ::std::str::from_utf8_unchecked(&name) }; + c_metric.set_name(name); + metric.fill_capnp(&mut c_metric); + }) + .last(); + } + codec.send(snapshot_message).map(|_| ()).map_err(move |e| { + debug!(elog, "codec error"; "error"=>e.to_string()); + PeerError::Capnp(e) + }) }) .map_err(move |e| { PEER_ERRORS.fetch_add(1, Ordering::Relaxed); - debug!(log, "error sending snapshot: {}", e) + debug!(dlog, "error sending snapshot: {}", e) }) .then(|_| Ok(())) // we don't want to faill the whole timer cycle because of one send error }) @@ -411,28 +444,164 @@ impl IntoFuture for PeerCommandClient { let future = tokio::net::TcpStream::connect(&address) .map_err(|e| PeerError::Io(e)) .and_then(move |conn| { - let codec = PeerCodec::new(conn); - codec.send(Some(PeerMessage::Command(command))) + let transport = ReadStream::new(conn, ReaderOptions::default()); + transport + .map_err(|e| PeerError::Capnp(e)) + .for_each(move |reader| { + // decode incoming capnp data into message + let reader = reader + .get_root::() + .map_err(PeerError::Capnp)?; + match reader.which().map_err(PeerError::CapnpSchema)? { + peer_command::Which::Status(()) => (), + _ => info!(log, "command not implemented"), + } + Ok(()) + }) + }); + Box::new(future) + } +} + +#[cfg(test)] +mod test { + + use std::net::SocketAddr; + use std::thread; + use {slog, slog_async, slog_term}; + + use bytes::Bytes; + use capnp::message::Builder; + use futures::sync::mpsc::{self, Receiver}; + use metric::{Metric, MetricType}; + use slog::Drain; + use slog::Logger; + use std::time::{SystemTime, UNIX_EPOCH}; + use tokio::runtime::current_thread::Runtime; + use tokio::timer::Delay; + + use {LONG_CACHE, SHORT_CACHE}; + + use super::*; + fn prepare_log() -> Logger { + // Set logging + let decorator = slog_term::TermDecorator::new().build(); + let drain = slog_term::FullFormat::new(decorator).build().fuse(); + let filter = slog::LevelFilter::new(drain, slog::Level::Trace).fuse(); + let drain = slog_async::Async::new(filter).build().fuse(); + let rlog = slog::Logger::root(drain, o!("program"=>"test")); + return rlog; + } + + fn prepare_runtime_with_server( + test_timeout: Instant, + ) -> ( + Runtime, + Vec>, + Receiver, + Logger, + SocketAddr, + ) { + let rlog = prepare_log(); + let mut chans = Vec::new(); + let (tx, rx) = mpsc::channel(5); + chans.push(tx); + + let address: ::std::net::SocketAddr = "127.0.0.1:8136".parse().unwrap(); + let mut runtime = Runtime::new().expect("creating runtime for main thread"); + + let c_peer_listen = address.clone(); + let c_serv_log = rlog.clone(); + let peer_server = NativeProtocolServer::new(rlog.clone(), c_peer_listen, chans.clone()) + .into_future() + .map_err(move |e| { + warn!(c_serv_log, "shot server gone with error: {:?}", e); + }); + runtime.spawn(peer_server); + + (runtime, chans, rx, rlog, address) + } + + #[test] + fn test_peer_protocol_capnp() { + let test_timeout = Instant::now() + Duration::from_secs(3); + let (mut runtime, chans, rx, rlog, address) = prepare_runtime_with_server(test_timeout); + + let future = rx.for_each(move |task: Task| ok(task.run()).and_then(|_| Ok(()))); + runtime.spawn(future); + + let ts = SystemTime::now().duration_since(UNIX_EPOCH).unwrap(); + let ts = ts.as_secs() as u64; + + let outmetric = Metric::new(42f64, MetricType::Gauge(None), ts.into(), None).unwrap(); + let log = rlog.clone(); + + let metric = outmetric.clone(); + let sender = TcpStream::connect(&address) + .map_err(|e| { + println!("connection err: {:?}", e); }) - .and_then(move |codec| { - if resp_required { - let resp = codec - .into_future() - .and_then(move |(status, _)| { - if let Some(PeerMessage::Status(status)) = status { - println!("status of {:?}: {:?}", address, status,); - } else { - warn!(log, "Unknown response from server: {:?}", status); - } - Ok(()) - }) - .then(|_| Ok(())); - Box::new(resp) as Box> - } else { - Box::new(ok::<(), PeerError>(())) + .and_then(move |conn| { + let codec = + ::capnp_futures::serialize::Transport::new(conn, ReaderOptions::default()); + + let mut single_message = Builder::new_default(); + { + let builder = single_message.init_root::<::protocol_capnp::message::Builder>(); + let mut c_metric = builder.init_single(); + c_metric.set_name("complex.test.bioyino_single"); + metric.fill_capnp(&mut c_metric); } - }); - Box::new(future) + let mut multi_message = Builder::new_default(); + { + let builder = multi_message.init_root::<::protocol_capnp::message::Builder>(); + let multi_metric = builder.init_multi(1); + let mut new_metric = multi_metric.get(0); + new_metric.set_name("complex.test.bioyino_multi"); + metric.fill_capnp(&mut new_metric); + } + + let mut snapshot_message = Builder::new_default(); + { + let builder = + snapshot_message.init_root::<::protocol_capnp::message::Builder>(); + let multi_metric = builder.init_snapshot(1); + let mut new_metric = multi_metric.get(0); + new_metric.set_name("complex.test.bioyino_snapshot"); + metric.fill_capnp(&mut new_metric); + } + codec + .send(single_message) + .and_then(|codec| { + codec + .send(multi_message) + .and_then(|codec| codec.send(snapshot_message)) + }) + .map(|_| ()) + .map_err(|e| println!("codec error: {:?}", e)) + }) + .map_err(move |e| debug!(log, "error sending snapshot: {:?}", e)); + + let metric = outmetric.clone(); + let log = rlog.clone(); + + let d = Delay::new(Instant::now() + Duration::from_secs(1)); + let delayed = d.map_err(|_| ()).and_then(|_| sender); + runtime.spawn(delayed); + + let test_delay = Delay::new(test_timeout); + runtime.block_on(test_delay).expect("runtime"); + + let single_name: Bytes = "complex.test.bioyino_single".into(); + let multi_name: Bytes = "complex.test.bioyino_multi".into(); + let shot_name: Bytes = "complex.test.bioyino_snapshot".into(); + LONG_CACHE.with(|c| { + assert_eq!(c.borrow().get(&shot_name), Some(&outmetric)); + }); + SHORT_CACHE.with(|c| { + assert_eq!(c.borrow().get(&single_name), Some(&outmetric)); + assert_eq!(c.borrow().get(&multi_name), Some(&outmetric)); + }); } } diff --git a/src/task.rs b/src/task.rs index f7f779f..8944a7b 100644 --- a/src/task.rs +++ b/src/task.rs @@ -3,10 +3,10 @@ use std::sync::atomic::Ordering; use bytes::{BufMut, Bytes, BytesMut}; use combine::Parser; -use combine::primitives::FastResult; -use futures::Sink; +use futures::future::Either; use futures::sync::mpsc::UnboundedSender; use futures::sync::oneshot; +use futures::Sink; use metric::Metric; use parser::metric_parser; @@ -27,108 +27,48 @@ pub struct AggregateData { #[derive(Debug)] pub enum Task { Parse(Bytes), - AddMetrics(Cache), - JoinSnapshot(Vec), + AddMetric(Bytes, Metric), + AddMetrics(Vec<(Bytes, Metric)>), + AddSnapshot(Vec<(Bytes, Metric)>), TakeSnapshot(oneshot::Sender), Rotate(oneshot::Sender), Aggregate(AggregateData), } +fn update_metric(cache: &mut Cache, name: Bytes, metric: Metric) { + match cache.entry(name) { + Entry::Occupied(ref mut entry) => { + entry.get_mut().aggregate(metric).unwrap_or_else(|_| { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + }); + } + Entry::Vacant(entry) => { + entry.insert(metric); + } + }; +} + impl Task { pub fn run(self) { match self { - Task::Parse(buf) => { - let mut input: &[u8] = &buf; - let mut size_left = buf.len(); - let mut parser = metric_parser::(); - loop { - match parser.parse_stream_consumed(&mut input) { - FastResult::ConsumedOk(((name, metric), rest)) => { - INGRESS_METRICS.fetch_add(1, Ordering::Relaxed); - size_left -= rest.len(); - if size_left == 0 { - break; - } - input = rest; - SHORT_CACHE.with(|c| { - match c.borrow_mut().entry(name) { - Entry::Occupied(ref mut entry) => { - entry.get_mut().aggregate(metric).unwrap_or_else(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }); - } - Entry::Vacant(entry) => { - entry.insert(metric); - } - }; - }); - } - FastResult::EmptyOk(_) | FastResult::EmptyErr(_) => { - break; - } - FastResult::ConsumedErr(_e) => { - // println!( - //"error parsing {:?}: {:?}", - //String::from_utf8(input.to_vec()), - //_e - //); - PARSE_ERRORS.fetch_add(1, Ordering::Relaxed); - // try to skip bad metric taking all bytes before \n - match input.iter().position(|&c| c == 10u8) { - Some(pos) if pos < input.len() - 1 => { - input = input.split_at(pos + 1).1; - } - Some(_) => { - break; - } - None => { - break; - } - } - } - } - } - } - Task::AddMetrics(mut cache) => { - SHORT_CACHE.with(move |c| { - let mut short = c.borrow_mut(); - cache - .drain() - .map(|(name, metric)| { - match short.entry(name) { - Entry::Occupied(ref mut entry) => { - entry.get_mut().aggregate(metric).unwrap_or_else(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }); - } - Entry::Vacant(entry) => { - entry.insert(metric); - } - }; - }) - .last(); - }); - } - Task::JoinSnapshot(mut shot) => { - LONG_CACHE.with(move |c| { - let mut long = c.borrow_mut(); - shot.drain(..) - .flat_map(|hmap| hmap.into_iter()) - .map(|(name, metric)| { - match long.entry(name) { - Entry::Occupied(ref mut entry) => { - entry.get_mut().aggregate(metric).unwrap_or_else(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }); - } - Entry::Vacant(entry) => { - entry.insert(metric); - } - }; - }) - .last(); - }); - } + Task::Parse(buf) => parse_and_insert(buf), + Task::AddMetric(name, metric) => SHORT_CACHE.with(move |c| { + let mut short = c.borrow_mut(); + update_metric(&mut short, name, metric); + }), + Task::AddMetrics(mut list) => SHORT_CACHE.with(move |c| { + let mut short = c.borrow_mut(); + list.drain(..) + .map(|(name, metric)| update_metric(&mut short, name, metric)) + .last(); + }), + Task::AddSnapshot(mut list) => LONG_CACHE.with(move |c| { + // snapshots go to long cache to avoid being duplicated to other nodes + let mut long = c.borrow_mut(); + list.drain(..) + .map(|(name, metric)| update_metric(&mut long, name, metric)) + .last(); + }), Task::TakeSnapshot(channel) => { let mut short = SHORT_CACHE.with(|c| { let short = c.borrow().clone(); @@ -147,18 +87,7 @@ impl Task { let mut long = c.borrow_mut(); short .drain() - .map(|(name, metric)| { - match long.entry(name) { - Entry::Occupied(ref mut entry) => { - entry.get_mut().aggregate(metric).unwrap_or_else(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }); - } - Entry::Vacant(entry) => { - entry.insert(metric); - } - }; - }) + .map(|(name, metric)| update_metric(&mut long, name, metric)) .last(); }); } @@ -234,3 +163,105 @@ impl Task { } } } + +fn cut_bad(buf: &mut Bytes) -> Option { + PARSE_ERRORS.fetch_add(1, Ordering::Relaxed); + match buf.iter().position(|&c| c == 10u8) { + Some(pos) if pos <= buf.len() - 1 => { + buf.advance(pos + 1); + Some(pos) + } + Some(_) => None, + None => None, + } +} + +fn parse_and_insert(mut buf: Bytes) { + // Cloned buf is shallow copy, so input and buf are the same bytes. + // We are going to parse the whole slice, so for parser we use input as readonly + // while buf follows the parser progress and is cut to get only names + // so they are zero-copied + let mut input: &[u8] = &(buf.clone()); + let mut parser = metric_parser::(); + loop { + let buflen = buf.len(); + match parser.parse(&input) { + Ok(((name, value, mtype, sampling), rest)) => { + // name is always at the beginning of the buf + let name = buf.split_to(name.len()); + buf.advance(buflen - rest.len() - name.len()); + input = rest; + + // check if name is valid UTF-8 + if let Err(_) = ::std::str::from_utf8(&name) { + if let Some(pos) = cut_bad(&mut buf) { + input = input.split_at(pos + 1).1; + continue; + } else { + break; + } + } + + let metric = match Metric::::new(value, mtype, None, sampling) { + Ok(metric) => metric, + Err(_) => { + if let Some(pos) = cut_bad(&mut buf) { + input = input.split_at(pos + 1).1; + continue; + } else { + break; + } + } + }; + + INGRESS_METRICS.fetch_add(1, Ordering::Relaxed); + SHORT_CACHE.with(|c| { + let mut short = c.borrow_mut(); + update_metric(&mut short, name, metric); + }); + if rest.len() == 0 { + break; + } + } + Err(_e) => { + if let Some(pos) = cut_bad(&mut buf) { + input = input.split_at(pos + 1).1; + continue; + } else { + break; + } + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use metric::MetricType; + + #[test] + fn parse_trashed_metric_buf() { + let mut data = Bytes::new(); + data.extend_from_slice( + b"trash\ngorets1:+1000|g\nTRASH\ngorets2:-1000|g|@0.5\nMORETrasH\nFUUU", + ); + + parse_and_insert(data); + + SHORT_CACHE.with(|c| { + let c = c.borrow(); + let key: Bytes = "gorets1".into(); + let metric = c.get(&key).unwrap().clone(); + assert_eq!(metric.value, 1000f64); + assert_eq!(metric.mtype, MetricType::Gauge(Some(1i8))); + assert_eq!(metric.sampling, None); + + let key: Bytes = "gorets2".into(); + let metric = c.get(&key).unwrap().clone(); + assert_eq!(metric.value, 1000f64); + assert_eq!(metric.mtype, MetricType::Gauge(Some(-1i8))); + assert_eq!(metric.sampling, Some(0.5f32)); + }); + } +} diff --git a/src/util.rs b/src/util.rs index 08c5466..b7fb36c 100644 --- a/src/util.rs +++ b/src/util.rs @@ -2,7 +2,7 @@ use std::collections::HashMap; use std::sync::atomic::Ordering; use std::time::{Duration, Instant}; -use bytes::{Bytes, BytesMut}; +use bytes::{BufMut, Bytes, BytesMut}; use futures::future::Either; use futures::stream::futures_unordered; use futures::sync::mpsc::{Sender, UnboundedSender}; @@ -40,15 +40,25 @@ impl OwnStats { } pub fn get_stats(&mut self) { - let mut metrics = HashMap::new(); + let mut buf = BytesMut::with_capacity((self.prefix.len() + 10) * 7); // 10 is suffix len, 7 is number of metrics macro_rules! add_metric { ($global:ident, $value:ident, $suffix:expr) => { let $value = $global.swap(0, Ordering::Relaxed) as Float; if self.interval > 0 { - metrics.insert( - self.prefix.clone() + "." + $suffix, - Metric::new($value, MetricType::Counter, None).unwrap(), - ); + buf.put(&self.prefix); + buf.put("."); + buf.put(&$suffix); + let name = buf.take().freeze(); + let metric = Metric::new($value, MetricType::Counter, None, None).unwrap(); + let log = self.log.clone(); + let sender = self.chan + .clone() + .send(Task::AddMetric(name, metric)) + .map(|_| ()) + .map_err(move |_| { + warn!(log, "stats future could not send metric to task") + }); + spawn(sender); } }; }; @@ -61,6 +71,7 @@ impl OwnStats { add_metric!(DROPS, drops, "drop"); if self.interval > 0 { let s_interval = self.interval as f64; + info!(self.log, "stats"; "egress" => format!("{:2}", egress / s_interval), "ingress" => format!("{:2}", ingress / s_interval), @@ -71,14 +82,6 @@ impl OwnStats { "drops" => format!("{:2}", drops / s_interval), ); } - let log = self.log.clone(); - spawn( - self.chan - .clone() - .send(Task::AddMetrics(metrics)) - .map(|_| ()) - .map_err(move |_| warn!(log, "stats future could not send metric to task")), - ); } } @@ -128,7 +131,7 @@ impl Aggregator { options: AggregateOptions, chans: Vec>, tx: UnboundedSender<(Bytes, Float)>, - ) -> Self { + ) -> Self { Self { options, chans, tx } } } @@ -168,35 +171,35 @@ impl IntoFuture for Aggregator { // }) }); - let aggregate = accumulate.and_then(move |accumulated| { - accumulated - .into_iter() - .inspect(|_| { - EGRESS.fetch_add(1, Ordering::Relaxed); - }) - .map(move |(name, metric)| { - let buf = BytesMut::with_capacity(1024); - let task = Task::Aggregate(AggregateData { - buf, - name: Bytes::from(name), - metric, - options: options.clone(), - response: tx.clone(), - }); - // as of now we just run each task in the current thread - // there is a reason we should not in general run the task in the counting workers: - // workers will block on heavy computation and may cause metrics goind to them over - // network to be dropped because of backpressure - // at the same time counting aggregation is not urgent because of current backend(carbon/graphite) - // nature where one can send metrics with any timestamp - // TODO: at some day counting workers will probably work in work-stealing mode, - // after that we probably will be able to run task in common mode - task.run(); - }) - .last(); - Ok(()) - }); - Box::new(aggregate) + let aggregate = accumulate.and_then(move |accumulated| { + accumulated + .into_iter() + .inspect(|_| { + EGRESS.fetch_add(1, Ordering::Relaxed); + }) + .map(move |(name, metric)| { + let buf = BytesMut::with_capacity(1024); + let task = Task::Aggregate(AggregateData { + buf, + name: Bytes::from(name), + metric, + options: options.clone(), + response: tx.clone(), + }); + // as of now we just run each task in the current thread + // there is a reason we should not in general run the task in the counting workers: + // workers will block on heavy computation and may cause metrics goind to them over + // network to be dropped because of backpressure + // at the same time counting aggregation is not urgent because of current backend(carbon/graphite) + // nature where one can send metrics with any timestamp + // TODO: at some day counting workers will probably work in work-stealing mode, + // after that we probably will be able to run task in common mode + task.run(); + }) + .last(); + Ok(()) + }); + Box::new(aggregate) } else { // only get metrics from threads let not_leader = futures_unordered(metrics).for_each(|_| Ok(())); @@ -205,6 +208,7 @@ impl IntoFuture for Aggregator { } } +#[derive(Clone, Debug)] pub struct BackoffRetryBuilder { pub delay: u64, pub delay_mul: f32, @@ -225,16 +229,16 @@ impl Default for BackoffRetryBuilder { impl BackoffRetryBuilder { pub fn spawn(self, action: F) -> BackoffRetry - where + where F: IntoFuture + Clone, - { - let inner = Either::A(action.clone().into_future()); - BackoffRetry { - action, - inner: inner, - options: self, - } + { + let inner = Either::A(action.clone().into_future()); + BackoffRetry { + action, + inner: inner, + options: self, } + } } /// TCP client that is able to reconnect with customizable settings @@ -246,7 +250,7 @@ pub struct BackoffRetry { impl Future for BackoffRetry where -F: IntoFuture + Clone, + F: IntoFuture + Clone, { type Item = F::Item; type Error = Option; From 8ee4e1ecb587ac22dd47a85cf5b2f5291e749188 Mon Sep 17 00:00:00 2001 From: Albibek Date: Mon, 11 Jun 2018 13:28:08 +0300 Subject: [PATCH 02/24] Remake consul and some options --- Cargo.toml | 5 +- src/carbon.rs | 4 +- src/config.rs | 9 +- src/consul.rs | 247 ++++++++++++++++++++++++++------------------------ src/errors.rs | 3 + src/main.rs | 83 ++++++++++------- src/peer.rs | 7 +- 7 files changed, 197 insertions(+), 161 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index c069822..67d9150 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -5,6 +5,7 @@ authors = ["Sergey Noskov aka Albibek "] description = "StatsD-compatible, high-performance, fault-tolerant metric aggregator" [dependencies] +lazy_static="^1.0" clap="^2.31" failure="^0.1" failure_derive="^0.1" @@ -12,13 +13,13 @@ libc="^0.2" num_cpus="^1.8" futures="^0.1" tokio="^0.1" -tokio-core="^0.1" tokio-io="^0.1" bytes = { version = "^0.4", features = [ "serde" ] } resolve="^0.1" net2="^0.2" combine="^3.3" -hyper="^0.11" +hyper="^0.12" +mime="^0.3" serde="^1.0" serde_derive="^1.0" serde_json="^1.0" diff --git a/src/carbon.rs b/src/carbon.rs index ca88272..a07d603 100644 --- a/src/carbon.rs +++ b/src/carbon.rs @@ -1,6 +1,6 @@ use std::net::SocketAddr; -use std::sync::Arc; use std::sync::atomic::Ordering; +use std::sync::Arc; use std::time::Duration; use bytes::{BufMut, Bytes, BytesMut}; @@ -9,8 +9,8 @@ use ftoa; use futures::stream; use futures::{Future, IntoFuture, Sink, Stream}; use tokio::net::TcpStream; -use tokio_io::AsyncRead; use tokio_io::codec::{Decoder, Encoder}; +use tokio_io::AsyncRead; use errors::GeneralError; diff --git a/src/config.rs b/src/config.rs index 827a82a..ae2479d 100644 --- a/src/config.rs +++ b/src/config.rs @@ -4,6 +4,7 @@ use std::fs::File; use std::io::Read; use std::net::SocketAddr; use toml; +use ConsensusState; #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] @@ -32,6 +33,9 @@ pub(crate) struct System { /// queue size for single counting thread before packet is dropped pub task_queue_size: usize, + /// Should we start as leader state enabled or not + pub start_as_leader: bool, + /// How often to gather own stats, in ms. Use 0 to disable (stats are still gathered, but not included in /// metric dump) pub stats_interval: u64, @@ -52,6 +56,7 @@ impl Default for System { w_threads: 4, stats_interval: 10000, task_queue_size: 2048, + start_as_leader: false, stats_prefix: "resources.monitoring.bioyino".to_string(), } } @@ -179,7 +184,7 @@ impl Default for Network { #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] pub(crate) struct Consul { /// Start in disabled leader finding mode - pub start_disabled: bool, + pub start_as: ConsensusState, /// Consul agent address pub agent: SocketAddr, @@ -197,7 +202,7 @@ pub(crate) struct Consul { impl Default for Consul { fn default() -> Self { Self { - start_disabled: false, + start_as: ConsensusState::Disabled, agent: "127.0.0.1:8500".parse().unwrap(), session_ttl: 11000, renew_time: 1000, diff --git a/src/consul.rs b/src/consul.rs index b4dc988..231935e 100644 --- a/src/consul.rs +++ b/src/consul.rs @@ -1,15 +1,20 @@ use std::net::SocketAddr; use std::sync::atomic::Ordering; -use std::time::Duration; +use std::time::{Duration, Instant}; +use futures::future::{err, loop_fn, ok, Either, Future, IntoFuture, Loop}; use futures::Stream; -use futures::future::{err, loop_fn, ok, Future, IntoFuture, Loop}; use hyper; -use hyper::header::{ContentLength, ContentType}; +use hyper::header::{HeaderValue, CONTENT_LENGTH, CONTENT_TYPE}; +use hyper::{Method, StatusCode}; +use mime::WWW_FORM_URLENCODED; use serde_json::{self, from_slice}; use slog::Logger; -use tokio_core::reactor::{Handle, Interval, Timeout}; -use {CAN_LEADER, FORCE_LEADER, IS_LEADER}; +use tokio::executor::current_thread::spawn; +use tokio::timer::{self, Delay, Interval}; + +use util::BackoffRetryBuilder; +use {ConsensusState, CONSENSUS_STATE, IS_LEADER}; #[derive(Fail, Debug)] pub enum ConsulError { @@ -32,6 +37,9 @@ pub enum ConsulError { #[fail(display = "{}", _0)] Renew(String), + + #[fail(display = "creating timer: {}", _0)] + Timer(timer::Error), } #[derive(Deserialize)] @@ -43,7 +51,6 @@ struct ConsulSessionResponse { pub struct ConsulConsensus { log: Logger, agent: SocketAddr, - handle: Handle, key: String, session_ttl: Duration, renew_time: Duration, @@ -51,11 +58,10 @@ pub struct ConsulConsensus { } impl ConsulConsensus { - pub fn new(log: &Logger, agent: SocketAddr, key: String, handle: &Handle) -> Self { + pub fn new(log: &Logger, agent: SocketAddr, key: String) -> Self { Self { log: log.new(o!("source"=>"consensus")), agent, - handle: handle.clone(), key: key, session_ttl: Duration::from_secs(7), renew_time: Duration::from_secs(1), @@ -89,69 +95,85 @@ impl IntoFuture for ConsulConsensus { let Self { log, agent, - handle, key, session_ttl, renew_time, error_pause, } = self; - let thandle = handle.clone(); let renew_loop = loop_fn((), move |()| { let key = key.clone(); - let handle = handle.clone(); let log = log.clone(); let session = ConsulSession { log: log.new(o!("source"=>"consul-session")), agent: agent.clone(), - handle: handle.clone(), ttl: session_ttl.clone(), }; let renewlog = log.clone(); + /* + let session_retrier = BackoffRetryBuilder { + delay: 1000, + delay_mul: 1f32, + delay_max: 1000, + retries: ::std::usize::MAX, + }; - let thandle = thandle.clone(); + let session = session_retrier.spawn(session); + */ // this tries to reconnect to consul infinitely let loop_session = loop_fn(session, move |session| { let log = log.clone(); let new_session = session.clone(); - - let thandle = thandle.clone(); - session.into_future().then(move |res| match res { - Err(e) => { - warn!(log, "error getting consul session"; "error" => format!("{}", e)); - let new_session = new_session.clone(); - Box::new( - Timeout::new(error_pause, &thandle) - .unwrap() - .then(move |_| Ok(Loop::Continue(new_session))), - ) as Box, Error = _>> - //ok(Loop::Continue(new_session)) - } - Ok(None) => { - warn!(log, "timed out getting consul session"); - Box::new(ok(Loop::Continue(new_session))) - } - Ok(Some(s)) => Box::new(ok(Loop::Break(s))), - }) + let should_connect = { + let state = &*CONSENSUS_STATE.lock().unwrap(); + // connect to consul only in Enabled/Paused state + state != &ConsensusState::Disabled + }; + + if should_connect { + Either::A(session.into_future().then(move |res| match res { + Err(e) => { + warn!(log, "error getting consul session"; "error" => format!("{}", e)); + let new_session = new_session.clone(); + Box::new( + Delay::new(Instant::now() + error_pause) + .then(move |_| Ok(Loop::Continue(new_session))), + ) + as Box, Error = _>> + //ok(Loop::Continue(new_session)) + } + Ok(None) => { + warn!(log, "timed out getting consul session"); + Box::new(ok(Loop::Continue(new_session))) + } + Ok(Some(s)) => Box::new(ok(Loop::Break(s))), + })) + } else { + Either::B( + Delay::new(Instant::now() + error_pause) + .then(move |_| Ok(Loop::Continue(new_session))), + ) + } }); - let rhandle = handle.clone(); // the returned future will work until renew error let renew = // make connection, then start to renew - loop_session.and_then(move |sid| { - let handle = rhandle.clone(); - let timer = Interval::new(renew_time, &handle).unwrap(); + loop_session + .and_then(move |sid| { + let timer = Interval::new(Instant::now()+renew_time, renew_time); - timer.map_err(|e| ConsulError::Io(e)).for_each(move |_| { + timer.map_err(|e| ConsulError::Timer(e)).for_each(move |_| { let log = renewlog.clone(); - let can_leader = CAN_LEADER.load(Ordering::SeqCst); - // do work only if taking leadership is enabled - if can_leader { + let should_renew = { + let state = &*CONSENSUS_STATE.lock().unwrap(); + // renew the key only in Enabled/Paused state + state != &ConsensusState::Disabled + }; + if should_renew { let renew = ConsulRenew { agent: agent, - handle: handle.clone(), sid: sid.clone(), ttl: session_ttl, }.into_future() @@ -164,7 +186,6 @@ impl IntoFuture for ConsulConsensus { let acquire = ConsulAcquire { log: log.new(o!("source"=>"consul-acquire")), agent, - handle: handle.clone(), sid: sid.clone(), key: key.clone(), }.into_future() @@ -173,10 +194,9 @@ impl IntoFuture for ConsulConsensus { e }); - Box::new(renew.join(acquire).map(|_|())) + Either::A(renew.join(acquire).map(|_|())) } else { - IS_LEADER.store(FORCE_LEADER.load(Ordering::SeqCst), Ordering::SeqCst); - Box::new(ok(())) as Box> + Either::B(ok(()))// as Box>) } }) }); @@ -184,9 +204,7 @@ impl IntoFuture for ConsulConsensus { // restart the whole loop as soon as ANY future exits with any result // (is is supposed to exit only with error) renew.then(move |_| { - Timeout::new(error_pause, &handle) - .unwrap() - .then(move |_| Ok(Loop::Continue(()))) + Delay::new(Instant::now() + error_pause).then(move |_| Ok(Loop::Continue(()))) }) }); Box::new(renew_loop) @@ -197,7 +215,6 @@ impl IntoFuture for ConsulConsensus { pub struct ConsulSession { log: Logger, agent: SocketAddr, - handle: Handle, ttl: Duration, } @@ -207,42 +224,39 @@ impl IntoFuture for ConsulSession { type Future = Box>; fn into_future(self) -> Self::Future { - let Self { - log, - agent, - handle, - ttl, - } = self; + let Self { log, agent, ttl } = self; // create HTTP client for consul agent leader - let client = hyper::Client::new(&handle); - let mut session_req = hyper::Request::new( - hyper::Method::Put, - format!("http://{}/v1/session/create", agent) - .parse() - .expect("bad session create url"), - ); - - let ttl_ns = ttl.as_secs() * 1000000000u64 + ttl.subsec_nanos() as u64; + let client = hyper::Client::::new(); + let mut session_req = hyper::Request::default(); + *session_req.method_mut() = Method::PUT; + *session_req.uri_mut() = format!("http://{}/v1/session/create", agent) + .parse() + .expect("bad session create url"); + + let ttl_ns = ttl.as_secs() * 1_000_000_000u64 + ttl.subsec_nanos() as u64; let b = format!( "{{\"TTL\": \"{}ns\", \"LockDelay\": \"{}ns\"}}", ttl_ns, ttl_ns ); let bodylen = b.len() as u64; - session_req.set_body(b); + *session_req.body_mut() = hyper::Body::from(b); // Override sending request as multipart - session_req.headers_mut().set(ContentLength(bodylen)); - session_req - .headers_mut() - .set(ContentType::form_url_encoded()); + session_req.headers_mut().insert( + CONTENT_LENGTH, + HeaderValue::from_str(&format!("{}", bodylen)).unwrap(), + ); + session_req.headers_mut().insert( + CONTENT_TYPE, + HeaderValue::from_str(WWW_FORM_URLENCODED.as_str()).unwrap(), + ); - let thandle = handle.clone(); let c_session = client .request(session_req) .map_err(|e| ConsulError::Http(e)) .and_then(move |resp| { let status = resp.status(); - if status == hyper::StatusCode::Ok { - let body = resp.body() + if status == StatusCode::OK { + let body = resp.into_body() .concat2() .map_err(|e| ConsulError::Http(e)) .and_then(move |body| { @@ -253,11 +267,10 @@ impl IntoFuture for ConsulSession { }); Box::new(body) as Box, Error = ConsulError>> } else { - let body = resp.body().concat2().map_err(|e| ConsulError::Http(e)); + let body = resp.into_body().concat2().map_err(|e| ConsulError::Http(e)); // TODO make this into option - let sleep = Timeout::new(Duration::from_millis(1000), &thandle) - .unwrap() - .map_err(|e| ConsulError::Io(e)); + let sleep = Delay::new(Instant::now() + Duration::from_millis(1000)) + .map_err(|e| ConsulError::Timer(e)); let future = sleep.join(body).then(move |res| match res { Ok((_, body)) => Err::, _>(ConsulError::HttpStatus( status, @@ -268,9 +281,9 @@ impl IntoFuture for ConsulSession { Box::new(future) } }); - let timeout = Timeout::new(ttl, &handle).unwrap(); + let timeout = Delay::new(Instant::now() + ttl); let future = timeout - .map_err(|e| ConsulError::Io(e)) + .map_err(|e| ConsulError::Timer(e)) .map(|_| None) .select(c_session) .map(|res| res.0) @@ -281,7 +294,6 @@ impl IntoFuture for ConsulSession { pub struct ConsulRenew { agent: SocketAddr, - handle: Handle, sid: String, ttl: Duration, } @@ -292,34 +304,35 @@ impl IntoFuture for ConsulRenew { type Future = Box>; fn into_future(self) -> Self::Future { - let Self { - agent, - handle, - sid, - ttl, - } = self; - let mut renew_req = hyper::Request::new( - hyper::Method::Put, - format!("http://{}/v1/session/renew/{}", agent, sid.clone()) - .parse() - .expect("creating session renew url"), - ); + let Self { agent, sid, ttl } = self; + let mut renew_req = hyper::Request::default(); + *renew_req.method_mut() = Method::PUT; + *renew_req.uri_mut() = format!("http://{}/v1/session/renew/{}", agent, sid.clone()) + .parse() + .expect("creating session renew url"); let ttl_ns = ttl.as_secs() * 1000000000u64 + ttl.subsec_nanos() as u64; let b = format!("{{\"TTL\": \"{}ns\"}}", ttl_ns); let bodylen = b.len() as u64; - renew_req.set_body(b); - renew_req.headers_mut().set(ContentLength(bodylen)); - renew_req.headers_mut().set(ContentType::form_url_encoded()); + *renew_req.body_mut() = hyper::Body::from(b); + // Override sending request as multipart + renew_req.headers_mut().insert( + CONTENT_LENGTH, + HeaderValue::from_str(&format!("{}", bodylen)).unwrap(), + ); + renew_req.headers_mut().insert( + CONTENT_TYPE, + HeaderValue::from_str(WWW_FORM_URLENCODED.as_str()).unwrap(), + ); - let renew_client = hyper::Client::new(&handle); + let renew_client = hyper::Client::new(); let future = renew_client.request(renew_req).then(move |res| match res { Err(e) => Box::new(err(ConsulError::Http(e))), Ok(resp) => { - if resp.status() != hyper::StatusCode::Ok { + if resp.status() != StatusCode::OK { let status = resp.status().clone(); - let body = resp.body() + let body = resp.into_body() .concat2() .map_err(|e| ConsulError::Http(e)) .and_then(move |body| { @@ -343,7 +356,6 @@ impl IntoFuture for ConsulRenew { pub struct ConsulAcquire { log: Logger, agent: SocketAddr, - handle: Handle, sid: String, key: String, } @@ -357,38 +369,41 @@ impl IntoFuture for ConsulAcquire { let Self { log, agent, - handle, sid, key, } = self; - let req = hyper::Request::new( - hyper::Method::Put, - format!("http://{}/v1/kv/{}/?acquire={}", agent, key, sid) - .parse() - .expect("bad key acquire url"), - ); + let mut req = hyper::Request::default(); + *req.method_mut() = Method::PUT; + *req.uri_mut() = format!("http://{}/v1/kv/{}/?acquire={}", agent, key, sid) + .parse() + .expect("bad key acquire url"); + //.body(hyper::Body::empty()) + //.expect("building acquire request"); - let client = hyper::Client::new(&handle); + let client = hyper::Client::new(); let acquire = client .request(req) .map_err(|e| ConsulError::Http(e)) .and_then(move |resp| { - resp.body() + resp.into_body() .concat2() .map_err(|e| ConsulError::Http(e)) .and_then(move |body| { - let mut acquired: bool = + let acquired: bool = try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); - let force_leader = FORCE_LEADER.load(Ordering::SeqCst); - let is_leader = IS_LEADER.load(Ordering::SeqCst); - if force_leader { - acquired = true - } - IS_LEADER.store(acquired, Ordering::SeqCst); - if is_leader != acquired { - warn!(log, "leader state change: {} -> {}", is_leader, acquired); + let should_set = { + let state = &*CONSENSUS_STATE.lock().unwrap(); + // only set leader when consensus is enabled + state == &ConsensusState::Enabled + }; + if should_set { + let is_leader = IS_LEADER.load(Ordering::SeqCst); + if is_leader != acquired { + warn!(log, "leader state change: {} -> {}", is_leader, acquired); + } + IS_LEADER.store(acquired, Ordering::SeqCst); } Ok(()) }) diff --git a/src/errors.rs b/src/errors.rs index cd4b10f..b5b5628 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -19,4 +19,7 @@ pub enum GeneralError { #[fail(display = "future send error")] FutureSend, + + #[fail(display = "unknown consensus state")] + UnknownState, } diff --git a/src/main.rs b/src/main.rs index 7eca2b8..969ab3c 100644 --- a/src/main.rs +++ b/src/main.rs @@ -1,4 +1,6 @@ // General +#[macro_use] +extern crate lazy_static; extern crate failure; #[macro_use] extern crate failure_derive; @@ -21,11 +23,11 @@ extern crate capnp; extern crate capnp_futures; extern crate hyper; extern crate libc; +extern crate mime; extern crate net2; extern crate num_cpus; extern crate resolve; extern crate tokio; -extern crate tokio_core; extern crate tokio_io; // Other @@ -59,7 +61,7 @@ use std::io; use std::net::SocketAddr; use std::str::FromStr; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering, ATOMIC_BOOL_INIT, ATOMIC_USIZE_INIT}; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use std::thread; use std::time::{self, Duration, Instant, SystemTime}; @@ -73,7 +75,6 @@ use futures::{Future, IntoFuture, Stream}; use tokio::net::UdpSocket; use tokio::runtime::current_thread::Runtime; use tokio::timer::Interval; -use tokio_core::reactor::Core; use net2::unix::UnixUdpBuilderExt; use net2::UdpBuilder; @@ -89,11 +90,17 @@ use server::StatsdServer; use task::Task; use util::{AggregateOptions, Aggregator, BackoffRetryBuilder, OwnStats, UpdateCounterOptions}; +// floating type used all over the code, can be changed to f32, to use less memory at the price of +// precision +// TODO: make in into compilation feature pub type Float = f64; + +// a type to store pre-aggregated data pub type Cache = HashMap>; thread_local!(static LONG_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); thread_local!(static SHORT_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); +// statistic counters pub static PARSE_ERRORS: AtomicUsize = ATOMIC_USIZE_INIT; pub static AGG_ERRORS: AtomicUsize = ATOMIC_USIZE_INIT; pub static PEER_ERRORS: AtomicUsize = ATOMIC_USIZE_INIT; @@ -102,9 +109,31 @@ pub static INGRESS_METRICS: AtomicUsize = ATOMIC_USIZE_INIT; pub static EGRESS: AtomicUsize = ATOMIC_USIZE_INIT; pub static DROPS: AtomicUsize = ATOMIC_USIZE_INIT; -pub static CAN_LEADER: AtomicBool = ATOMIC_BOOL_INIT; +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +pub enum ConsensusState { + Enabled, + Paused, + Disabled, +} + +impl FromStr for ConsensusState { + type Err = GeneralError; + + fn from_str(s: &str) -> Result { + match s { + "enabled" | "enable" => Ok(ConsensusState::Enabled), + "disabled" | "disable" => Ok(ConsensusState::Disabled), + "pause" | "paused" => Ok(ConsensusState::Paused), + _ => Err(GeneralError::UnknownState), + } + } +} + +lazy_static! { + pub static ref CONSENSUS_STATE: Mutex = { Mutex::new(ConsensusState::Disabled) }; +} + pub static IS_LEADER: AtomicBool = ATOMIC_BOOL_INIT; -pub static FORCE_LEADER: AtomicBool = ATOMIC_BOOL_INIT; pub fn try_resolve(s: &str) -> SocketAddr { s.parse().unwrap_or_else(|_| { @@ -141,7 +170,7 @@ fn main() { }, consul: Consul { - start_disabled: consul_disable, + start_as: consul_start_as, agent, session_ttl: consul_session_ttl, renew_time: consul_renew_time, @@ -160,6 +189,7 @@ fn main() { w_threads, stats_interval: s_interval, task_queue_size, + start_as_leader, stats_prefix, } = system.clone(); @@ -260,35 +290,22 @@ fn main() { runtime.spawn(peer_server); - // TODO (maybe) change to option, not-depending on number of nodes - if nodes.len() > 0 { - info!(log, "consul is enabled, starting consul consensus"); - if consul_disable { - CAN_LEADER.store(false, Ordering::SeqCst); - IS_LEADER.store(false, Ordering::SeqCst); - } else { - CAN_LEADER.store(true, Ordering::SeqCst); - } - let consul_log = rlog.clone(); - thread::Builder::new() - .name("bioyino_consul".into()) - .spawn(move || { - let mut core = Core::new().unwrap(); - let handle = core.handle(); - - let mut consensus = ConsulConsensus::new(&consul_log, agent, consul_key, &handle); - consensus.set_session_ttl(Duration::from_millis(consul_session_ttl as u64)); - consensus.set_renew_time(Duration::from_millis(consul_renew_time as u64)); - core.run(consensus.into_future().map_err(|_| ())) - .expect("running core for Consul consensus"); - }) - .expect("starting thread for running consul"); - } else { - info!(log, "consul is diabled, starting as leader"); - IS_LEADER.store(true, Ordering::SeqCst); - CAN_LEADER.store(false, Ordering::SeqCst); + // Init leader state before starting backend + IS_LEADER.store(start_as_leader, Ordering::SeqCst); + + { + let mut con_state = CONSENSUS_STATE.lock().unwrap(); + info!(log, "starting consul consensus"; "initial_state"=>format!("{:?}", con_state)); + *con_state = consul_start_as; } + let consul_log = rlog.clone(); + + let mut consensus = ConsulConsensus::new(&consul_log, agent, consul_key); + consensus.set_session_ttl(Duration::from_millis(consul_session_ttl as u64)); + consensus.set_renew_time(Duration::from_millis(consul_renew_time as u64)); + runtime.spawn(consensus.into_future().map_err(|_| ())); // TODO errors + info!(log, "starting carbon backend"); let tchans = chans.clone(); let carbon_log = rlog.clone(); diff --git a/src/peer.rs b/src/peer.rs index d752a7f..b07eee4 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -25,7 +25,7 @@ use protocol_capnp::message as cmsg; use protocol_capnp::peer_command; use task::Task; -use {Cache, Float, CAN_LEADER, FORCE_LEADER, IS_LEADER, PEER_ERRORS}; +use {Cache, ConsensusState, Float, CONSENSUS_STATE, IS_LEADER, PEER_ERRORS}; #[derive(Fail, Debug)] pub enum PeerError { @@ -436,11 +436,6 @@ impl IntoFuture for PeerCommandClient { command, } = self; - let resp_required = if let PeerCommand::Status = command { - true - } else { - false - }; let future = tokio::net::TcpStream::connect(&address) .map_err(|e| PeerError::Io(e)) .and_then(move |conn| { From ec39cac53bd76de85bfa6959c5c4d7ee887c0a23 Mon Sep 17 00:00:00 2001 From: Albibek Date: Wed, 13 Jun 2018 07:26:21 +0300 Subject: [PATCH 03/24] Remake command schema slightly --- schema/protocol.capnp | 44 ++++++++++++++++++++++++++++--------------- 1 file changed, 29 insertions(+), 15 deletions(-) diff --git a/schema/protocol.capnp b/schema/protocol.capnp index 448d220..bd03a90 100644 --- a/schema/protocol.capnp +++ b/schema/protocol.capnp @@ -23,25 +23,39 @@ struct Message { # WARNING: This is reserved for future, only some commands may work struct PeerCommand { union { - # pause consensus leadership changes, see description below - pauseConsensus @0 :PauseConsensusCommand; - - # Resume consensus. Leader will be set to one from consensus - resumeConsensus @1 :Void; - # server will answer with ServerStatus message - status @2 :Void; + status @0 :Void; + + # send a command to consensus module + consensusCommand @1 :ConsensusCommand; } } -# Turn consensus off for time(in milliseconds). The consensus module will still work, but signals -# on leadership changes will be ignored -# Leader state will be unchanged if setLeader is 0 -# Leader will be disabled if setLeader is < 0 -# Leader will be enabled if setLeader is > 0 -struct PauseConsensusCommand { - pause @0 :UInt64; - setLeader @1 :Int8; +# Turn consensus off for time(in milliseconds). +struct ConsensusCommand { + action :union { + + # enable consensus leadership + enable @0 :Void; + + # disable consensus leadership changes consensus will be turned off + disable @1 :Void; + + # Pause consensus leadership. + # The consensus module will still work and interact with others, + # but any leadership changes will not be counted by backend as internal leader state + pause @2 :UInt64; + + # resume consensus from pause + resume @3 :Void; + } + + # Along with the consensus state change the internal leadership state can be changed + setLeader :union { + unchanged @4 :Void; + enable @5 :Void; + disable @6 :Void; + } } struct ServerStatus { From 76dc883e516af274838af4fb5cc7250aa64e89d3 Mon Sep 17 00:00:00 2001 From: Albibek Date: Wed, 27 Jun 2018 21:54:19 +0300 Subject: [PATCH 04/24] Intermediate step to make management web based --- Cargo.toml | 1 - schema/protocol.capnp | 44 ------- src/config.rs | 19 +-- src/main.rs | 9 +- src/management.rs | 285 ++++++++++++++++++++++++++++++++++++++++++ src/peer.rs | 161 ------------------------ 6 files changed, 301 insertions(+), 218 deletions(-) create mode 100644 src/management.rs diff --git a/Cargo.toml b/Cargo.toml index 67d9150..b36cd3a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,7 +23,6 @@ mime="^0.3" serde="^1.0" serde_derive="^1.0" serde_json="^1.0" -bincode="^1.0" slog="^2.2" slog-term="^2.4" slog-async="^2.3" diff --git a/schema/protocol.capnp b/schema/protocol.capnp index bd03a90..9ae1abb 100644 --- a/schema/protocol.capnp +++ b/schema/protocol.capnp @@ -19,50 +19,6 @@ struct Message { } } -# A message type for internal messaging, should not be used by clients -# WARNING: This is reserved for future, only some commands may work -struct PeerCommand { - union { - # server will answer with ServerStatus message - status @0 :Void; - - # send a command to consensus module - consensusCommand @1 :ConsensusCommand; - } -} - -# Turn consensus off for time(in milliseconds). -struct ConsensusCommand { - action :union { - - # enable consensus leadership - enable @0 :Void; - - # disable consensus leadership changes consensus will be turned off - disable @1 :Void; - - # Pause consensus leadership. - # The consensus module will still work and interact with others, - # but any leadership changes will not be counted by backend as internal leader state - pause @2 :UInt64; - - # resume consensus from pause - resume @3 :Void; - } - - # Along with the consensus state change the internal leadership state can be changed - setLeader :union { - unchanged @4 :Void; - enable @5 :Void; - disable @6 :Void; - } -} - -struct ServerStatus { - leaderStatus @0 :Bool; - consensusPaused @1 :UInt64; -} - struct Metric { # everyone should have a name, even metrics diff --git a/src/config.rs b/src/config.rs index ae2479d..55613cd 100644 --- a/src/config.rs +++ b/src/config.rs @@ -1,9 +1,12 @@ -use clap::{Arg, SubCommand}; -use peer::PeerCommand; use std::fs::File; use std::io::Read; use std::net::SocketAddr; + +use clap::{Arg, SubCommand}; use toml; + +use management::MgmtCommand; + use ConsensusState; #[derive(Debug, Clone, Serialize, Deserialize)] @@ -214,7 +217,7 @@ impl Default for Consul { #[derive(Debug)] pub enum Command { Daemon, - Query(PeerCommand, String), + Query(MgmtCommand, String), } impl System { @@ -238,9 +241,9 @@ impl System { ) .subcommand( SubCommand::with_name("query") - .about("send a request to running peer server") - .arg(Arg::with_name("peer_command").index(1)) - .arg(Arg::with_name("server").default_value("127.0.0.1:8136")), + .about("send a management command to running bioyino server") + .arg(Arg::with_name("mgmt_command").index(1)) + .arg(Arg::with_name("server").default_value("127.0.0.1:8137")), ) .get_matches(); @@ -256,8 +259,8 @@ impl System { } if let Some(matches) = app.subcommand_matches("query") { - let cmd = - value_t!(matches.value_of("peer_command"), PeerCommand).expect("bad peer command"); + let cmd = value_t!(matches.value_of("mgmt_command"), MgmtCommand) + .expect("bad management command"); let server = value_t!(matches.value_of("server"), String).expect("bad server"); (system, Command::Query(cmd, server)) } else { diff --git a/src/main.rs b/src/main.rs index 969ab3c..29f40a4 100644 --- a/src/main.rs +++ b/src/main.rs @@ -31,7 +31,6 @@ extern crate tokio; extern crate tokio_io; // Other -extern crate bincode; extern crate combine; extern crate serde; #[macro_use] @@ -44,6 +43,7 @@ pub mod carbon; pub mod config; pub mod consul; pub mod errors; +pub mod management; pub mod metric; pub mod parser; pub mod peer; @@ -69,7 +69,7 @@ use slog::{Drain, Level}; use bytes::{Bytes, BytesMut}; use futures::future::{empty, ok}; -use futures::sync::mpsc; +use futures::sync::{mpsc, oneshot}; use futures::{Future, IntoFuture, Stream}; use tokio::net::UdpSocket; @@ -84,8 +84,9 @@ use carbon::CarbonBackend; use config::{Command, Consul, Metrics, Network, System}; use consul::ConsulConsensus; use errors::GeneralError; +use management::{MgmtClient, MgmtServer}; use metric::Metric; -use peer::{NativeProtocolServer, NativeProtocolSnapshot, PeerCommandClient}; +use peer::{NativeProtocolServer, NativeProtocolSnapshot}; use server::StatsdServer; use task::Task; use util::{AggregateOptions, Aggregator, BackoffRetryBuilder, OwnStats, UpdateCounterOptions}; @@ -213,7 +214,7 @@ fn main() { if let Command::Query(command, dest) = command { let dest = try_resolve(&dest); - let command = PeerCommandClient::new(rlog.clone(), dest.clone(), command); + let command = MgmtClient::new(rlog.clone(), dest.clone(), command); runtime.block_on(command.into_future()).unwrap_or_else(|e| { warn!(rlog, diff --git a/src/management.rs b/src/management.rs new file mode 100644 index 0000000..d349c18 --- /dev/null +++ b/src/management.rs @@ -0,0 +1,285 @@ +use std::net::SocketAddr; +use std::str::FromStr; +use std::sync::atomic::Ordering; +use std::time::{Duration, Instant}; + +use capnp; +use capnp::message::{Builder, ReaderOptions}; +use capnp_futures::ReadStream; +use futures::future::{err, join_all, ok, Future, IntoFuture}; +use futures::sync::mpsc::Sender; +use futures::sync::oneshot; +use futures::{Async, AsyncSink, Poll, Sink, StartSend, Stream}; +use serde_json; +use slog::Logger; +use tokio; +use tokio::executor::current_thread::spawn; +use tokio::net::{TcpListener, TcpStream}; +use tokio::timer::Interval; +use tokio_io::codec::length_delimited; +use tokio_io::codec::length_delimited::Framed; +use tokio_io::{AsyncRead, AsyncWrite}; + +use hyper::service::{NewService, Service}; +use hyper::{self, Body, Method, Request, Response, StatusCode}; + +use metric::{Metric, MetricError}; +use protocol_capnp::message as cmsg; + +use failure::Compat; +use task::Task; +use {Cache, ConsensusState, Float, CONSENSUS_STATE, IS_LEADER, PEER_ERRORS}; + +#[derive(Fail, Debug)] +pub enum MgmtError { + #[fail(display = "I/O error: {}", _0)] + Io(#[cause] ::std::io::Error), + + #[fail(display = "Http error: {}", _0)] + Http(#[cause] hyper::Error), + + #[fail(display = "JSON decoding error {}", _0)] + Decode(#[cause] serde_json::error::Error), + + #[fail(display = "JSON encoding error: {}", _0)] + Encode(#[cause] serde_json::error::Error), + + #[fail(display = "bad command")] + BadCommand, + + #[fail(display = "response not sent")] + Response, +} + +// Top level list of available commands +#[derive(Clone, Debug, Serialize, Deserialize)] +pub enum MgmtCommand { + // server will answer with ServerStatus message + Status, + // send a command to consensus module + ConsensusCommand(ConsensusAction, LeaderAction), +} + +impl FromStr for MgmtCommand { + type Err = Compat; + fn from_str(s: &str) -> Result { + // FIXME: + unimplemented!() + } +} + +// Turn consensus off for time(in milliseconds). +#[derive(Clone, Debug, Serialize, Deserialize)] +pub enum ConsensusAction { + // enable consensus leadership + Enable, + // disable consensus leadership changes consensus will be turned off + Disable, + // Pause consensus leadership. + // The consensus module will still work and interact with others, + // but any leadership changes will not be counted by backend as internal leader state + Pause, + // resume consensus from pause + Resume, +} + +// Along with the consensus state change the internal leadership state can be changed +#[derive(Clone, Debug, Serialize, Deserialize)] +pub enum LeaderAction { + Unchanged, + Enable, + Disable, +} + +// this is what answered as server response +#[derive(Debug, Serialize, Deserialize)] +struct ServerStatus { + leader_status: bool, + consensus_status: ConsensusState, +} + +pub struct MgmtServer; +impl Service for MgmtServer { + type ReqBody = Body; + type ResBody = Body; + type Error = Compat; + type Future = Box, Error = Self::Error>>; + fn call(&mut self, req: Request) -> Self::Future { + let mut response = Response::new(Body::empty()); + + match (req.method(), req.uri().path()) { + (&Method::GET, "/") => { + *response.body_mut() = Body::from( + "Available endpoints: + status - will show server status + consensus - posting will change consensus state", + ); + } + (&Method::GET, "/status") => { + *response.body_mut() = Body::from("HI"); + } + (&Method::POST, "/consensus") => { + // we'll be back + } + _ => { + *response.status_mut() = StatusCode::NOT_FOUND; + } + } + Box::new(ok(response)) + } +} + +#[derive(Clone, Debug)] +pub struct MgmtClient { + log: Logger, + address: SocketAddr, + command: MgmtCommand, +} + +impl MgmtClient { + pub fn new(log: Logger, address: SocketAddr, command: MgmtCommand) -> Self { + Self { + log: log.new( + o!("source"=>"management-client", "server"=>format!("{}", address.clone())), + ), + address, + command, + } + } +} + +impl IntoFuture for MgmtClient { + type Item = (); + type Error = MgmtError; + type Future = Box>; + + fn into_future(self) -> Self::Future { + let Self { + log, + address, + command, + } = self; + let mut req = hyper::Request::default(); + match command { + MgmtCommand::Status => { + *req.method_mut() = Method::GET; + *req.uri_mut() = format!("http://{}/status", address) + .parse() + .expect("creating url for management command "); + + let client = hyper::Client::new(); + let future = client.request(req).then(move |res| match res { + Err(e) => Box::new(err(MgmtError::Http(e))), + Ok(resp) => { + if resp.status() != StatusCode::OK { + let body = resp.into_body() + .concat2() + .map_err(|e| MgmtError::Http(e)) + .map(move |body| { + // FIXME unwrap + let status: ServerStatus = + serde_json::from_slice(&*body).unwrap(); + println!("{:?}", status); + }); + Box::new(body) as Box> + } else { + Box::new(ok(println!("Bad status returned from server"))) + } + } + }); + Box::new(future) + } + MgmtCommand::ConsensusCommand(consensus_action, leader_action) => { + let mut constate = CONSENSUS_STATE.lock().unwrap(); + + match consensus_action { + ConsensusAction::Enable | ConsensusAction::Resume => { + *constate = ConsensusState::Enabled; + } + ConsensusAction::Disable => { + *constate = ConsensusState::Disabled; + } + ConsensusAction::Pause => { + *constate = ConsensusState::Paused; + } + ConsensusAction::Resume => { + *constate = ConsensusState::Enabled; + } + } + + match leader_action { + LeaderAction::Enable => { + IS_LEADER.store(true, Ordering::SeqCst); + } + LeaderAction::Disable => { + IS_LEADER.store(false, Ordering::SeqCst); + } + _ => (), + } + + Box::new(ok(println!("State is set"))) + } + } + } +} + +#[cfg(test)] +mod test { + + use std::net::SocketAddr; + use std::thread; + use {slog, slog_async, slog_term}; + + use bytes::Bytes; + use capnp::message::Builder; + use futures::sync::mpsc::{self, Receiver}; + use metric::{Metric, MetricType}; + use slog::Drain; + use slog::Logger; + use std::time::{SystemTime, UNIX_EPOCH}; + use tokio::runtime::current_thread::Runtime; + use tokio::timer::Delay; + + use {LONG_CACHE, SHORT_CACHE}; + + use super::*; + fn prepare_log() -> Logger { + // Set logging + let decorator = slog_term::TermDecorator::new().build(); + let drain = slog_term::FullFormat::new(decorator).build().fuse(); + let filter = slog::LevelFilter::new(drain, slog::Level::Trace).fuse(); + let drain = slog_async::Async::new(filter).build().fuse(); + let rlog = slog::Logger::root(drain, o!("program"=>"test")); + return rlog; + } + + fn prepare_runtime_with_server( + test_timeout: Instant, + ) -> ( + Runtime, + Vec>, + Receiver, + Logger, + SocketAddr, + ) { + let rlog = prepare_log(); + let mut chans = Vec::new(); + let (tx, rx) = mpsc::channel(5); + chans.push(tx); + + let address: ::std::net::SocketAddr = "127.0.0.1:8136".parse().unwrap(); + let mut runtime = Runtime::new().expect("creating runtime for main thread"); + + let c_peer_listen = address.clone(); + let c_serv_log = rlog.clone(); + let peer_server = NativeProtocolServer::new(rlog.clone(), c_peer_listen, chans.clone()) + .into_future() + .map_err(move |e| { + warn!(c_serv_log, "shot server gone with error: {:?}", e); + }); + runtime.spawn(peer_server); + + (runtime, chans, rx, rlog, address) + } + +} diff --git a/src/peer.rs b/src/peer.rs index b07eee4..a064d3d 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -3,7 +3,6 @@ use std::str::FromStr; use std::sync::atomic::Ordering; use std::time::{Duration, Instant}; -use bincode; use capnp; use capnp::message::{Builder, ReaderOptions}; use capnp_futures::ReadStream; @@ -23,7 +22,6 @@ use tokio_io::{AsyncRead, AsyncWrite}; use metric::{Metric, MetricError}; use protocol_capnp::message as cmsg; -use protocol_capnp::peer_command; use task::Task; use {Cache, ConsensusState, Float, CONSENSUS_STATE, IS_LEADER, PEER_ERRORS}; @@ -35,12 +33,6 @@ pub enum PeerError { #[fail(display = "Error when creating timer: {}", _0)] Timer(#[cause] ::tokio::timer::Error), - #[fail(display = "bincode decoding error {}", _0)] - Decode(#[cause] Box), - - #[fail(display = "bincode encoding error: {}", _0)] - Encode(#[cause] Box), - #[fail(display = "error sending task to worker thread")] TaskSend, @@ -63,107 +55,6 @@ pub enum PeerError { Metric(MetricError), } -#[derive(Debug, Serialize, Deserialize)] -pub enum PeerCommand { - LeaderDisable, - LeaderEnable, - ForceLeader, - Status, -} - -impl FromStr for PeerCommand { - type Err = PeerError; - - fn from_str(s: &str) -> Result { - match s { - "leader_enable" => Ok(PeerCommand::LeaderEnable), - "leader_disable" => Ok(PeerCommand::LeaderDisable), - "force_leader" => Ok(PeerCommand::ForceLeader), - "status" => Ok(PeerCommand::Status), - _ => Err(PeerError::BadCommand), - } - } -} - -#[derive(Debug, Serialize, Deserialize)] -pub struct PeerStatus { - is_leader: bool, - can_leader: bool, - force_leader: bool, -} - -#[derive(Debug, Serialize, Deserialize)] -pub enum PeerMessage { - Snapshot(Vec), - Command(PeerCommand), - Status(PeerStatus), -} - -pub struct PeerCodec { - inner: Framed>, -} - -impl PeerCodec -where - T: AsyncRead + AsyncWrite, -{ - pub fn new(conn: T) -> Self { - Self { - inner: length_delimited::Builder::new() - .length_field_length(8) - // TODO: currently max snapshot size is 4Gb - // we should make it into an option or make - // snapshot sending iterative and splittable - .max_frame_length(::std::u32::MAX as usize) - .new_framed(conn), - } - } -} - -// Wrapper to decode message from length-encoded frame -impl Stream for PeerCodec -where - T: AsyncRead + AsyncWrite, -{ - type Item = PeerMessage; - type Error = PeerError; - - fn poll(&mut self) -> Poll, Self::Error> { - let message = match try_ready!(self.inner.poll().map_err(|e| PeerError::Io(e))) { - Some(buf) => Some(bincode::deserialize(&buf).map_err(|e| PeerError::Decode(e))?), - None => None, - }; - Ok(Async::Ready(message)) - } -} - -// Wrapper to encode message to length-encoded frame -impl Sink for PeerCodec -where - T: AsyncRead + AsyncWrite, -{ - type SinkItem = Option; - type SinkError = PeerError; - - fn start_send(&mut self, item: Self::SinkItem) -> StartSend { - match item { - Some(item) => { - let message = bincode::serialize(&item).map_err(|e| PeerError::Decode(e))?; - match self.inner.start_send(message) { - Ok(AsyncSink::NotReady(_)) => Ok(AsyncSink::NotReady(Some(item))), - Ok(AsyncSink::Ready) => Ok(AsyncSink::Ready), - Err(e) => Err(PeerError::Io(e)), - } - } - None => Ok(AsyncSink::Ready), - } - } - - fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { - self.inner.poll_complete().map_err(|e| PeerError::Io(e)) - } -} - #[derive(Clone, Debug)] pub struct NativeProtocolServer { log: Logger, @@ -406,58 +297,6 @@ impl IntoFuture for NativeProtocolSnapshot { } } -pub struct PeerCommandClient { - log: Logger, - address: SocketAddr, - command: PeerCommand, -} - -impl PeerCommandClient { - pub fn new(log: Logger, address: SocketAddr, command: PeerCommand) -> Self { - Self { - log: log.new( - o!("source"=>"peer-command-client", "server"=>format!("{}", address.clone())), - ), - address, - command, - } - } -} - -impl IntoFuture for PeerCommandClient { - type Item = (); - type Error = PeerError; - type Future = Box>; - - fn into_future(self) -> Self::Future { - let Self { - log, - address, - command, - } = self; - - let future = tokio::net::TcpStream::connect(&address) - .map_err(|e| PeerError::Io(e)) - .and_then(move |conn| { - let transport = ReadStream::new(conn, ReaderOptions::default()); - transport - .map_err(|e| PeerError::Capnp(e)) - .for_each(move |reader| { - // decode incoming capnp data into message - let reader = reader - .get_root::() - .map_err(PeerError::Capnp)?; - match reader.which().map_err(PeerError::CapnpSchema)? { - peer_command::Which::Status(()) => (), - _ => info!(log, "command not implemented"), - } - Ok(()) - }) - }); - Box::new(future) - } -} - #[cfg(test)] mod test { From 038a08369a31a4f21ec217809663d6314becf938 Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Wed, 11 Jul 2018 01:56:17 +0300 Subject: [PATCH 05/24] Done with management client and server --- Cargo.toml | 1 + config.toml | 12 +- src/carbon.rs | 6 +- src/config.rs | 47 +++++-- src/consul.rs | 2 - src/main.rs | 23 +++- src/management.rs | 343 ++++++++++++++++++++++++++++++++-------------- src/metric.rs | 4 +- src/peer.rs | 36 ++--- src/task.rs | 7 +- 10 files changed, 325 insertions(+), 156 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index b36cd3a..ff64ed6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -14,6 +14,7 @@ num_cpus="^1.8" futures="^0.1" tokio="^0.1" tokio-io="^0.1" +tokio-codec="^0.1" bytes = { version = "^0.4", features = [ "serde" ] } resolve="^0.1" net2="^0.2" diff --git a/config.toml b/config.toml index 796566c..0445060 100644 --- a/config.toml +++ b/config.toml @@ -11,6 +11,11 @@ w-threads = 4 # Queue size for single counting thread before packet is dropped task-queue-size = 1024 +# if server should start in leader enabled state +# NOTE: this is importnt to set this value to true, when working in standalone mode +# since leader state only changes on command +start-as-leader = false + # How often to gather own stats, in ms. Use 0 to disable (stats are still gathered and printed to log, # but not included in metric dump stats-interval = 10000 @@ -57,9 +62,12 @@ send-retries = 30 # Address:port to listen for metrics at listen = "127.0.0.1:8125" -# Address and port for replication/command server to listen on +# Address and port for replication server to listen on peer-listen = "127.0.0.1:8136" +# Address and port for management server to listen on +mgmt-listen = "127.0.0.1:8137" + # UDP buffer size for single packet. Needs to be around MTU. Packet's bytes after that value # may be lost bufsize = 1500 @@ -84,7 +92,7 @@ snapshot-interval = 1000 [consul] # Start in disabled leader finding mode -start-disabled = false +start-as = "disabled" # Consul agent address agent = "127.0.0.1:8500" diff --git a/src/carbon.rs b/src/carbon.rs index a07d603..6b23b3c 100644 --- a/src/carbon.rs +++ b/src/carbon.rs @@ -9,8 +9,7 @@ use ftoa; use futures::stream; use futures::{Future, IntoFuture, Sink, Stream}; use tokio::net::TcpStream; -use tokio_io::codec::{Decoder, Encoder}; -use tokio_io::AsyncRead; +use tokio_codec::{Decoder, Encoder}; use errors::GeneralError; @@ -63,8 +62,7 @@ impl IntoFuture for CarbonBackend { let conn = TcpStream::connect(&addr).map_err(|e| GeneralError::Io(e)); let future = conn.and_then(move |conn| { - let writer = conn.framed(CarbonCodec::new()); - //let metric_stream = stream::iter_ok::<_, ()>(metrics.clone()); + let writer = CarbonCodec::new().framed(conn); let metric_stream = stream::iter_ok::<_, ()>(SharedIter::new(metrics)); metric_stream .map_err(|_| GeneralError::CarbonBackend) diff --git a/src/config.rs b/src/config.rs index 55613cd..1f88dcc 100644 --- a/src/config.rs +++ b/src/config.rs @@ -5,7 +5,7 @@ use std::net::SocketAddr; use clap::{Arg, SubCommand}; use toml; -use management::MgmtCommand; +use management::{ConsensusAction, LeaderAction, MgmtCommand}; use ConsensusState; @@ -141,9 +141,12 @@ pub(crate) struct Network { /// Address and UDP port to listen for statsd metrics on pub listen: SocketAddr, - /// Address and port for replication/command server to listen on + /// Address and port for replication server to listen on pub peer_listen: SocketAddr, + /// Address and port for management server to listen on + pub mgmt_listen: SocketAddr, + /// UDP buffer size for single packet. Needs to be around MTU. Packet's bytes after that value /// may be lost pub bufsize: usize, @@ -172,6 +175,7 @@ impl Default for Network { Self { listen: "127.0.0.1:8125".parse().unwrap(), peer_listen: "127.0.0.1:8136".parse().unwrap(), + mgmt_listen: "127.0.0.1:8137".parse().unwrap(), bufsize: 1500, multimessage: false, mm_packets: 100, @@ -242,8 +246,21 @@ impl System { .subcommand( SubCommand::with_name("query") .about("send a management command to running bioyino server") - .arg(Arg::with_name("mgmt_command").index(1)) - .arg(Arg::with_name("server").default_value("127.0.0.1:8137")), + .arg( + Arg::with_name("host") + .short("h") + .default_value("127.0.0.1:8137"), + ) + .subcommand(SubCommand::with_name("status").about("get server state")) + .subcommand( + SubCommand::with_name("consensus") + .arg(Arg::with_name("action").index(1)) + .arg( + Arg::with_name("leader_action") + .index(2) + .default_value("unchanged"), + ), + ), ) .get_matches(); @@ -258,11 +275,23 @@ impl System { system.verbosity = v.into() } - if let Some(matches) = app.subcommand_matches("query") { - let cmd = value_t!(matches.value_of("mgmt_command"), MgmtCommand) - .expect("bad management command"); - let server = value_t!(matches.value_of("server"), String).expect("bad server"); - (system, Command::Query(cmd, server)) + if let Some(query) = app.subcommand_matches("query") { + let server = value_t!(query.value_of("host"), String).expect("bad server"); + if let Some(_) = query.subcommand_matches("status") { + (system, Command::Query(MgmtCommand::Status, server)) + } else if let Some(args) = query.subcommand_matches("consensus") { + let c_action = value_t!(args.value_of("action"), ConsensusAction) + .expect("bad consensus action"); + let l_action = value_t!(args.value_of("leader_action"), LeaderAction) + .expect("bad leader action"); + ( + system, + Command::Query(MgmtCommand::ConsensusCommand(c_action, l_action), server), + ) + } else { + // shold be unreachable + unreachable!("clap bug?") + } } else { (system, Command::Daemon) } diff --git a/src/consul.rs b/src/consul.rs index 231935e..663a32a 100644 --- a/src/consul.rs +++ b/src/consul.rs @@ -10,10 +10,8 @@ use hyper::{Method, StatusCode}; use mime::WWW_FORM_URLENCODED; use serde_json::{self, from_slice}; use slog::Logger; -use tokio::executor::current_thread::spawn; use tokio::timer::{self, Delay, Interval}; -use util::BackoffRetryBuilder; use {ConsensusState, CONSENSUS_STATE, IS_LEADER}; #[derive(Fail, Debug)] diff --git a/src/main.rs b/src/main.rs index 29f40a4..9e516ee 100644 --- a/src/main.rs +++ b/src/main.rs @@ -17,10 +17,9 @@ extern crate toml; // Network extern crate bytes; -#[macro_use] -extern crate futures; extern crate capnp; extern crate capnp_futures; +extern crate futures; extern crate hyper; extern crate libc; extern crate mime; @@ -28,6 +27,7 @@ extern crate net2; extern crate num_cpus; extern crate resolve; extern crate tokio; +extern crate tokio_codec; extern crate tokio_io; // Other @@ -69,7 +69,7 @@ use slog::{Drain, Level}; use bytes::{Bytes, BytesMut}; use futures::future::{empty, ok}; -use futures::sync::{mpsc, oneshot}; +use futures::sync::mpsc; use futures::{Future, IntoFuture, Stream}; use tokio::net::UdpSocket; @@ -111,6 +111,7 @@ pub static EGRESS: AtomicUsize = ATOMIC_USIZE_INIT; pub static DROPS: AtomicUsize = ATOMIC_USIZE_INIT; #[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case", deny_unknown_fields)] pub enum ConsensusState { Enabled, Paused, @@ -131,7 +132,8 @@ impl FromStr for ConsensusState { } lazy_static! { - pub static ref CONSENSUS_STATE: Mutex = { Mutex::new(ConsensusState::Disabled) }; + pub static ref CONSENSUS_STATE: Mutex = + { Mutex::new(ConsensusState::Disabled) }; } pub static IS_LEADER: AtomicBool = ATOMIC_BOOL_INIT; @@ -161,6 +163,7 @@ fn main() { Network { listen, peer_listen, + mgmt_listen, bufsize, multimessage, mm_packets, @@ -234,6 +237,7 @@ fn main() { let update_counter_prefix: Bytes = update_counter_prefix.into(); let update_counter_suffix: Bytes = update_counter_suffix.into(); let log = rlog.new(o!("thread" => "main")); + // Start counting threads info!(log, "starting counting threads"); let mut chans = Vec::with_capacity(w_threads); @@ -307,6 +311,17 @@ fn main() { consensus.set_renew_time(Duration::from_millis(consul_renew_time as u64)); runtime.spawn(consensus.into_future().map_err(|_| ())); // TODO errors + info!(log, "starting management server"); + let m_serv_log = rlog.clone(); + let m_serv_err_log = rlog.clone(); + let m_server = hyper::Server::bind(&mgmt_listen) + .serve(move || ok::<_, hyper::Error>(MgmtServer::new(m_serv_log.clone(), &mgmt_listen))) + .map_err(move |e| { + warn!(m_serv_err_log, "management server gone with error: {:?}", e); + }); + + runtime.spawn(m_server); + info!(log, "starting carbon backend"); let tchans = chans.clone(); let carbon_log = rlog.clone(); diff --git a/src/management.rs b/src/management.rs index d349c18..791ae79 100644 --- a/src/management.rs +++ b/src/management.rs @@ -1,34 +1,17 @@ use std::net::SocketAddr; use std::str::FromStr; use std::sync::atomic::Ordering; -use std::time::{Duration, Instant}; - -use capnp; -use capnp::message::{Builder, ReaderOptions}; -use capnp_futures::ReadStream; -use futures::future::{err, join_all, ok, Future, IntoFuture}; -use futures::sync::mpsc::Sender; -use futures::sync::oneshot; -use futures::{Async, AsyncSink, Poll, Sink, StartSend, Stream}; + +use futures::future::{err, ok, Future, IntoFuture}; +use futures::Stream; use serde_json; use slog::Logger; -use tokio; -use tokio::executor::current_thread::spawn; -use tokio::net::{TcpListener, TcpStream}; -use tokio::timer::Interval; -use tokio_io::codec::length_delimited; -use tokio_io::codec::length_delimited::Framed; -use tokio_io::{AsyncRead, AsyncWrite}; - -use hyper::service::{NewService, Service}; -use hyper::{self, Body, Method, Request, Response, StatusCode}; -use metric::{Metric, MetricError}; -use protocol_capnp::message as cmsg; +use hyper::service::Service; +use hyper::{self, Body, Method, Request, Response, StatusCode}; -use failure::Compat; -use task::Task; -use {Cache, ConsensusState, Float, CONSENSUS_STATE, IS_LEADER, PEER_ERRORS}; +use failure::{Compat, Fail}; +use {ConsensusState, CONSENSUS_STATE, IS_LEADER}; #[derive(Fail, Debug)] pub enum MgmtError { @@ -53,6 +36,7 @@ pub enum MgmtError { // Top level list of available commands #[derive(Clone, Debug, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case", deny_unknown_fields)] pub enum MgmtCommand { // server will answer with ServerStatus message Status, @@ -60,20 +44,13 @@ pub enum MgmtCommand { ConsensusCommand(ConsensusAction, LeaderAction), } -impl FromStr for MgmtCommand { - type Err = Compat; - fn from_str(s: &str) -> Result { - // FIXME: - unimplemented!() - } -} - // Turn consensus off for time(in milliseconds). #[derive(Clone, Debug, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case", deny_unknown_fields)] pub enum ConsensusAction { // enable consensus leadership Enable, - // disable consensus leadership changes consensus will be turned off + // disable consensus leadership changes, consensus will be turned off Disable, // Pause consensus leadership. // The consensus module will still work and interact with others, @@ -83,49 +60,159 @@ pub enum ConsensusAction { Resume, } +impl FromStr for ConsensusAction { + type Err = Compat; + fn from_str(s: &str) -> Result { + match s { + "enable" | "enabled" => Ok(ConsensusAction::Enable), + "disable" | "disabled" => Ok(ConsensusAction::Disable), + "pause" | "paused" => Ok(ConsensusAction::Pause), + "resume" | "resumed" | "unpause" | "unpaused" => Ok(ConsensusAction::Resume), + _ => Err(MgmtError::BadCommand.compat()), + } + } +} + // Along with the consensus state change the internal leadership state can be changed #[derive(Clone, Debug, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case", deny_unknown_fields)] pub enum LeaderAction { Unchanged, Enable, Disable, } +impl FromStr for LeaderAction { + type Err = Compat; + fn from_str(s: &str) -> Result { + match s { + "unchanged" | "unchange" => Ok(LeaderAction::Unchanged), + "enable" | "enabled" => Ok(LeaderAction::Enable), + "disable" | "disabled" => Ok(LeaderAction::Disable), + _ => Err(MgmtError::BadCommand.compat()), + } + } +} // this is what answered as server response -#[derive(Debug, Serialize, Deserialize)] +#[derive(Debug, Serialize, Deserialize, PartialEq)] +#[serde(rename_all = "kebab-case", deny_unknown_fields)] struct ServerStatus { leader_status: bool, consensus_status: ConsensusState, } -pub struct MgmtServer; +impl ServerStatus { + fn new() -> Self { + let state = &*CONSENSUS_STATE.lock().unwrap(); + Self { + leader_status: IS_LEADER.load(Ordering::SeqCst), + consensus_status: state.clone(), + } + } +} + +pub struct MgmtServer { + log: Logger, +} + +impl MgmtServer { + pub fn new(log: Logger, address: &SocketAddr) -> Self { + Self { + log: log.new(o!("source"=>"management-server", "server"=>format!("{}", address))), + } + } +} + impl Service for MgmtServer { type ReqBody = Body; type ResBody = Body; - type Error = Compat; - type Future = Box, Error = Self::Error>>; + type Error = hyper::Error; + type Future = Box, Error = Self::Error> + Send>; fn call(&mut self, req: Request) -> Self::Future { let mut response = Response::new(Body::empty()); + let log = self.log.clone(); match (req.method(), req.uri().path()) { (&Method::GET, "/") => { *response.body_mut() = Body::from( "Available endpoints: - status - will show server status - consensus - posting will change consensus state", + status - will show server status + consensus - posting will change consensus state", ); + Box::new(ok(response)) } (&Method::GET, "/status") => { - *response.body_mut() = Body::from("HI"); + let status = ServerStatus::new(); + let body = serde_json::to_vec_pretty(&status).unwrap(); // TODO unwrap + *response.body_mut() = Body::from(body); + Box::new(ok(response)) + } + (&Method::GET, _) => { + *response.status_mut() = StatusCode::NOT_FOUND; + Box::new(ok(response)) } (&Method::POST, "/consensus") => { - // we'll be back + let fut = req.into_body().concat2().map(move |body| { + match serde_json::from_slice(&*body) { + Ok(MgmtCommand::ConsensusCommand(consensus_action, leader_action)) => { + { + // free a lock to avoid server status deadlocking + let mut constate = CONSENSUS_STATE.lock().unwrap(); + + *constate = match consensus_action { + ConsensusAction::Enable | ConsensusAction::Resume => { + ConsensusState::Enabled + } + ConsensusAction::Disable => ConsensusState::Disabled, + ConsensusAction::Pause => ConsensusState::Paused, + }; + } + + match leader_action { + LeaderAction::Enable => { + IS_LEADER.store(true, Ordering::SeqCst); + } + LeaderAction::Disable => { + IS_LEADER.store(false, Ordering::SeqCst); + } + _ => (), + }; + + *response.status_mut() = StatusCode::OK; + + let status = ServerStatus::new(); + let body = serde_json::to_vec_pretty(&status).unwrap(); // TODO unwrap + *response.body_mut() = Body::from(body); + info!(log, "state changed"; "consensus_state"=>format!("{:?}", status.consensus_status), "leader_state"=>status.leader_status); + + response + } + Ok(command) => { + info!(log, "bad command received"; "command"=>format!("{:?}", command)); + *response.status_mut() = StatusCode::BAD_REQUEST; + + response + } + Err(e) => { + info!(log, "error parsing command"; "error"=>e.to_string()); + *response.status_mut() = StatusCode::BAD_REQUEST; + + response + } + } + }); + + Box::new(fut) } - _ => { + (&Method::POST, _) => { *response.status_mut() = StatusCode::NOT_FOUND; + Box::new(ok(response)) + } + _ => { + *response.status_mut() = StatusCode::METHOD_NOT_ALLOWED; + Box::new(ok(response)) } } - Box::new(ok(response)) } } @@ -160,6 +247,8 @@ impl IntoFuture for MgmtClient { command, } = self; let mut req = hyper::Request::default(); + + info!(log, "received command {:?}", command); match command { MgmtCommand::Status => { *req.method_mut() = Method::GET; @@ -168,56 +257,78 @@ impl IntoFuture for MgmtClient { .expect("creating url for management command "); let client = hyper::Client::new(); + let clog = log.clone(); let future = client.request(req).then(move |res| match res { Err(e) => Box::new(err(MgmtError::Http(e))), Ok(resp) => { - if resp.status() != StatusCode::OK { + if resp.status() == StatusCode::OK { let body = resp.into_body() .concat2() .map_err(|e| MgmtError::Http(e)) .map(move |body| { - // FIXME unwrap - let status: ServerStatus = - serde_json::from_slice(&*body).unwrap(); - println!("{:?}", status); + match serde_json::from_slice::(&*body) { + Ok(status) => { + println!("{:?}", status); + } + Err(e) => { + println!( + "Error parsing server response: {}", + e.to_string() + ); + } + } }); Box::new(body) as Box> } else { - Box::new(ok(println!("Bad status returned from server"))) + Box::new(ok(warn!( + clog, + "Bad status returned from server: {:?}", resp + ))) } } }); Box::new(future) } - MgmtCommand::ConsensusCommand(consensus_action, leader_action) => { - let mut constate = CONSENSUS_STATE.lock().unwrap(); - - match consensus_action { - ConsensusAction::Enable | ConsensusAction::Resume => { - *constate = ConsensusState::Enabled; - } - ConsensusAction::Disable => { - *constate = ConsensusState::Disabled; - } - ConsensusAction::Pause => { - *constate = ConsensusState::Paused; - } - ConsensusAction::Resume => { - *constate = ConsensusState::Enabled; - } - } + command @ MgmtCommand::ConsensusCommand(_, _) => { + *req.method_mut() = Method::POST; + *req.uri_mut() = format!("http://{}/consensus", address) + .parse() + .expect("creating url for management command"); + let body = serde_json::to_vec_pretty(&command).unwrap(); + *req.body_mut() = Body::from(body); - match leader_action { - LeaderAction::Enable => { - IS_LEADER.store(true, Ordering::SeqCst); - } - LeaderAction::Disable => { - IS_LEADER.store(false, Ordering::SeqCst); + let client = hyper::Client::new(); + let clog = log.clone(); + let future = client.request(req).then(move |res| match res { + Err(e) => Box::new(err(MgmtError::Http(e))), + Ok(resp) => { + if resp.status() == StatusCode::OK { + let body = resp.into_body() + .concat2() + .map_err(|e| MgmtError::Http(e)) + .map(move |body| { + match serde_json::from_slice::(&*body) { + Ok(status) => { + println!("New server state: {:?}", status); + } + Err(e) => { + println!( + "Error parsing server response: {}", + e.to_string() + ); + } + } + }); + Box::new(body) as Box> + } else { + Box::new(ok(warn!( + clog, + "Bad status returned from server: {:?}", resp + ))) + } } - _ => (), - } - - Box::new(ok(println!("State is set"))) + }); + Box::new(future) } } } @@ -227,21 +338,14 @@ impl IntoFuture for MgmtClient { mod test { use std::net::SocketAddr; - use std::thread; + use std::time::{Duration, Instant}; use {slog, slog_async, slog_term}; - use bytes::Bytes; - use capnp::message::Builder; - use futures::sync::mpsc::{self, Receiver}; - use metric::{Metric, MetricType}; use slog::Drain; use slog::Logger; - use std::time::{SystemTime, UNIX_EPOCH}; use tokio::runtime::current_thread::Runtime; use tokio::timer::Delay; - use {LONG_CACHE, SHORT_CACHE}; - use super::*; fn prepare_log() -> Logger { // Set logging @@ -253,33 +357,66 @@ mod test { return rlog; } - fn prepare_runtime_with_server( - test_timeout: Instant, - ) -> ( - Runtime, - Vec>, - Receiver, - Logger, - SocketAddr, - ) { + fn prepare_runtime_with_server() -> (Runtime, Logger, SocketAddr) { let rlog = prepare_log(); - let mut chans = Vec::new(); - let (tx, rx) = mpsc::channel(5); - chans.push(tx); - let address: ::std::net::SocketAddr = "127.0.0.1:8136".parse().unwrap(); + let address: ::std::net::SocketAddr = "127.0.0.1:8137".parse().unwrap(); let mut runtime = Runtime::new().expect("creating runtime for main thread"); - let c_peer_listen = address.clone(); let c_serv_log = rlog.clone(); - let peer_server = NativeProtocolServer::new(rlog.clone(), c_peer_listen, chans.clone()) - .into_future() + let c_serv_err_log = rlog.clone(); + let s_addr = address.clone(); + let server = hyper::Server::bind(&address) + .serve(move || ok::<_, hyper::Error>(MgmtServer::new(c_serv_log.clone(), &s_addr))) .map_err(move |e| { - warn!(c_serv_log, "shot server gone with error: {:?}", e); + warn!(c_serv_err_log, "management server gone with error: {:?}", e); }); - runtime.spawn(peer_server); - (runtime, chans, rx, rlog, address) + runtime.spawn(server); + + (runtime, rlog, address) } + #[test] + fn management_command() { + let test_timeout = Instant::now() + Duration::from_secs(3); + let (mut runtime, log, address) = prepare_runtime_with_server(); + + let command = MgmtCommand::Status; + let client = MgmtClient::new(log.clone(), address, command); + + // let server some time to settle + // then test the status command + let d = Delay::new(Instant::now() + Duration::from_secs(1)); + let delayed = d.map_err(|_| ()) + .and_then(move |_| client.into_future().map_err(|e| panic!(e))); + runtime.spawn(delayed); + + // then send a status change command + let d = Delay::new(Instant::now() + Duration::from_secs(2)); + let delayed = d.map_err(|_| ()).and_then(move |_| { + let command = + MgmtCommand::ConsensusCommand(ConsensusAction::Enable, LeaderAction::Enable); + let client = MgmtClient::new(log.clone(), address, command); + + client + .into_future() + .map_err(|e| panic!("{:?}", e)) + .map(move |_| { + // ensure state has changed + let state = ServerStatus::new(); + assert_eq!( + state, + ServerStatus { + consensus_status: ConsensusState::Enabled, + leader_status: true, + } + ) + }) + }); + runtime.spawn(delayed); + + let test_delay = Delay::new(test_timeout); + runtime.block_on(test_delay).expect("runtime"); + } } diff --git a/src/metric.rs b/src/metric.rs index cc39679..8ff63e6 100644 --- a/src/metric.rs +++ b/src/metric.rs @@ -3,8 +3,7 @@ use std::ops::{Add, AddAssign, Div, Mul, Sub, SubAssign}; use bytes::Bytes; use capnp; -use capnp::message::{Allocator, Builder, HeapAllocator, Reader, ReaderSegments}; -//use capnp::message::ReaderSegments; +use capnp::message::{Allocator, Builder, HeapAllocator}; use failure::Error; use protocol_capnp::metric as cmetric; @@ -210,7 +209,6 @@ where } else { None }, - // .ok() Some(reader.get_update_counter()), ), Err(_) => (None, None), diff --git a/src/peer.rs b/src/peer.rs index a064d3d..f7f1388 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -1,29 +1,24 @@ use std::net::SocketAddr; -use std::str::FromStr; use std::sync::atomic::Ordering; use std::time::{Duration, Instant}; use capnp; use capnp::message::{Builder, ReaderOptions}; use capnp_futures::ReadStream; -use futures::future::{join_all, ok, Future, IntoFuture}; +use futures::future::{join_all, Future, IntoFuture}; use futures::sync::mpsc::Sender; use futures::sync::oneshot; -use futures::{Async, AsyncSink, Poll, Sink, StartSend, Stream}; +use futures::{Sink, Stream}; use slog::Logger; -use tokio; use tokio::executor::current_thread::spawn; use tokio::net::{TcpListener, TcpStream}; use tokio::timer::Interval; -use tokio_io::codec::length_delimited; -use tokio_io::codec::length_delimited::Framed; -use tokio_io::{AsyncRead, AsyncWrite}; use metric::{Metric, MetricError}; use protocol_capnp::message as cmsg; use task::Task; -use {Cache, ConsensusState, Float, CONSENSUS_STATE, IS_LEADER, PEER_ERRORS}; +use {Float, PEER_ERRORS}; #[derive(Fail, Debug)] pub enum PeerError { @@ -107,7 +102,7 @@ impl IntoFuture for NativeProtocolServer { }) }) .for_each(|_| { - // + // Consume all messages from the stream Ok(()) }) }); @@ -239,7 +234,7 @@ impl IntoFuture for NativeProtocolSnapshot { // All nodes have to receive the same metrics // so we don't parallel connections and metrics fetching - // TODO: we probably clne a lots of bytes here, + // TODO: we probably clone a lots of bytes here, // could've changed them to Arc let dlog = log.clone(); let elog = log.clone(); @@ -301,11 +296,11 @@ impl IntoFuture for NativeProtocolSnapshot { mod test { use std::net::SocketAddr; - use std::thread; use {slog, slog_async, slog_term}; use bytes::Bytes; use capnp::message::Builder; + use futures::future::ok; use futures::sync::mpsc::{self, Receiver}; use metric::{Metric, MetricType}; use slog::Drain; @@ -327,15 +322,7 @@ mod test { return rlog; } - fn prepare_runtime_with_server( - test_timeout: Instant, - ) -> ( - Runtime, - Vec>, - Receiver, - Logger, - SocketAddr, - ) { + fn prepare_runtime_with_server() -> (Runtime, Receiver, Logger, SocketAddr) { let rlog = prepare_log(); let mut chans = Vec::new(); let (tx, rx) = mpsc::channel(5); @@ -346,20 +333,20 @@ mod test { let c_peer_listen = address.clone(); let c_serv_log = rlog.clone(); - let peer_server = NativeProtocolServer::new(rlog.clone(), c_peer_listen, chans.clone()) + let peer_server = NativeProtocolServer::new(rlog.clone(), c_peer_listen, chans) .into_future() .map_err(move |e| { warn!(c_serv_log, "shot server gone with error: {:?}", e); }); runtime.spawn(peer_server); - (runtime, chans, rx, rlog, address) + (runtime, rx, rlog, address) } #[test] fn test_peer_protocol_capnp() { let test_timeout = Instant::now() + Duration::from_secs(3); - let (mut runtime, chans, rx, rlog, address) = prepare_runtime_with_server(test_timeout); + let (mut runtime, rx, rlog, address) = prepare_runtime_with_server(); let future = rx.for_each(move |task: Task| ok(task.run()).and_then(|_| Ok(()))); runtime.spawn(future); @@ -417,9 +404,6 @@ mod test { }) .map_err(move |e| debug!(log, "error sending snapshot: {:?}", e)); - let metric = outmetric.clone(); - let log = rlog.clone(); - let d = Delay::new(Instant::now() + Duration::from_secs(1)); let delayed = d.map_err(|_| ()).and_then(|_| sender); runtime.spawn(delayed); diff --git a/src/task.rs b/src/task.rs index 8944a7b..39b3475 100644 --- a/src/task.rs +++ b/src/task.rs @@ -3,7 +3,6 @@ use std::sync::atomic::Ordering; use bytes::{BufMut, Bytes, BytesMut}; use combine::Parser; -use futures::future::Either; use futures::sync::mpsc::UnboundedSender; use futures::sync::oneshot; use futures::Sink; @@ -12,8 +11,10 @@ use metric::Metric; use parser::metric_parser; use util::AggregateOptions; -use {Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, LONG_CACHE, PARSE_ERRORS, PEER_ERRORS, - SHORT_CACHE}; +use { + Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, LONG_CACHE, PARSE_ERRORS, PEER_ERRORS, + SHORT_CACHE, +}; #[derive(Debug)] pub struct AggregateData { From 0e4fdbbc16ffed9af08a9ab35993b7305d7fbdd1 Mon Sep 17 00:00:00 2001 From: Albibek Date: Sun, 16 Sep 2018 12:03:14 +0300 Subject: [PATCH 06/24] Add test for parsing non-split metrics --- src/parser.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/parser.rs b/src/parser.rs index f8b0d77..9b74944 100644 --- a/src/parser.rs +++ b/src/parser.rs @@ -128,6 +128,21 @@ mod tests { assert_eq!(rest.len(), 0); } + #[test] + fn parse_metric_without_newline() { + let data = b"complex.bioyino.test1:-1e10|gcomplex.bioyino.test10:-1e10|g\n"; + let mut parser = metric_parser::(); + let (v, rest) = parser.parse(data).unwrap(); + assert_eq!(v.0, b"complex.bioyino.test1"); + assert_eq!(v.1, 1e10f64); + assert_eq!(v.2, MetricType::Gauge(Some(-1i8))); + let (v, rest) = parser.parse(rest).unwrap(); + assert_eq!(v.0, b"complex.bioyino.test10"); + assert_eq!(v.1, 1e10f64); + assert_eq!(v.2, MetricType::Gauge(Some(-1i8))); + assert_eq!(rest.len(), 0); + } + #[test] fn parse_metric_short() { let data = b"gorets:1|c"; From bcef683980fa8cb2f51b605e5e886cda70bec2b1 Mon Sep 17 00:00:00 2001 From: Albibek Date: Tue, 2 Oct 2018 22:14:10 +0300 Subject: [PATCH 07/24] Internal Raft, buffer flush, mmsg improvements --- Cargo.toml | 8 +- config.toml | 32 +++- schema/protocol.capnp | 8 +- src/carbon.rs | 26 ++- src/config.rs | 129 +++++++++---- src/consul.rs | 100 +++++----- src/errors.rs | 15 +- src/main.rs | 419 +++++++++++++----------------------------- src/management.rs | 113 ++++++------ src/metric.rs | 85 +-------- src/parser.rs | 28 +-- src/peer.rs | 69 +++---- src/raft.rs | 76 ++++++++ src/server.rs | 70 ++++--- src/task.rs | 65 ++++--- src/udp.rs | 256 ++++++++++++++++++++++++++ src/util.rs | 178 +++++++++++------- 17 files changed, 961 insertions(+), 716 deletions(-) create mode 100644 src/raft.rs create mode 100644 src/udp.rs diff --git a/Cargo.toml b/Cargo.toml index ff64ed6..427ccbb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -30,8 +30,10 @@ slog-async="^2.3" slog-scope="^4.0" toml="^0.4" ftoa = "^0.1" -capnp = "^0.8" -capnp-futures = "^0.1" +capnp = "^0.9" +capnp-futures = "^0.9" +raft-tokio = { git = "https://github.com/Albibek/raft-tokio" } +rand = "^0.5" [build-dependencies] -capnpc = "^0.8" +capnpc = "^0.9" diff --git a/config.toml b/config.toml index 0445060..8359338 100644 --- a/config.toml +++ b/config.toml @@ -23,6 +23,9 @@ stats-interval = 10000 # Prefix for sending own stats stats-prefix = "resources.monitoring.bioyino" +# What consensus to use: "consul", "internal" or "none" +consensus = "internal" + [metrics] # Should we provide metrics that update more than update-counter-threshold times diring aggregation interval count-updates = true @@ -76,13 +79,23 @@ bufsize = 1500 multimessage = false # Number of multimessage packets to receive at once if in multimessage mode +# Note that this setting is per thread, so in reality one can only see metrics +# after receiving at least mm-packets*n_threads datagrams mm-packets = 100 +# Do multimessage operations in async mode. +# This means recvmmsg will receive 0..mm-packets datagrams instead of waiting for mm-packets +mm-async = false + +# To avoid packets staying in queue forever, this option can be used to flush +# incoming data buffer forcing it to be sent even if it's not full +buffer-flush = 0 + # Nmber of green threads for single-message mode greens = 4 # Socket pool size for single-message mode -snum = 4 +async-sockets = 4 # List of nodes to replicate metrics to nodes = [] @@ -90,6 +103,23 @@ nodes = [] # Interval to send snapshots to nodes, ms snapshot-interval = 1000 +# Settings for internal Raft +[raft] +# Timeouts tuned according to the Raft paper and typical network latency. +# Better not to change if unsure +heartbeat-timeout = 250 +election-timeout-min = 500 +election-timeout-max = 750 + +# The name of the current node is taken from hostname by default +# After that all hostnames are resolved using DNS. If node name cannot +# be resolved through DNS for some reason, it can be specified in this-node +# parameter in a format similar to one in node list. +# this-node = + +# A list of other raft nodes in form of hostname:port or IP:port +nodes = [] + [consul] # Start in disabled leader finding mode start-as = "disabled" diff --git a/schema/protocol.capnp b/schema/protocol.capnp index 9ae1abb..4920fc8 100644 --- a/schema/protocol.capnp +++ b/schema/protocol.capnp @@ -27,7 +27,7 @@ struct Metric { # each metric has a value when it's sent value @1 :Float64; - # some types also imply additional internal values depending if metric type + # some types also imply additional internal values depending on metric type type @2 :MetricType; # a timesamp can optionally be sent, i.e. for historic reasons @@ -72,8 +72,14 @@ struct Gauge { struct MetricMeta { sampling @0 :Sampling; updateCounter @1 :UInt32; + tags @2 :List(Tag); } struct Sampling { sampling @0 :Float32; } + +struct Tag { + key @0 :Text; + value @1 :Text; +} diff --git a/src/carbon.rs b/src/carbon.rs index 6b23b3c..47764ab 100644 --- a/src/carbon.rs +++ b/src/carbon.rs @@ -27,25 +27,23 @@ impl CarbonBackend { let ts: Bytes = ts.as_secs().to_string().into(); let buf = BytesMut::with_capacity(metrics.len() * 200); // 200 is an approximate for full metric name + value - let (metrics, _) = metrics.iter().fold( - (Vec::new(), buf), - |(mut acc, mut buf), (name, metric)| { - let mut wr = buf.writer(); - let buf = match ftoa::write(&mut wr, *metric) { - Ok(()) => { + let (metrics, _) = metrics.iter().fold((Vec::new(), buf), |(mut acc, mut buf), + (name, metric)| { + let mut wr = buf.writer(); + let buf = match ftoa::write(&mut wr, *metric) { + Ok(()) => { buf = wr.into_inner(); let metric = buf.take().freeze(); acc.push((name.clone(), metric, ts.clone())); buf } - Err(_) => { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - wr.into_inner() - } - }; - (acc, buf) - }, - ); + Err(_) => { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + wr.into_inner() + } + }; + (acc, buf) + }); let metrics = Arc::new(metrics); let self_ = Self { addr, metrics }; self_ diff --git a/src/config.rs b/src/config.rs index 1f88dcc..8a3403d 100644 --- a/src/config.rs +++ b/src/config.rs @@ -1,13 +1,16 @@ use std::fs::File; use std::io::Read; use std::net::SocketAddr; +use std::time::Duration; +use std::ops::Range; use clap::{Arg, SubCommand}; use toml; use management::{ConsensusAction, LeaderAction, MgmtCommand}; -use ConsensusState; +use raft_tokio::RaftOptions; +use {ConsensusState, ConsensusKind}; #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] @@ -18,6 +21,9 @@ pub(crate) struct System { /// Network settings pub network: Network, + /// Internal Raft settings + pub raft: Raft, + /// Consul settings pub consul: Consul, @@ -45,6 +51,9 @@ pub(crate) struct System { /// Prefix to send own metrics with pub stats_prefix: String, + + /// Consensus kind to use + pub consensus: ConsensusKind, } impl Default for System { @@ -52,6 +61,7 @@ impl Default for System { Self { verbosity: "warn".to_string(), network: Network::default(), + raft: Raft::default(), consul: Consul::default(), metrics: Metrics::default(), carbon: Carbon::default(), @@ -61,6 +71,7 @@ impl Default for System { task_queue_size: 2048, start_as_leader: false, stats_prefix: "resources.monitoring.bioyino".to_string(), + consensus: ConsensusKind::None, } } } @@ -157,11 +168,17 @@ pub(crate) struct Network { /// Number of multimessage packets to receive at once if in multimessage mode pub mm_packets: usize, + /// Number of multimessage packets to receive at once if in multimessage mode + pub mm_async: bool, + + /// A timer to flush incoming buffer making sure metrics are not stuck there + pub buffer_flush: u64, + /// Nmber of green threads for single-message mode pub greens: usize, /// Socket pool size for single-message mode - pub snum: usize, + pub async_sockets: usize, /// List of nodes to replicate metrics to pub nodes: Vec, @@ -179,8 +196,10 @@ impl Default for Network { bufsize: 1500, multimessage: false, mm_packets: 100, + mm_async: false, + buffer_flush: 3000, greens: 4, - snum: 4, + async_sockets: 4, nodes: Vec::new(), snapshot_interval: 1000, } @@ -218,6 +237,47 @@ impl Default for Consul { } } +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case", default, deny_unknown_fields)] +pub(crate) struct Raft { + /// Raft heartbeat timeout (ms) + pub heartbeat_timeout: u64, + + /// Raft heartbeat timeout (ms) + pub election_timeout_min: u64, + + /// Raft heartbeat timeout (ms) + pub election_timeout_max: u64, + + /// Name of this node. By default is taken by resolving hostname in DNS. + pub this_node: Option, + + /// List of Raft nodes, may include this_node + pub nodes: Vec, +} + +impl Default for Raft { + fn default() -> Self { + Self { + heartbeat_timeout: 250, + election_timeout_min: 500, + election_timeout_max: 750, + this_node: None, + nodes: Vec::new(), + } + } +} + +impl Raft { + pub fn get_raft_options(&self) -> RaftOptions { + RaftOptions { + heartbeat_timeout: Duration::from_millis(self.heartbeat_timeout), + election_timeout: Range{start: Duration::from_millis(self.election_timeout_min), end: Duration::from_millis(self.election_timeout_max)}, + + } + } +} + #[derive(Debug)] pub enum Command { Daemon, @@ -230,45 +290,42 @@ impl System { let app = app_from_crate!() .arg( Arg::with_name("config") - .help("configuration file path") - .long("config") - .short("c") - .required(true) - .takes_value(true) - .default_value("/etc/bioyino/bioyino.toml"), - ) + .help("configuration file path") + .long("config") + .short("c") + .required(true) + .takes_value(true) + .default_value("/etc/bioyino/bioyino.toml"), + ) .arg( Arg::with_name("verbosity") - .short("v") - .help("logging level") - .takes_value(true), - ) + .short("v") + .help("logging level") + .takes_value(true), + ) .subcommand( SubCommand::with_name("query") - .about("send a management command to running bioyino server") - .arg( - Arg::with_name("host") - .short("h") - .default_value("127.0.0.1:8137"), - ) - .subcommand(SubCommand::with_name("status").about("get server state")) - .subcommand( - SubCommand::with_name("consensus") - .arg(Arg::with_name("action").index(1)) - .arg( - Arg::with_name("leader_action") - .index(2) - .default_value("unchanged"), + .about("send a management command to running bioyino server") + .arg(Arg::with_name("host").short("h").default_value( + "127.0.0.1:8137", + )) + .subcommand(SubCommand::with_name("status").about("get server state")) + .subcommand( + SubCommand::with_name("consensus") + .arg(Arg::with_name("action").index(1)) + .arg(Arg::with_name("leader_action").index(2).default_value( + "unchanged", + )), ), - ), - ) + ) .get_matches(); let config = value_t!(app.value_of("config"), String).expect("config file must be string"); let mut file = File::open(&config).expect(&format!("opening config file at {}", &config)); let mut config_str = String::new(); - file.read_to_string(&mut config_str) - .expect("reading config file"); + file.read_to_string(&mut config_str).expect( + "reading config file", + ); let mut system: System = toml::de::from_str(&config_str).expect("parsing config"); if let Some(v) = app.value_of("verbosity") { @@ -280,14 +337,12 @@ impl System { if let Some(_) = query.subcommand_matches("status") { (system, Command::Query(MgmtCommand::Status, server)) } else if let Some(args) = query.subcommand_matches("consensus") { - let c_action = value_t!(args.value_of("action"), ConsensusAction) - .expect("bad consensus action"); - let l_action = value_t!(args.value_of("leader_action"), LeaderAction) - .expect("bad leader action"); + let c_action = value_t!(args.value_of("action"), ConsensusAction).expect("bad consensus action"); + let l_action = value_t!(args.value_of("leader_action"), LeaderAction).expect("bad leader action"); ( system, Command::Query(MgmtCommand::ConsensusCommand(c_action, l_action), server), - ) + ) } else { // shold be unreachable unreachable!("clap bug?") diff --git a/src/consul.rs b/src/consul.rs index 663a32a..3cc8712 100644 --- a/src/consul.rs +++ b/src/consul.rs @@ -1,5 +1,4 @@ use std::net::SocketAddr; -use std::sync::atomic::Ordering; use std::time::{Duration, Instant}; use futures::future::{err, loop_fn, ok, Either, Future, IntoFuture, Loop}; @@ -11,8 +10,9 @@ use mime::WWW_FORM_URLENCODED; use serde_json::{self, from_slice}; use slog::Logger; use tokio::timer::{self, Delay, Interval}; +use util::switch_leader; -use {ConsensusState, CONSENSUS_STATE, IS_LEADER}; +use {ConsensusState, CONSENSUS_STATE}; #[derive(Fail, Debug)] pub enum ConsulError { @@ -26,12 +26,21 @@ pub enum ConsulError { ConnectionTimeout, #[fail(display = "Http error: {}", _0)] - Http(#[cause] hyper::Error), + Http( + #[cause] + hyper::Error + ), #[fail(display = "Parsing response: {}", _0)] - Parsing(#[cause] serde_json::Error), + Parsing( + #[cause] + serde_json::Error + ), #[fail(display = "I/O error {}", _0)] - Io(#[cause] ::std::io::Error), + Io( + #[cause] + ::std::io::Error + ), #[fail(display = "{}", _0)] Renew(String), @@ -110,15 +119,15 @@ impl IntoFuture for ConsulConsensus { let renewlog = log.clone(); /* - let session_retrier = BackoffRetryBuilder { - delay: 1000, - delay_mul: 1f32, - delay_max: 1000, - retries: ::std::usize::MAX, - }; - - let session = session_retrier.spawn(session); - */ + let session_retrier = BackoffRetryBuilder { + delay: 1000, + delay_mul: 1f32, + delay_max: 1000, + retries: ::std::usize::MAX, + }; + + let session = session_retrier.spawn(session); + */ // this tries to reconnect to consul infinitely let loop_session = loop_fn(session, move |session| { let log = log.clone(); @@ -136,10 +145,10 @@ impl IntoFuture for ConsulConsensus { let new_session = new_session.clone(); Box::new( Delay::new(Instant::now() + error_pause) - .then(move |_| Ok(Loop::Continue(new_session))), - ) + .then(move |_| Ok(Loop::Continue(new_session))), + ) as Box, Error = _>> - //ok(Loop::Continue(new_session)) + //ok(Loop::Continue(new_session)) } Ok(None) => { warn!(log, "timed out getting consul session"); @@ -150,8 +159,8 @@ impl IntoFuture for ConsulConsensus { } else { Either::B( Delay::new(Instant::now() + error_pause) - .then(move |_| Ok(Loop::Continue(new_session))), - ) + .then(move |_| Ok(Loop::Continue(new_session))), + ) } }); @@ -234,7 +243,8 @@ impl IntoFuture for ConsulSession { let ttl_ns = ttl.as_secs() * 1_000_000_000u64 + ttl.subsec_nanos() as u64; let b = format!( "{{\"TTL\": \"{}ns\", \"LockDelay\": \"{}ns\"}}", - ttl_ns, ttl_ns + ttl_ns, + ttl_ns ); let bodylen = b.len() as u64; *session_req.body_mut() = hyper::Body::from(b); @@ -242,11 +252,11 @@ impl IntoFuture for ConsulSession { session_req.headers_mut().insert( CONTENT_LENGTH, HeaderValue::from_str(&format!("{}", bodylen)).unwrap(), - ); + ); session_req.headers_mut().insert( CONTENT_TYPE, HeaderValue::from_str(WWW_FORM_URLENCODED.as_str()).unwrap(), - ); + ); let c_session = client .request(session_req) @@ -258,8 +268,7 @@ impl IntoFuture for ConsulSession { .concat2() .map_err(|e| ConsulError::Http(e)) .and_then(move |body| { - let resp: ConsulSessionResponse = - try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); + let resp: ConsulSessionResponse = try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); debug!(log, "new session"; "id"=>format!("{}", resp.id)); Ok(Some(resp.id)) }); @@ -267,13 +276,12 @@ impl IntoFuture for ConsulSession { } else { let body = resp.into_body().concat2().map_err(|e| ConsulError::Http(e)); // TODO make this into option - let sleep = Delay::new(Instant::now() + Duration::from_millis(1000)) - .map_err(|e| ConsulError::Timer(e)); + let sleep = Delay::new(Instant::now() + Duration::from_millis(1000)).map_err(|e| ConsulError::Timer(e)); let future = sleep.join(body).then(move |res| match res { Ok((_, body)) => Err::, _>(ConsulError::HttpStatus( - status, - format!("{:?}", String::from_utf8(body.to_vec())), - )), + status, + format!("{:?}", String::from_utf8(body.to_vec())), + )), Err(e) => Err(e), }); Box::new(future) @@ -318,11 +326,11 @@ impl IntoFuture for ConsulRenew { renew_req.headers_mut().insert( CONTENT_LENGTH, HeaderValue::from_str(&format!("{}", bodylen)).unwrap(), - ); + ); renew_req.headers_mut().insert( CONTENT_TYPE, HeaderValue::from_str(WWW_FORM_URLENCODED.as_str()).unwrap(), - ); + ); let renew_client = hyper::Client::new(); let future = renew_client.request(renew_req).then(move |res| match res { @@ -388,21 +396,21 @@ impl IntoFuture for ConsulAcquire { .concat2() .map_err(|e| ConsulError::Http(e)) .and_then(move |body| { - let acquired: bool = - try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); - - let should_set = { - let state = &*CONSENSUS_STATE.lock().unwrap(); - // only set leader when consensus is enabled - state == &ConsensusState::Enabled - }; - if should_set { - let is_leader = IS_LEADER.load(Ordering::SeqCst); - if is_leader != acquired { - warn!(log, "leader state change: {} -> {}", is_leader, acquired); - } - IS_LEADER.store(acquired, Ordering::SeqCst); - } + let acquired: bool = try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); + + switch_leader(acquired, &log); + // let should_set = { + //let state = &*CONSENSUS_STATE.lock().unwrap(); + //// only set leader when consensus is enabled + //state == &ConsensusState::Enabled + //}; + //if should_set { + //let is_leader = IS_LEADER.load(Ordering::SeqCst); + //if is_leader != acquired { + //warn!(log, "leader state change: {} -> {}", is_leader, acquired); + //} + //IS_LEADER.store(acquired, Ordering::SeqCst); + //} Ok(()) }) }); diff --git a/src/errors.rs b/src/errors.rs index b5b5628..af6061d 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -3,13 +3,22 @@ use std::net::SocketAddr; #[derive(Fail, Debug)] pub enum GeneralError { #[fail(display = "I/O error")] - Io(#[cause] ::std::io::Error), + Io( + #[cause] + ::std::io::Error + ), #[fail(display = "Error when creating timer: {}", _0)] - Timer(#[cause] ::tokio::timer::Error), + Timer( + #[cause] + ::tokio::timer::Error + ), #[fail(display = "getting system time")] - Time(#[cause] ::std::time::SystemTimeError), + Time( + #[cause] + ::std::time::SystemTimeError + ), #[fail(display = "Gave up connecting to {}", _0)] TcpOutOfTries(SocketAddr), diff --git a/src/main.rs b/src/main.rs index 9e516ee..fcb4d47 100644 --- a/src/main.rs +++ b/src/main.rs @@ -9,6 +9,7 @@ extern crate slog; extern crate slog_async; extern crate slog_scope; extern crate slog_term; +extern crate rand; // Options #[macro_use] @@ -30,6 +31,9 @@ extern crate tokio; extern crate tokio_codec; extern crate tokio_io; +// Raft +extern crate raft_tokio; + // Other extern crate combine; extern crate serde; @@ -48,8 +52,10 @@ pub mod metric; pub mod parser; pub mod peer; pub mod server; +pub mod udp; pub mod task; pub mod util; +pub mod raft; pub mod protocol_capnp { include!(concat!(env!("OUT_DIR"), "/schema/protocol_capnp.rs")); @@ -57,8 +63,6 @@ pub mod protocol_capnp { use std::cell::RefCell; use std::collections::HashMap; -use std::io; -use std::net::SocketAddr; use std::str::FromStr; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering, ATOMIC_BOOL_INIT, ATOMIC_USIZE_INIT}; use std::sync::{Arc, Mutex}; @@ -67,18 +71,15 @@ use std::time::{self, Duration, Instant, SystemTime}; use slog::{Drain, Level}; -use bytes::{Bytes, BytesMut}; +use bytes::{Bytes}; use futures::future::{empty, ok}; use futures::sync::mpsc; use futures::{Future, IntoFuture, Stream}; -use tokio::net::UdpSocket; use tokio::runtime::current_thread::Runtime; use tokio::timer::Interval; -use net2::unix::UnixUdpBuilderExt; -use net2::UdpBuilder; -use resolve::resolver; +use udp::{start_sync_udp, start_async_udp}; use carbon::CarbonBackend; use config::{Command, Consul, Metrics, Network, System}; @@ -87,9 +88,9 @@ use errors::GeneralError; use management::{MgmtClient, MgmtServer}; use metric::Metric; use peer::{NativeProtocolServer, NativeProtocolSnapshot}; -use server::StatsdServer; use task::Task; -use util::{AggregateOptions, Aggregator, BackoffRetryBuilder, OwnStats, UpdateCounterOptions}; +use util::{AggregateOptions, Aggregator, BackoffRetryBuilder, OwnStats, UpdateCounterOptions, try_resolve}; +use raft::start_internal_raft; // floating type used all over the code, can be changed to f32, to use less memory at the price of // precision @@ -131,63 +132,55 @@ impl FromStr for ConsensusState { } } +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case")] +pub enum ConsensusKind { + None, + Consul, + Internal, +} + lazy_static! { pub static ref CONSENSUS_STATE: Mutex = - { Mutex::new(ConsensusState::Disabled) }; + { Mutex::new(ConsensusState::Disabled) }; } pub static IS_LEADER: AtomicBool = ATOMIC_BOOL_INIT; -pub fn try_resolve(s: &str) -> SocketAddr { - s.parse().unwrap_or_else(|_| { - // for name that have failed to be parsed we try to resolve it via DNS - let mut split = s.split(':'); - let host = split.next().unwrap(); // Split always has first element - let port = split.next().expect("port not found"); - let port = port.parse().expect("bad port value"); - - let first_ip = resolver::resolve_host(host) - .expect("failed resolving backend name") - .next() - .expect("at least one IP address required"); - SocketAddr::new(first_ip, port) - }) -} - fn main() { let (system, command) = System::load(); let System { verbosity, - network: - Network { - listen, - peer_listen, - mgmt_listen, - bufsize, - multimessage, - mm_packets, - greens, - snum, - nodes, - snapshot_interval, - }, - consul: - Consul { - start_as: consul_start_as, - agent, - session_ttl: consul_session_ttl, - renew_time: consul_renew_time, - key_name: consul_key, - }, - metrics: - Metrics { - // max_metrics, - mut count_updates, - update_counter_prefix, - update_counter_suffix, - update_counter_threshold, - }, + network: Network { + listen, + peer_listen, + mgmt_listen, + bufsize, + multimessage, + mm_packets, + mm_async, + buffer_flush, + greens, + async_sockets, + nodes, + snapshot_interval, + }, + raft, + consul: Consul { + start_as: consul_start_as, + agent, + session_ttl: consul_session_ttl, + renew_time: consul_renew_time, + key_name: consul_key, + }, + metrics: Metrics { + // max_metrics, + mut count_updates, + update_counter_prefix, + update_counter_suffix, + update_counter_threshold, + }, carbon, n_threads, w_threads, @@ -195,6 +188,7 @@ fn main() { task_queue_size, start_as_leader, stats_prefix, + consensus, } = system.clone(); let verbosity = Level::from_str(&verbosity).expect("bad verbosity"); @@ -230,7 +224,10 @@ fn main() { } if count_updates && update_counter_prefix.len() == 0 && update_counter_suffix.len() == 0 { - warn!(rlog, "update counting suffix and prefix are empty, update counting disabled to avoid metric rewriting"); + warn!( + rlog, + "update counting suffix and prefix are empty, update counting disabled to avoid metric rewriting" + ); count_updates = false; } @@ -251,7 +248,7 @@ fn main() { let future = rx.for_each(move |task: Task| ok(task.run())); runtime.block_on(future).expect("worker thread failed"); }) - .expect("starting counting worker thread"); + .expect("starting counting worker thread"); } let stats_prefix = stats_prefix.trim_right_matches(".").to_string(); @@ -271,7 +268,7 @@ fn main() { nodes.clone(), Duration::from_millis(snapshot_interval as u64), &chans, - ).into_future() + ).into_future() .map_err(move |e| { PEER_ERRORS.fetch_add(1, Ordering::Relaxed); info!(snap_err_log, "error sending snapshot";"error"=>format!("{}", e)); @@ -298,27 +295,38 @@ fn main() { // Init leader state before starting backend IS_LEADER.store(start_as_leader, Ordering::SeqCst); - { - let mut con_state = CONSENSUS_STATE.lock().unwrap(); - info!(log, "starting consul consensus"; "initial_state"=>format!("{:?}", con_state)); - *con_state = consul_start_as; - } - let consul_log = rlog.clone(); + let consensus_log = rlog.clone(); - let mut consensus = ConsulConsensus::new(&consul_log, agent, consul_key); - consensus.set_session_ttl(Duration::from_millis(consul_session_ttl as u64)); - consensus.set_renew_time(Duration::from_millis(consul_renew_time as u64)); - runtime.spawn(consensus.into_future().map_err(|_| ())); // TODO errors + match consensus { + ConsensusKind::Internal => { + start_internal_raft(raft, &mut runtime, consensus_log); + } + ConsensusKind::Consul => { + { + let mut con_state = CONSENSUS_STATE.lock().unwrap(); + info!(log, "starting consul consensus"; "initial_state"=>format!("{:?}", con_state)); + *con_state = consul_start_as; + } + + let mut consensus = ConsulConsensus::new(&consensus_log, agent, consul_key); + consensus.set_session_ttl(Duration::from_millis(consul_session_ttl as u64)); + consensus.set_renew_time(Duration::from_millis(consul_renew_time as u64)); + runtime.spawn(consensus.into_future().map_err(|_| ())); // TODO errors + } + ConsensusKind::None => {IS_LEADER.store(true, Ordering::SeqCst);}, + } info!(log, "starting management server"); let m_serv_log = rlog.clone(); let m_serv_err_log = rlog.clone(); let m_server = hyper::Server::bind(&mgmt_listen) - .serve(move || ok::<_, hyper::Error>(MgmtServer::new(m_serv_log.clone(), &mgmt_listen))) - .map_err(move |e| { - warn!(m_serv_err_log, "management server gone with error: {:?}", e); - }); + .serve(move || { + ok::<_, hyper::Error>(MgmtServer::new(m_serv_log.clone(), &mgmt_listen)) + }) + .map_err(move |e| { + warn!(m_serv_err_log, "management server gone with error: {:?}", e); + }); runtime.spawn(m_server); @@ -328,12 +336,13 @@ fn main() { let dur = Duration::from_millis(carbon.interval); let carbon_timer = Interval::new(Instant::now() + dur, dur); - let carbon_timer = carbon_timer - .map_err(|e| GeneralError::Timer(e)) - .for_each(move |_tick| { - let ts = SystemTime::now() - .duration_since(time::UNIX_EPOCH) - .map_err(|e| GeneralError::Time(e))?; + let carbon_timer = carbon_timer.map_err(|e| GeneralError::Timer(e)).for_each( + move |_tick| { + let ts = SystemTime::now().duration_since(time::UNIX_EPOCH).map_err( + |e| { + GeneralError::Time(e) + }, + )?; let backend_addr = try_resolve(&carbon.address); let tchans = tchans.clone(); @@ -376,13 +385,10 @@ fn main() { runtime.spawn(aggregator); let backend = backend_rx - .inspect(|_| { - EGRESS.fetch_add(1, Ordering::Relaxed); - }) + .inspect(|_| { EGRESS.fetch_add(1, Ordering::Relaxed); }) .collect() .and_then(|metrics| { - let backend = - CarbonBackend::new(backend_addr, ts, Arc::new(metrics)); + let backend = CarbonBackend::new(backend_addr, ts, Arc::new(metrics)); let retrier = BackoffRetryBuilder { delay: backend_opts.connect_delay, @@ -404,229 +410,54 @@ fn main() { let aggregator = Aggregator::new(options, tchans, backend_tx).into_future(); runtime .block_on(aggregator.then(|_| Ok::<(), ()>(()))) - .unwrap_or_else( - |e| error!(carbon_log, "Failed to join aggregated metrics"; "error"=>e), - ); + .unwrap_or_else(|e| { + error!(carbon_log, "Failed to join aggregated metrics"; "error"=>e) + }); } }) - .expect("starting thread for sending to graphite"); + .expect("starting thread for sending to graphite"); Ok(()) - }); + }, + ); let tlog = rlog.clone(); runtime.spawn(carbon_timer.map_err(move |e| { warn!(tlog, "error running carbon"; "error"=>e.to_string()); })); - if multimessage { - info!(log, "multimessage enabled, starting in sync UDP mode"); - use std::os::unix::io::AsRawFd; - - // It is crucial for recvmmsg to have one socket per many threads - // to avoid drops because at lease two threads have to work on socket - // simultaneously - let socket = UdpBuilder::new_v4().unwrap(); - socket.reuse_address(true).unwrap(); - socket.reuse_port(true).unwrap(); - let sck = socket.bind(listen).unwrap(); - - for i in 0..n_threads { - let chans = chans.clone(); - let log = rlog.new(o!("source"=>"mudp_thread")); - - let sck = sck.try_clone().unwrap(); - thread::Builder::new() - .name(format!("bioyino_mudp{}", i).into()) - .spawn(move || { - let fd = sck.as_raw_fd(); - let messages = mm_packets; - { - // <--- this limits `use::libc::*` scope - use libc::*; - use std::ptr::null_mut; - - let mut ichans = chans.iter().cycle(); - - // a vector to avoid dropping iovec structures - let mut iovecs = Vec::with_capacity(messages); - - // a vector to avoid dropping message buffers - let mut message_vec = Vec::new(); - - let mut v: Vec = Vec::with_capacity(messages); - for _ in 0..messages { - let mut buf = Vec::with_capacity(bufsize); - buf.resize(bufsize, 0); - - let mut iov = Vec::with_capacity(1); - iov.resize( - 1, - iovec { - iov_base: buf.as_mut_ptr() as *mut c_void, - iov_len: bufsize as size_t, - }, - ); - let m = mmsghdr { - msg_hdr: msghdr { - msg_name: null_mut(), - msg_namelen: 0 as socklen_t, - msg_iov: iov.as_mut_ptr(), - msg_iovlen: iov.len() as size_t, - msg_control: null_mut(), - msg_controllen: 0, - msg_flags: 0, - }, - msg_len: 0, - }; - v.push(m); - iovecs.push(iov); - message_vec.push(buf); - } - - let vp = v.as_mut_ptr(); - let vlen = v.len(); - - // This is the buffer we fill with metrics and periodically send to - // tasks - // To avoid allocations we make it bigger than multimsg message count - // Also, it can be the huge value already allocated here, so for even less - // allocations, we split the filled part and leave the rest for future bytes - let mut b = BytesMut::with_capacity(bufsize * messages * task_queue_size); - - // We cannot count on allocator to allocate a value close to capacity - // it can be much more than that and stall our buffer for too long - // So we set our chunk size ourselves and send buffer when this value - // exhausted, but we only allocate when buffer becomes empty - let mut chunks = task_queue_size as isize; - - loop { - let res = unsafe { - recvmmsg( - fd as c_int, - vp, - vlen as c_uint, - MSG_WAITFORONE, - null_mut(), - ) - }; - - use bytes::BufMut; - if res >= 0 { - let end = res as usize; - - // Check if we can fit all packets into buffer - let mut total_bytes = 0; - for i in 0..end { - total_bytes += v[i].msg_len as usize; - } - // newlines - total_bytes += end - 1; - - // if we cannot, allocate more - if b.remaining_mut() < total_bytes { - b.reserve(bufsize * messages * task_queue_size) - } - - // put packets into buffer - for i in 0..end { - let len = v[i].msg_len as usize; - - b.put(&message_vec[i][0..len]); - chunks -= end as isize; - } - - // when it's time to send bytes, send them - if chunks <= 0 { - let mut chan = ichans.next().unwrap().clone(); - INGRESS.fetch_add(res as usize, Ordering::Relaxed); - chan.try_send(Task::Parse(b.take().freeze())) - .map_err(|_| { - warn!(log, "error sending buffer(queue full?)"); - DROPS.fetch_add(res as usize, Ordering::Relaxed); - }) - .unwrap_or(()); - chunks = task_queue_size as isize; - } - } else { - let errno = unsafe { *__errno_location() }; - if errno == EAGAIN { - } else { - warn!(log, "UDP receive error"; - "code"=> format!("{}",res), - "error"=>format!("{}", io::Error::last_os_error()) - ) - } - } - } - } - }) - .expect("starting multimsg thread"); + // For each thread we create + let mut flush_flags = Arc::new(Vec::new()); + if let Some(flags) = Arc::get_mut(&mut flush_flags) { + for _ in 0..n_threads { + flags.push(AtomicBool::new(false)); } - } else { - info!(log, "multimessage is disabled, starting in async UDP mode"); - // Create a pool of listener sockets - let mut sockets = Vec::new(); - for _ in 0..snum { - let socket = UdpBuilder::new_v4().unwrap(); - socket.reuse_address(true).unwrap(); - socket.reuse_port(true).unwrap(); - let socket = socket.bind(&listen).unwrap(); - sockets.push(socket); - } - - for i in 0..n_threads { - // Each thread gets the clone of a socket pool - let sockets = sockets - .iter() - .map(|s| s.try_clone().unwrap()) - .collect::>(); + } - let chans = chans.clone(); - thread::Builder::new() - .name(format!("bioyino_udp{}", i).into()) - .spawn(move || { - // each thread runs it's own runtime - let mut runtime = Runtime::new().expect("creating runtime for counting worker"); - - // Inside each green thread - for _ in 0..greens { - // start a listener for all sockets - for socket in sockets.iter() { - let buf = BytesMut::with_capacity(task_queue_size * bufsize); - - let mut readbuf = BytesMut::with_capacity(bufsize); - unsafe { readbuf.set_len(bufsize) } - let chans = chans.clone(); - // create UDP listener - let socket = socket.try_clone().expect("cloning socket"); - let socket = - UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()) - .expect("adding socket to event loop"); - - let server = StatsdServer::new( - socket, - chans.clone(), - buf, - task_queue_size, - bufsize, - i, - readbuf, - task_queue_size * bufsize, - ); - - runtime.spawn(server.into_future()); - } - } + if buffer_flush > 0 { + let dur = Duration::from_millis(buffer_flush); + let flush_timer = Interval::new(Instant::now() + dur, dur); + + let tlog = rlog.clone(); + let flags = flush_flags.clone(); + let flush_timer = flush_timer.map_err(|e| GeneralError::Timer(e)).for_each( + move |_tick| { + debug!(tlog, "buffer flush requested"); + flags.iter().map(|flag| flag.swap(true, Ordering::SeqCst)).last(); + Ok(()) + }); + let tlog = rlog.clone(); + runtime.spawn(flush_timer.map_err(move |e| { + warn!(tlog, "error running buffer flush timer"; "error"=>e.to_string()); + })); + } - runtime - .block_on(empty::<(), ()>()) - .expect("starting runtime for async UDP"); - }) - .expect("creating UDP reader thread"); - } + if multimessage { + start_sync_udp(log, listen, &chans, n_threads, bufsize, mm_packets, mm_async, task_queue_size, buffer_flush, flush_flags.clone()); + } else { + start_async_udp(log, listen, &chans, n_threads, greens, async_sockets, bufsize, task_queue_size, flush_flags.clone()); } - runtime - .block_on(empty::<(), ()>()) - .expect("running runtime in main thread"); + runtime.block_on(empty::<(), ()>()).expect( + "running runtime in main thread", + ); } diff --git a/src/management.rs b/src/management.rs index 791ae79..39fbf92 100644 --- a/src/management.rs +++ b/src/management.rs @@ -16,16 +16,28 @@ use {ConsensusState, CONSENSUS_STATE, IS_LEADER}; #[derive(Fail, Debug)] pub enum MgmtError { #[fail(display = "I/O error: {}", _0)] - Io(#[cause] ::std::io::Error), + Io( + #[cause] + ::std::io::Error + ), #[fail(display = "Http error: {}", _0)] - Http(#[cause] hyper::Error), + Http( + #[cause] + hyper::Error + ), #[fail(display = "JSON decoding error {}", _0)] - Decode(#[cause] serde_json::error::Error), + Decode( + #[cause] + serde_json::error::Error + ), #[fail(display = "JSON encoding error: {}", _0)] - Encode(#[cause] serde_json::error::Error), + Encode( + #[cause] + serde_json::error::Error + ), #[fail(display = "bad command")] BadCommand, @@ -118,7 +130,9 @@ pub struct MgmtServer { impl MgmtServer { pub fn new(log: Logger, address: &SocketAddr) -> Self { Self { - log: log.new(o!("source"=>"management-server", "server"=>format!("{}", address))), + log: log.new( + o!("source"=>"management-server", "server"=>format!("{}", address)), + ), } } } @@ -160,9 +174,7 @@ impl Service for MgmtServer { let mut constate = CONSENSUS_STATE.lock().unwrap(); *constate = match consensus_action { - ConsensusAction::Enable | ConsensusAction::Resume => { - ConsensusState::Enabled - } + ConsensusAction::Enable | ConsensusAction::Resume => ConsensusState::Enabled, ConsensusAction::Disable => ConsensusState::Disabled, ConsensusAction::Pause => ConsensusState::Paused, }; @@ -252,9 +264,9 @@ impl IntoFuture for MgmtClient { match command { MgmtCommand::Status => { *req.method_mut() = Method::GET; - *req.uri_mut() = format!("http://{}/status", address) - .parse() - .expect("creating url for management command "); + *req.uri_mut() = format!("http://{}/status", address).parse().expect( + "creating url for management command ", + ); let client = hyper::Client::new(); let clog = log.clone(); @@ -265,25 +277,21 @@ impl IntoFuture for MgmtClient { let body = resp.into_body() .concat2() .map_err(|e| MgmtError::Http(e)) - .map(move |body| { - match serde_json::from_slice::(&*body) { - Ok(status) => { - println!("{:?}", status); - } - Err(e) => { - println!( - "Error parsing server response: {}", - e.to_string() - ); - } + .map(move |body| match serde_json::from_slice::( + &*body, + ) { + Ok(status) => { + println!("{:?}", status); + } + Err(e) => { + println!("Error parsing server response: {}", e.to_string()); } }); Box::new(body) as Box> } else { - Box::new(ok(warn!( - clog, - "Bad status returned from server: {:?}", resp - ))) + Box::new(ok( + warn!(clog, "Bad status returned from server: {:?}", resp), + )) } } }); @@ -291,9 +299,9 @@ impl IntoFuture for MgmtClient { } command @ MgmtCommand::ConsensusCommand(_, _) => { *req.method_mut() = Method::POST; - *req.uri_mut() = format!("http://{}/consensus", address) - .parse() - .expect("creating url for management command"); + *req.uri_mut() = format!("http://{}/consensus", address).parse().expect( + "creating url for management command", + ); let body = serde_json::to_vec_pretty(&command).unwrap(); *req.body_mut() = Body::from(body); @@ -306,25 +314,21 @@ impl IntoFuture for MgmtClient { let body = resp.into_body() .concat2() .map_err(|e| MgmtError::Http(e)) - .map(move |body| { - match serde_json::from_slice::(&*body) { - Ok(status) => { - println!("New server state: {:?}", status); - } - Err(e) => { - println!( - "Error parsing server response: {}", - e.to_string() - ); - } + .map(move |body| match serde_json::from_slice::( + &*body, + ) { + Ok(status) => { + println!("New server state: {:?}", status); + } + Err(e) => { + println!("Error parsing server response: {}", e.to_string()); } }); Box::new(body) as Box> } else { - Box::new(ok(warn!( - clog, - "Bad status returned from server: {:?}", resp - ))) + Box::new(ok( + warn!(clog, "Bad status returned from server: {:?}", resp), + )) } } }); @@ -367,7 +371,9 @@ mod test { let c_serv_err_log = rlog.clone(); let s_addr = address.clone(); let server = hyper::Server::bind(&address) - .serve(move || ok::<_, hyper::Error>(MgmtServer::new(c_serv_log.clone(), &s_addr))) + .serve(move || { + ok::<_, hyper::Error>(MgmtServer::new(c_serv_log.clone(), &s_addr)) + }) .map_err(move |e| { warn!(c_serv_err_log, "management server gone with error: {:?}", e); }); @@ -388,21 +394,19 @@ mod test { // let server some time to settle // then test the status command let d = Delay::new(Instant::now() + Duration::from_secs(1)); - let delayed = d.map_err(|_| ()) - .and_then(move |_| client.into_future().map_err(|e| panic!(e))); + let delayed = d.map_err(|_| ()).and_then(move |_| { + client.into_future().map_err(|e| panic!(e)) + }); runtime.spawn(delayed); // then send a status change command let d = Delay::new(Instant::now() + Duration::from_secs(2)); let delayed = d.map_err(|_| ()).and_then(move |_| { - let command = - MgmtCommand::ConsensusCommand(ConsensusAction::Enable, LeaderAction::Enable); + let command = MgmtCommand::ConsensusCommand(ConsensusAction::Enable, LeaderAction::Enable); let client = MgmtClient::new(log.clone(), address, command); - client - .into_future() - .map_err(|e| panic!("{:?}", e)) - .map(move |_| { + client.into_future().map_err(|e| panic!("{:?}", e)).map( + move |_| { // ensure state has changed let state = ServerStatus::new(); assert_eq!( @@ -412,7 +416,8 @@ mod test { leader_status: true, } ) - }) + }, + ) }); runtime.spawn(delayed); diff --git a/src/metric.rs b/src/metric.rs index 8ff63e6..de165fc 100644 --- a/src/metric.rs +++ b/src/metric.rs @@ -82,27 +82,9 @@ where impl Metric where - F: Add - + AddAssign - + Sub - + SubAssign - + Clone - + Div - + Mul - + PartialOrd - + PartialEq - + Into - + From - + Copy - + Debug - + Sync, + F: Add + AddAssign + Sub + SubAssign + Clone + Div + Mul + PartialOrd + PartialEq + Into + From + Copy + Debug + Sync, { - pub fn new( - value: F, - mtype: MetricType, - timestamp: Option, - sampling: Option, - ) -> Result { + pub fn new(value: F, mtype: MetricType, timestamp: Option, sampling: Option) -> Result { let mut metric = Metric { value, mtype, @@ -165,9 +147,7 @@ where Ok(()) } - pub fn from_capnp<'a>( - reader: cmetric::Reader<'a>, - ) -> Result<(Bytes, Metric), MetricError> { + pub fn from_capnp<'a>(reader: cmetric::Reader<'a>) -> Result<(Bytes, Metric), MetricError> { let name = reader.get_name().map_err(MetricError::Capnp)?.into(); let value = reader.get_value(); @@ -202,10 +182,9 @@ where let (sampling, up_counter) = match reader.get_meta() { Ok(reader) => ( if reader.has_sampling() { - reader - .get_sampling() - .ok() - .map(|reader| reader.get_sampling()) + reader.get_sampling().ok().map( + |reader| reader.get_sampling(), + ) } else { None }, @@ -291,18 +270,7 @@ where impl IntoIterator for Metric where - F: Debug - + Add - + AddAssign - + Sub - + SubAssign - + Div - + Mul - + Clone - + Copy - + PartialOrd - + PartialEq - + Into, + F: Debug + Add + AddAssign + Sub + SubAssign + Div + Mul + Clone + Copy + PartialOrd + PartialEq + Into, { type Item = (&'static str, Float); type IntoIter = MetricIter; @@ -313,18 +281,7 @@ where pub struct MetricIter where - F: Debug - + Add - + AddAssign - + Sub - + SubAssign - + Div - + Mul - + Clone - + Copy - + PartialOrd - + PartialEq - + Into, + F: Debug + Add + AddAssign + Sub + SubAssign + Div + Mul + Clone + Copy + PartialOrd + PartialEq + Into, { m: Metric, count: usize, @@ -334,18 +291,7 @@ where impl MetricIter where - F: Debug - + Add - + AddAssign - + Sub - + SubAssign - + Div - + Mul - + Clone - + Copy - + PartialOrd - + PartialEq - + Into, + F: Debug + Add + AddAssign + Sub + SubAssign + Div + Mul + Clone + Copy + PartialOrd + PartialEq + Into, { fn new(mut metric: Metric) -> Self { let sum = if let MetricType::Timer(ref mut agg) = metric.mtype { @@ -365,18 +311,7 @@ where impl Iterator for MetricIter where - F: Debug - + Add - + AddAssign - + Sub - + SubAssign - + Div - + Mul - + Clone - + Copy - + PartialOrd - + PartialEq - + Into, + F: Debug + Add + AddAssign + Sub + SubAssign + Div + Mul + Clone + Copy + PartialOrd + PartialEq + Into, { type Item = (&'static str, Float); diff --git a/src/parser.rs b/src/parser.rs index 9b74944..d2b0583 100644 --- a/src/parser.rs +++ b/src/parser.rs @@ -13,26 +13,9 @@ use metric::MetricType; // to make his zero-copy and get better errors, parser only recognizes parts // of the metric: (name, value, type, sampling) -pub fn metric_parser<'a, F>( -) -> impl Parser, Option), Input = &'a [u8]> +pub fn metric_parser<'a, F>() -> impl Parser, Option), Input = &'a [u8]> where - F: FromStr - + Add - + AddAssign - + Sub - + SubAssign - + Div - + Mul - + Neg - + PartialOrd - + Into - + From - + Debug - + Default - + Clone - + Copy - + PartialEq - + Sync, +F: FromStr + Add + AddAssign + Sub + SubAssign + Div + Mul + Neg + PartialOrd + Into + From + Debug + Default + Clone + Copy + PartialEq + Sync, { // This will parse metric name and separator let name = take_while1(|c: u8| c != b':' && c != b'\n').skip(byte(b':')); @@ -60,7 +43,9 @@ where let sampling = (bytes(b"|@"), take_while(|c: u8| c != b'\n')).and_then(|(_, value)| { from_utf8(value) .map_err(|_e| UnexpectedParse::Unexpected) - .map(|v| v.parse::().map_err(|_e| UnexpectedParse::Unexpected))? + .map(|v| { + v.parse::().map_err(|_e| UnexpectedParse::Unexpected) + })? }); ( name, @@ -69,7 +54,7 @@ where mtype, optional(sampling), skip_many(newline()).or(eof()), - ).and_then(|(name, sign, mut value, mtype, sampling, _)| { + ).and_then(|(name, sign, mut value, mtype, sampling, _)| { let mtype = if let MetricType::Gauge(_) = mtype { MetricType::Gauge(sign) } else { @@ -86,7 +71,6 @@ where #[cfg(test)] mod tests { - // WARNING: these tests most probably don't work as of now use super::*; // TODO: Questioned cases: diff --git a/src/peer.rs b/src/peer.rs index f7f1388..6867438 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -23,10 +23,16 @@ use {Float, PEER_ERRORS}; #[derive(Fail, Debug)] pub enum PeerError { #[fail(display = "I/O error: {}", _0)] - Io(#[cause] ::std::io::Error), + Io( + #[cause] + ::std::io::Error + ), #[fail(display = "Error when creating timer: {}", _0)] - Timer(#[cause] ::tokio::timer::Error), + Timer( + #[cause] + ::tokio::timer::Error + ), #[fail(display = "error sending task to worker thread")] TaskSend, @@ -60,7 +66,9 @@ pub struct NativeProtocolServer { impl NativeProtocolServer { pub fn new(log: Logger, listen: SocketAddr, chans: Vec>) -> Self { Self { - log: log.new(o!("source"=>"canproto-peer-server", "ip"=>format!("{}", listen.clone()))), + log: log.new( + o!("source"=>"canproto-peer-server", "ip"=>format!("{}", listen.clone())), + ), listen, chans: chans, } @@ -79,9 +87,10 @@ impl IntoFuture for NativeProtocolServer { .incoming() .map_err(|e| PeerError::Io(e)) .for_each(move |conn| { - let peer_addr = conn.peer_addr() - .map(|addr| addr.to_string()) - .unwrap_or("[UNCONNECTED]".into()); + let peer_addr = conn.peer_addr().map(|addr| addr.to_string()).unwrap_or( + "[UNCONNECTED]" + .into(), + ); let transport = ReadStream::new(conn, ReaderOptions::default()); let log = log.new(o!("remote"=>peer_addr)); @@ -110,11 +119,7 @@ impl IntoFuture for NativeProtocolServer { } } -fn parse_and_send( - reader: cmsg::Reader, - next_chan: Sender, - log: Logger, -) -> Result<(), MetricError> { +fn parse_and_send(reader: cmsg::Reader, next_chan: Sender, log: Logger) -> Result<(), MetricError> { match reader.which().map_err(MetricError::CapnpSchema)? { cmsg::Single(reader) => { let reader = reader.map_err(MetricError::Capnp)?; @@ -135,8 +140,7 @@ fn parse_and_send( reader .iter() .map(|reader| { - Metric::::from_capnp(reader) - .map(|(name, metric)| metrics.push((name, metric))) + Metric::::from_capnp(reader).map(|(name, metric)| metrics.push((name, metric))) }) .last(); let future = next_chan @@ -155,8 +159,7 @@ fn parse_and_send( reader .iter() .map(|reader| { - Metric::::from_capnp(reader) - .map(|(name, metric)| metrics.push((name, metric))) + Metric::::from_capnp(reader).map(|(name, metric)| metrics.push((name, metric))) }) .last(); let future = next_chan @@ -180,12 +183,7 @@ pub struct NativeProtocolSnapshot { } impl NativeProtocolSnapshot { - pub fn new( - log: &Logger, - nodes: Vec, - interval: Duration, - chans: &Vec>, - ) -> Self { + pub fn new(log: &Logger, nodes: Vec, interval: Duration, chans: &Vec>) -> Self { Self { log: log.new(o!("source"=>"peer-client")), nodes, @@ -248,26 +246,19 @@ impl IntoFuture for NativeProtocolSnapshot { TcpStream::connect(&address) .map_err(|e| PeerError::Io(e)) .and_then(move |conn| { - let codec = ::capnp_futures::serialize::Transport::new( - conn, - ReaderOptions::default(), - ); + let codec = ::capnp_futures::serialize::Transport::new(conn, ReaderOptions::default()); let mut snapshot_message = Builder::new_default(); { - let builder = snapshot_message - .init_root::<::protocol_capnp::message::Builder>(); - let mut multi_metric = - builder.init_snapshot(metrics.len() as u32); + let builder = snapshot_message.init_root::<::protocol_capnp::message::Builder>(); + let mut multi_metric = builder.init_snapshot(metrics.len() as u32); metrics .into_iter() .flat_map(|hmap| hmap.into_iter()) .enumerate() .map(|(idx, (name, metric))| { - let mut c_metric = - multi_metric.reborrow().get(idx as u32); - let name = - unsafe { ::std::str::from_utf8_unchecked(&name) }; + let mut c_metric = multi_metric.reborrow().get(idx as u32); + let name = unsafe { ::std::str::from_utf8_unchecked(&name) }; c_metric.set_name(name); metric.fill_capnp(&mut c_metric); }) @@ -363,8 +354,7 @@ mod test { println!("connection err: {:?}", e); }) .and_then(move |conn| { - let codec = - ::capnp_futures::serialize::Transport::new(conn, ReaderOptions::default()); + let codec = ::capnp_futures::serialize::Transport::new(conn, ReaderOptions::default()); let mut single_message = Builder::new_default(); { @@ -385,8 +375,7 @@ mod test { let mut snapshot_message = Builder::new_default(); { - let builder = - snapshot_message.init_root::<::protocol_capnp::message::Builder>(); + let builder = snapshot_message.init_root::<::protocol_capnp::message::Builder>(); let multi_metric = builder.init_snapshot(1); let mut new_metric = multi_metric.get(0); new_metric.set_name("complex.test.bioyino_snapshot"); @@ -395,9 +384,9 @@ mod test { codec .send(single_message) .and_then(|codec| { - codec - .send(multi_message) - .and_then(|codec| codec.send(snapshot_message)) + codec.send(multi_message).and_then(|codec| { + codec.send(snapshot_message) + }) }) .map(|_| ()) .map_err(|e| println!("codec error: {:?}", e)) diff --git a/src/raft.rs b/src/raft.rs new file mode 100644 index 0000000..860910b --- /dev/null +++ b/src/raft.rs @@ -0,0 +1,76 @@ +use std::collections::HashMap; +use rand::random; + +//use slog::{Drain, Level, Logger}; +use slog::{Logger}; + +use futures::future::lazy; +use tokio::runtime::current_thread::Runtime; + +use raft_tokio::raft_consensus::persistent_log::mem::MemLog; +use raft_tokio::raft_consensus::state::ConsensusState; +use raft_tokio::raft_consensus::state_machine::null::NullStateMachine; +//use raft_tokio::raft_consensus::ServerId; + +////use raft_tokio::raft::RaftPeerProtocol; +use raft_tokio::start_raft_tcp; +use raft_tokio::{Notifier}; +use util::{switch_leader, try_resolve, get_hostname}; +use config::Raft; + +pub struct LeaderNotifier(Logger); + +impl Notifier for LeaderNotifier { + fn state_changed(&mut self, old: ConsensusState, new: ConsensusState) { + if old != new { + if new == ConsensusState::Leader { + switch_leader(true, &self.0) + } else if old == ConsensusState::Leader { + switch_leader(false, &self.0) + } + } + } +} + +pub(crate) fn start_internal_raft(options: Raft, runtime: &mut Runtime, logger: Logger) { + let this = if let Some(name) = options.this_node.clone() { + try_resolve(&name) + } else { + let hostname = get_hostname().expect("getting own hostname") + ":8138"; + try_resolve(&hostname) + }; + + let mut this_id = None; + if options.nodes.len() < 3 { + warn!(logger, "raft requires at least 3 nodes, this may work not as intended"); + } + + // resolve nodes and generate random ServerId + let mut nodes = options.nodes.iter().map(|node| { + let id = random::().into(); + let addr = try_resolve(node); + if addr == this { + this_id = Some(id) + } + (id, addr) + }).collect::>(); + + //let id = this_id/.expect("list of nodes must contain own hostname"); + use raft_tokio::raft_consensus::ServerId; + let id = this_id.unwrap_or_else(|| { + let id: ServerId = random::().into(); + nodes.insert(id, this); + id + }); + // prepare consensus + let raft_log = MemLog::new(); + let sm = NullStateMachine; + let notifier = LeaderNotifier(logger.clone()); + let options = options.get_raft_options(); + + // Create the runtime + let raft = lazy(move || {start_raft_tcp(id, nodes, raft_log, sm, notifier, options, logger); Ok(())}); + + runtime.spawn(raft); + +} diff --git a/src/server.rs b/src/server.rs index 76bf592..6d02993 100644 --- a/src/server.rs +++ b/src/server.rs @@ -1,4 +1,5 @@ -use std::sync::atomic::Ordering; +use std::sync::atomic::{Ordering, AtomicBool}; +use std::sync::Arc; use {DROPS, INGRESS}; @@ -20,19 +21,22 @@ pub struct StatsdServer { next: usize, readbuf: BytesMut, chunks: usize, + flush_flags: Arc>, + thread_idx: usize, } impl StatsdServer { - pub fn new( - socket: UdpSocket, - chans: Vec>, - buf: BytesMut, - buf_queue_size: usize, - bufsize: usize, - next: usize, - readbuf: BytesMut, - chunks: usize, - ) -> Self { + pub fn new(socket: UdpSocket, + chans: Vec>, + buf: BytesMut, + buf_queue_size: usize, + bufsize: usize, + next: usize, + readbuf: BytesMut, + chunks: usize, + flush_flags: Arc>, + thread_idx: usize, + ) -> Self { Self { socket, chans, @@ -42,6 +46,8 @@ impl StatsdServer { next, readbuf, chunks, + flush_flags, + thread_idx } } } @@ -61,6 +67,8 @@ impl IntoFuture for StatsdServer { next, readbuf, chunks, + flush_flags, + thread_idx } = self; let future = socket @@ -74,7 +82,9 @@ impl IntoFuture for StatsdServer { buf.put(&received[0..size]); - if buf.remaining_mut() < bufsize || chunks == 0 { + + let flush = flush_flags.get(thread_idx).unwrap().swap(false, Ordering::SeqCst); + if buf.remaining_mut() < bufsize || chunks == 0 || flush { let (chan, next) = if next >= chans.len() { (chans[0].clone(), 1) } else { @@ -84,22 +94,22 @@ impl IntoFuture for StatsdServer { spawn( chan.send(Task::Parse(buf.freeze())) - .map_err(|_| { - DROPS.fetch_add(1, Ordering::Relaxed); - }) - .and_then(move |_| { - StatsdServer::new( - socket, - chans, - newbuf, - buf_queue_size, - bufsize, - next, - received, - buf_queue_size * bufsize, - ).into_future() - }), - ); + .map_err(|_| { DROPS.fetch_add(1, Ordering::Relaxed); }) + .and_then(move |_| { + StatsdServer::new( + socket, + chans, + newbuf, + buf_queue_size, + bufsize, + next, + received, + buf_queue_size * bufsize, + flush_flags, + thread_idx + ).into_future() + }), + ); } else { spawn( StatsdServer::new( @@ -111,8 +121,10 @@ impl IntoFuture for StatsdServer { next, received, chunks - 1, + flush_flags, + thread_idx ).into_future(), - ); + ); } Ok(()) }); diff --git a/src/task.rs b/src/task.rs index 39b3475..d75bb6c 100644 --- a/src/task.rs +++ b/src/task.rs @@ -11,10 +11,7 @@ use metric::Metric; use parser::metric_parser; use util::AggregateOptions; -use { - Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, LONG_CACHE, PARSE_ERRORS, PEER_ERRORS, - SHORT_CACHE, -}; +use {Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, LONG_CACHE, PARSE_ERRORS, PEER_ERRORS, SHORT_CACHE}; #[derive(Debug)] pub struct AggregateData { @@ -53,23 +50,29 @@ impl Task { pub fn run(self) { match self { Task::Parse(buf) => parse_and_insert(buf), - Task::AddMetric(name, metric) => SHORT_CACHE.with(move |c| { - let mut short = c.borrow_mut(); - update_metric(&mut short, name, metric); - }), - Task::AddMetrics(mut list) => SHORT_CACHE.with(move |c| { - let mut short = c.borrow_mut(); - list.drain(..) - .map(|(name, metric)| update_metric(&mut short, name, metric)) - .last(); - }), - Task::AddSnapshot(mut list) => LONG_CACHE.with(move |c| { - // snapshots go to long cache to avoid being duplicated to other nodes - let mut long = c.borrow_mut(); - list.drain(..) - .map(|(name, metric)| update_metric(&mut long, name, metric)) - .last(); - }), + Task::AddMetric(name, metric) => { + SHORT_CACHE.with(move |c| { + let mut short = c.borrow_mut(); + update_metric(&mut short, name, metric); + }) + } + Task::AddMetrics(mut list) => { + SHORT_CACHE.with(move |c| { + let mut short = c.borrow_mut(); + list.drain(..) + .map(|(name, metric)| update_metric(&mut short, name, metric)) + .last(); + }) + } + Task::AddSnapshot(mut list) => { + LONG_CACHE.with(move |c| { + // snapshots go to long cache to avoid being duplicated to other nodes + let mut long = c.borrow_mut(); + list.drain(..) + .map(|(name, metric)| update_metric(&mut long, name, metric)) + .last(); + }) + } Task::TakeSnapshot(channel) => { let mut short = SHORT_CACHE.with(|c| { let short = c.borrow().clone(); @@ -103,12 +106,12 @@ impl Task { }); } Task::Aggregate(AggregateData { - mut buf, - name, - metric, - options, - mut response, - }) => { + mut buf, + name, + metric, + options, + mut response, + }) => { let upd = if let Some(options) = options.update_counter { if metric.update_counter > options.threshold { // + 2 is for dots @@ -146,18 +149,14 @@ impl Task { .map(|data| { response .start_send(data) - .map_err(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }) + .map_err(|_| { AGG_ERRORS.fetch_add(1, Ordering::Relaxed); }) .map(|_| ()) .unwrap_or(()); }) .last(); response .poll_complete() - .map_err(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }) + .map_err(|_| { AGG_ERRORS.fetch_add(1, Ordering::Relaxed); }) .map(|_| ()) .unwrap_or_else(|_| ()); } diff --git a/src/udp.rs b/src/udp.rs new file mode 100644 index 0000000..7fee023 --- /dev/null +++ b/src/udp.rs @@ -0,0 +1,256 @@ +use std::net::SocketAddr; +use std::io; +use std::thread; +use std::sync::atomic::{Ordering, AtomicBool}; +use std::sync::Arc; + +use std::ptr::null_mut; + +use {INGRESS, DROPS}; +use task::Task; +use server::StatsdServer; + +use tokio::net::UdpSocket; +use std::os::unix::io::AsRawFd; +use tokio::runtime::current_thread::Runtime; +use bytes::{BytesMut, BufMut}; +use futures::IntoFuture; +use futures::future::empty; +use futures::sync::mpsc::Sender; +use net2::unix::UnixUdpBuilderExt; +use net2::UdpBuilder; +use slog::Logger; + + +pub(crate) fn start_sync_udp(log: Logger, + listen: SocketAddr, + chans: &Vec>, + n_threads: usize, + bufsize: usize, + mm_packets: usize, + mm_async: bool, + task_queue_size: usize, + buffer_flush: u64, + flush_flags: Arc>) +{ + info!(log, "multimessage enabled, starting in sync UDP mode"; "socket-is-blocking"=>mm_async, "packets"=>mm_packets); + + // It is crucial for recvmmsg to have one socket per many threads + // to avoid drops because at lease two threads have to work on socket + // simultaneously + let socket = UdpBuilder::new_v4().unwrap(); + socket.reuse_address(true).unwrap(); + socket.reuse_port(true).unwrap(); + let sck = socket.bind(listen).unwrap(); + sck.set_nonblocking(mm_async).unwrap(); + + for i in 0..n_threads { + let chans = chans.clone(); + let log = log.new(o!("source"=>"mudp_thread")); + + let sck = sck.try_clone().unwrap(); + let flush_flags = flush_flags.clone(); + thread::Builder::new() + .name(format!("bioyino_mudp{}", i).into()) + .spawn(move || { + let fd = sck.as_raw_fd(); + { + // <--- this limits `use::libc::*` scope + use libc::*; + + let mut ichans = chans.iter().cycle(); + + // a vector to avoid dropping iovec structures + let mut iovecs = Vec::with_capacity(mm_packets); + + // a vector to avoid dropping message buffers + let mut message_vec = Vec::new(); + + let mut v: Vec = Vec::with_capacity(mm_packets); + for _ in 0..mm_packets { + let mut buf = Vec::with_capacity(bufsize); + buf.resize(bufsize, 0); + + let mut iov = Vec::with_capacity(1); + iov.resize( + 1, + iovec { + iov_base: buf.as_mut_ptr() as *mut c_void, + iov_len: bufsize as size_t, + }, + ); + let m = mmsghdr { + msg_hdr: msghdr { + msg_name: null_mut(), + msg_namelen: 0 as socklen_t, + msg_iov: iov.as_mut_ptr(), + msg_iovlen: iov.len() as size_t, + msg_control: null_mut(), + msg_controllen: 0, + msg_flags: 0, + }, + msg_len: 0, + }; + v.push(m); + iovecs.push(iov); + message_vec.push(buf); + } + + let vp = v.as_mut_ptr(); + let vlen = v.len(); + + // This is the buffer we fill with metrics and periodically send to + // tasks + // To avoid allocations we make it bigger than multimsg message count + // Also, it can be the huge value already allocated here, so for even less + // allocations, we split the filled part and leave the rest for future bytes + let mut b = BytesMut::with_capacity(bufsize * mm_packets * task_queue_size); + + // We cannot count on allocator to allocate a value close to capacity + // it can be much more than that and stall our buffer for too long + // So we set our chunk size ourselves and send buffer when this value + // exhausted, but we only allocate when buffer becomes empty + let mut chunks = task_queue_size as isize; + let flags = if mm_async { + MSG_WAITFORONE + } else { + 0 + }; + loop { + let timeout = if buffer_flush > 0 { + &mut timespec { + tv_sec: buffer_flush as i64 / 1000, + tv_nsec: (buffer_flush as i64 % 1000) * 1000000, + } + } else { + null_mut() + }; + + let res = unsafe { recvmmsg(fd as c_int, vp, vlen as c_uint, flags, timeout) }; + + if res >= 0 { + let end = res as usize; + + // Check if we can fit all packets into buffer + let mut total_bytes = 0; + for i in 0..end { + total_bytes += v[i].msg_len as usize; + } + // newlines + total_bytes += end - 1; + + // if we cannot, allocate more + if b.remaining_mut() < total_bytes { + b.reserve(bufsize * mm_packets * task_queue_size) + } + + // put packets into buffer + for i in 0..end { + let len = v[i].msg_len as usize; + + b.put(&message_vec[i][0..len]); + chunks -= end as isize; + } + + // when it's time to send bytes, send them + let flush = flush_flags.get(i).unwrap().swap(false, Ordering::SeqCst); + if chunks <= 0 || flush { + let mut chan = ichans.next().unwrap().clone(); + INGRESS.fetch_add(res as usize, Ordering::Relaxed); + chan.try_send(Task::Parse(b.take().freeze())) + .map_err(|_| { + warn!(log, "error sending buffer(queue full?)"); + DROPS.fetch_add(res as usize, Ordering::Relaxed); + }) + .unwrap_or(()); + chunks = task_queue_size as isize; + } + } else { + let errno = unsafe { *__errno_location() }; + if errno == EAGAIN { + } else { + warn!(log, "UDP receive error"; + "code"=> format!("{}",res), + "error"=>format!("{}", io::Error::last_os_error()) + ) + } + } + } + } + }) + .expect("starting multimsg thread"); + } +} + +pub(crate) fn start_async_udp(log: Logger, + listen: SocketAddr, + chans: &Vec>, + n_threads: usize, + greens: usize, + async_sockets: usize, + bufsize: usize, + task_queue_size: usize, + flush_flags: Arc>) { + info!(log, "multimessage is disabled, starting in async UDP mode"); + // Create a pool of listener sockets + let mut sockets = Vec::new(); + for _ in 0..async_sockets { + let socket = UdpBuilder::new_v4().unwrap(); + socket.reuse_address(true).unwrap(); + socket.reuse_port(true).unwrap(); + let socket = socket.bind(&listen).unwrap(); + sockets.push(socket); + } + + for i in 0..n_threads { + // Each thread gets the clone of a socket pool + let sockets = sockets + .iter() + .map(|s| s.try_clone().unwrap()) + .collect::>(); + + let chans = chans.clone(); + let flush_flags = flush_flags.clone(); + thread::Builder::new() + .name(format!("bioyino_udp{}", i).into()) + .spawn(move || { + // each thread runs it's own runtime + let mut runtime = Runtime::new().expect("creating runtime for counting worker"); + + // Inside each green thread + for _ in 0..greens { + // start a listener for all sockets + for socket in sockets.iter() { + let buf = BytesMut::with_capacity(task_queue_size * bufsize); + + let mut readbuf = BytesMut::with_capacity(bufsize); + unsafe { readbuf.set_len(bufsize) } + let chans = chans.clone(); + // create UDP listener + let socket = socket.try_clone().expect("cloning socket"); + let socket = UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()).expect("adding socket to event loop"); + + let server = StatsdServer::new( + socket, + chans.clone(), + buf, + task_queue_size, + bufsize, + i, + readbuf, + task_queue_size * bufsize, + flush_flags.clone(), + i + ); + + runtime.spawn(server.into_future()); + } + } + + runtime.block_on(empty::<(), ()>()).expect( + "starting runtime for async UDP", + ); + }) + .expect("creating UDP reader thread"); + } +} diff --git a/src/util.rs b/src/util.rs index b7fb36c..bc14faa 100644 --- a/src/util.rs +++ b/src/util.rs @@ -1,6 +1,9 @@ use std::collections::HashMap; use std::sync::atomic::Ordering; use std::time::{Duration, Instant}; +use std::ffi::CStr; +use std::net::SocketAddr; +use libc; use bytes::{BufMut, Bytes, BytesMut}; use futures::future::Either; @@ -8,13 +11,62 @@ use futures::stream::futures_unordered; use futures::sync::mpsc::{Sender, UnboundedSender}; use futures::sync::oneshot; use futures::{Async, Future, IntoFuture, Poll, Sink, Stream}; +use resolve::resolver; use slog::Logger; use tokio::executor::current_thread::spawn; use tokio::timer::{Delay, Interval}; use metric::{Metric, MetricType}; use task::{AggregateData, Task}; -use {Cache, Float, AGG_ERRORS, DROPS, EGRESS, INGRESS, INGRESS_METRICS, PARSE_ERRORS, PEER_ERRORS}; +use {Cache, Float}; +use {AGG_ERRORS, DROPS, EGRESS, INGRESS, INGRESS_METRICS, PARSE_ERRORS, PEER_ERRORS}; + +use {ConsensusState, CONSENSUS_STATE, IS_LEADER}; + +pub fn try_resolve(s: &str) -> SocketAddr { + s.parse().unwrap_or_else(|_| { + // for name that have failed to be parsed we try to resolve it via DNS + let mut split = s.split(':'); + let host = split.next().unwrap(); // Split always has first element + let port = split.next().expect("port not found"); + let port = port.parse().expect("bad port value"); + + let first_ip = resolver::resolve_host(host) + .expect("failed resolving backend name") + .next() + .expect("at least one IP address required"); + SocketAddr::new(first_ip, port) + }) +} + +/// Get hostname. Copypyasted from some crate +pub fn get_hostname() -> Option { + let len = 255; + let mut buf = Vec::::with_capacity(len); + let ptr = buf.as_mut_ptr() as *mut libc::c_char; + + unsafe { + if libc::gethostname(ptr, len as libc::size_t) != 0 { + return None; + } + Some(CStr::from_ptr(ptr).to_string_lossy().into_owned()) + } +} + +pub fn switch_leader(acquired: bool, log: &Logger) { + let should_set = { + let state = &*CONSENSUS_STATE.lock().unwrap(); + // only set leader when consensus is enabled + state == &ConsensusState::Enabled + }; + if should_set { + let is_leader = IS_LEADER.load(Ordering::SeqCst); + if is_leader != acquired { + warn!(log, "leader state change: {} -> {}", is_leader, acquired); + } + IS_LEADER.store(acquired, Ordering::SeqCst); + } +} // A future to send own stats. Never gets ready. pub struct OwnStats { @@ -127,11 +179,7 @@ pub struct Aggregator { } impl Aggregator { - pub fn new( - options: AggregateOptions, - chans: Vec>, - tx: UnboundedSender<(Bytes, Float)>, - ) -> Self { + pub fn new(options: AggregateOptions, chans: Vec>, tx: UnboundedSender<(Bytes, Float)>) -> Self { Self { options, chans, tx } } } @@ -148,9 +196,9 @@ impl IntoFuture for Aggregator { // TODO: change oneshots to single channel // to do that, task must run in new tokio, then we will not have to pass handle to it //handle.spawn(chan.send(Task::Rotate(tx)).then(|_| Ok(()))); - chan.send(Task::Rotate(tx)) - .map_err(|_| ()) - .and_then(|_| rx.and_then(|m| Ok(m)).map_err(|_| ())) + chan.send(Task::Rotate(tx)).map_err(|_| ()).and_then(|_| { + rx.and_then(|m| Ok(m)).map_err(|_| ()) + }) }); if options.is_leader { @@ -171,35 +219,33 @@ impl IntoFuture for Aggregator { // }) }); - let aggregate = accumulate.and_then(move |accumulated| { - accumulated - .into_iter() - .inspect(|_| { - EGRESS.fetch_add(1, Ordering::Relaxed); - }) - .map(move |(name, metric)| { - let buf = BytesMut::with_capacity(1024); - let task = Task::Aggregate(AggregateData { - buf, - name: Bytes::from(name), - metric, - options: options.clone(), - response: tx.clone(), - }); - // as of now we just run each task in the current thread - // there is a reason we should not in general run the task in the counting workers: - // workers will block on heavy computation and may cause metrics goind to them over - // network to be dropped because of backpressure - // at the same time counting aggregation is not urgent because of current backend(carbon/graphite) - // nature where one can send metrics with any timestamp - // TODO: at some day counting workers will probably work in work-stealing mode, - // after that we probably will be able to run task in common mode - task.run(); - }) - .last(); - Ok(()) - }); - Box::new(aggregate) + let aggregate = accumulate.and_then(move |accumulated| { + accumulated + .into_iter() + .inspect(|_| { EGRESS.fetch_add(1, Ordering::Relaxed); }) + .map(move |(name, metric)| { + let buf = BytesMut::with_capacity(1024); + let task = Task::Aggregate(AggregateData { + buf, + name: Bytes::from(name), + metric, + options: options.clone(), + response: tx.clone(), + }); + // as of now we just run each task in the current thread + // there is a reason we should not in general run the task in the counting workers: + // workers will block on heavy computation and may cause metrics goind to them over + // network to be dropped because of backpressure + // at the same time counting aggregation is not urgent because of current backend(carbon/graphite) + // nature where one can send metrics with any timestamp + // TODO: at some day counting workers will probably work in work-stealing mode, + // after that we probably will be able to run task in common mode + task.run(); + }) + .last(); + Ok(()) + }); + Box::new(aggregate) } else { // only get metrics from threads let not_leader = futures_unordered(metrics).for_each(|_| Ok(())); @@ -229,16 +275,16 @@ impl Default for BackoffRetryBuilder { impl BackoffRetryBuilder { pub fn spawn(self, action: F) -> BackoffRetry - where + where F: IntoFuture + Clone, - { - let inner = Either::A(action.clone().into_future()); - BackoffRetry { - action, - inner: inner, - options: self, + { + let inner = Either::A(action.clone().into_future()); + BackoffRetry { + action, + inner: inner, + options: self, + } } - } } /// TCP client that is able to reconnect with customizable settings @@ -250,7 +296,7 @@ pub struct BackoffRetry { impl Future for BackoffRetry where - F: IntoFuture + Clone, +F: IntoFuture + Clone, { type Item = F::Item; type Error = Option; @@ -259,25 +305,29 @@ where loop { let (rotate_f, rotate_t) = match self.inner { // we are polling a future currently - Either::A(ref mut future) => match future.poll() { - Ok(Async::Ready(item)) => { - return Ok(Async::Ready(item)); - } - Ok(Async::NotReady) => return Ok(Async::NotReady), - Err(e) => { - if self.options.retries == 0 { - return Err(Some(e)); - } else { - (true, false) + Either::A(ref mut future) => { + match future.poll() { + Ok(Async::Ready(item)) => { + return Ok(Async::Ready(item)); + } + Ok(Async::NotReady) => return Ok(Async::NotReady), + Err(e) => { + if self.options.retries == 0 { + return Err(Some(e)); + } else { + (true, false) + } } } - }, - Either::B(ref mut timer) => match timer.poll() { - // we are waiting for the delay - Ok(Async::Ready(())) => (false, true), - Ok(Async::NotReady) => return Ok(Async::NotReady), - Err(_) => unreachable!(), // timer should not return error - }, + } + Either::B(ref mut timer) => { + match timer.poll() { + // we are waiting for the delay + Ok(Async::Ready(())) => (false, true), + Ok(Async::NotReady) => return Ok(Async::NotReady), + Err(_) => unreachable!(), // timer should not return error + } + } }; if rotate_f { From 3fe91d97d70cb28fa28c18817eb3ea3b421a2e1f Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Fri, 5 Oct 2018 15:42:01 +0300 Subject: [PATCH 08/24] Fix bug in snapshot client, remove tags for now --- schema/protocol.capnp | 2 +- src/carbon.rs | 36 ++++++----- src/config.rs | 75 +++++++++++----------- src/consul.rs | 63 +++++++++--------- src/errors.rs | 15 +---- src/main.rs | 146 +++++++++++++++++++++++++----------------- src/management.rs | 121 +++++++++++++++++----------------- src/metric.rs | 88 +++++++++++++++++++++---- src/parser.rs | 50 ++++++++++----- src/peer.rs | 81 ++++++++++++----------- src/raft.rs | 40 +++++++----- src/server.rs | 72 +++++++++++---------- src/task.rs | 57 ++++++++--------- src/udp.rs | 100 ++++++++++++++--------------- src/util.rs | 71 ++++++++++---------- 15 files changed, 566 insertions(+), 451 deletions(-) diff --git a/schema/protocol.capnp b/schema/protocol.capnp index 4920fc8..d0c3913 100644 --- a/schema/protocol.capnp +++ b/schema/protocol.capnp @@ -72,7 +72,7 @@ struct Gauge { struct MetricMeta { sampling @0 :Sampling; updateCounter @1 :UInt32; - tags @2 :List(Tag); +# tags @2 :List(Tag); } struct Sampling { diff --git a/src/carbon.rs b/src/carbon.rs index 47764ab..e0324f5 100644 --- a/src/carbon.rs +++ b/src/carbon.rs @@ -27,23 +27,25 @@ impl CarbonBackend { let ts: Bytes = ts.as_secs().to_string().into(); let buf = BytesMut::with_capacity(metrics.len() * 200); // 200 is an approximate for full metric name + value - let (metrics, _) = metrics.iter().fold((Vec::new(), buf), |(mut acc, mut buf), - (name, metric)| { - let mut wr = buf.writer(); - let buf = match ftoa::write(&mut wr, *metric) { - Ok(()) => { - buf = wr.into_inner(); - let metric = buf.take().freeze(); - acc.push((name.clone(), metric, ts.clone())); - buf - } - Err(_) => { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - wr.into_inner() - } - }; - (acc, buf) - }); + let (metrics, _) = + metrics + .iter() + .fold((Vec::new(), buf), |(mut acc, mut buf), (name, metric)| { + let mut wr = buf.writer(); + let buf = match ftoa::write(&mut wr, *metric) { + Ok(()) => { + buf = wr.into_inner(); + let metric = buf.take().freeze(); + acc.push((name.clone(), metric, ts.clone())); + buf + } + Err(_) => { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + wr.into_inner() + } + }; + (acc, buf) + }); let metrics = Arc::new(metrics); let self_ = Self { addr, metrics }; self_ diff --git a/src/config.rs b/src/config.rs index 8a3403d..ffbcacb 100644 --- a/src/config.rs +++ b/src/config.rs @@ -1,8 +1,8 @@ use std::fs::File; use std::io::Read; use std::net::SocketAddr; -use std::time::Duration; use std::ops::Range; +use std::time::Duration; use clap::{Arg, SubCommand}; use toml; @@ -10,7 +10,7 @@ use toml; use management::{ConsensusAction, LeaderAction, MgmtCommand}; use raft_tokio::RaftOptions; -use {ConsensusState, ConsensusKind}; +use {ConsensusKind, ConsensusState}; #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] @@ -272,8 +272,10 @@ impl Raft { pub fn get_raft_options(&self) -> RaftOptions { RaftOptions { heartbeat_timeout: Duration::from_millis(self.heartbeat_timeout), - election_timeout: Range{start: Duration::from_millis(self.election_timeout_min), end: Duration::from_millis(self.election_timeout_max)}, - + election_timeout: Range { + start: Duration::from_millis(self.election_timeout_min), + end: Duration::from_millis(self.election_timeout_max), + }, } } } @@ -290,42 +292,41 @@ impl System { let app = app_from_crate!() .arg( Arg::with_name("config") - .help("configuration file path") - .long("config") - .short("c") - .required(true) - .takes_value(true) - .default_value("/etc/bioyino/bioyino.toml"), - ) - .arg( + .help("configuration file path") + .long("config") + .short("c") + .required(true) + .takes_value(true) + .default_value("/etc/bioyino/bioyino.toml"), + ).arg( Arg::with_name("verbosity") - .short("v") - .help("logging level") - .takes_value(true), - ) - .subcommand( + .short("v") + .help("logging level") + .takes_value(true), + ).subcommand( SubCommand::with_name("query") - .about("send a management command to running bioyino server") - .arg(Arg::with_name("host").short("h").default_value( - "127.0.0.1:8137", - )) - .subcommand(SubCommand::with_name("status").about("get server state")) - .subcommand( - SubCommand::with_name("consensus") - .arg(Arg::with_name("action").index(1)) - .arg(Arg::with_name("leader_action").index(2).default_value( - "unchanged", - )), + .about("send a management command to running bioyino server") + .arg( + Arg::with_name("host") + .short("h") + .default_value("127.0.0.1:8137"), + ).subcommand(SubCommand::with_name("status").about("get server state")) + .subcommand( + SubCommand::with_name("consensus") + .arg(Arg::with_name("action").index(1)) + .arg( + Arg::with_name("leader_action") + .index(2) + .default_value("unchanged"), ), - ) - .get_matches(); + ), + ).get_matches(); let config = value_t!(app.value_of("config"), String).expect("config file must be string"); let mut file = File::open(&config).expect(&format!("opening config file at {}", &config)); let mut config_str = String::new(); - file.read_to_string(&mut config_str).expect( - "reading config file", - ); + file.read_to_string(&mut config_str) + .expect("reading config file"); let mut system: System = toml::de::from_str(&config_str).expect("parsing config"); if let Some(v) = app.value_of("verbosity") { @@ -337,12 +338,14 @@ impl System { if let Some(_) = query.subcommand_matches("status") { (system, Command::Query(MgmtCommand::Status, server)) } else if let Some(args) = query.subcommand_matches("consensus") { - let c_action = value_t!(args.value_of("action"), ConsensusAction).expect("bad consensus action"); - let l_action = value_t!(args.value_of("leader_action"), LeaderAction).expect("bad leader action"); + let c_action = value_t!(args.value_of("action"), ConsensusAction) + .expect("bad consensus action"); + let l_action = value_t!(args.value_of("leader_action"), LeaderAction) + .expect("bad leader action"); ( system, Command::Query(MgmtCommand::ConsensusCommand(c_action, l_action), server), - ) + ) } else { // shold be unreachable unreachable!("clap bug?") diff --git a/src/consul.rs b/src/consul.rs index 3cc8712..2ac61d1 100644 --- a/src/consul.rs +++ b/src/consul.rs @@ -19,28 +19,23 @@ pub enum ConsulError { #[fail(display = "session create error: {}", _0)] Session(String), - #[fail(display = "server responded with bad status code '{}': {}", _0, _1)] + #[fail( + display = "server responded with bad status code '{}': {}", + _0, + _1 + )] HttpStatus(hyper::StatusCode, String), #[fail(display = "agent connection timed out")] ConnectionTimeout, #[fail(display = "Http error: {}", _0)] - Http( - #[cause] - hyper::Error - ), + Http(#[cause] hyper::Error), #[fail(display = "Parsing response: {}", _0)] - Parsing( - #[cause] - serde_json::Error - ), + Parsing(#[cause] serde_json::Error), #[fail(display = "I/O error {}", _0)] - Io( - #[cause] - ::std::io::Error - ), + Io(#[cause] ::std::io::Error), #[fail(display = "{}", _0)] Renew(String), @@ -145,10 +140,10 @@ impl IntoFuture for ConsulConsensus { let new_session = new_session.clone(); Box::new( Delay::new(Instant::now() + error_pause) - .then(move |_| Ok(Loop::Continue(new_session))), - ) + .then(move |_| Ok(Loop::Continue(new_session))), + ) as Box, Error = _>> - //ok(Loop::Continue(new_session)) + //ok(Loop::Continue(new_session)) } Ok(None) => { warn!(log, "timed out getting consul session"); @@ -159,8 +154,8 @@ impl IntoFuture for ConsulConsensus { } else { Either::B( Delay::new(Instant::now() + error_pause) - .then(move |_| Ok(Loop::Continue(new_session))), - ) + .then(move |_| Ok(Loop::Continue(new_session))), + ) } }); @@ -243,8 +238,7 @@ impl IntoFuture for ConsulSession { let ttl_ns = ttl.as_secs() * 1_000_000_000u64 + ttl.subsec_nanos() as u64; let b = format!( "{{\"TTL\": \"{}ns\", \"LockDelay\": \"{}ns\"}}", - ttl_ns, - ttl_ns + ttl_ns, ttl_ns ); let bodylen = b.len() as u64; *session_req.body_mut() = hyper::Body::from(b); @@ -252,11 +246,11 @@ impl IntoFuture for ConsulSession { session_req.headers_mut().insert( CONTENT_LENGTH, HeaderValue::from_str(&format!("{}", bodylen)).unwrap(), - ); + ); session_req.headers_mut().insert( CONTENT_TYPE, HeaderValue::from_str(WWW_FORM_URLENCODED.as_str()).unwrap(), - ); + ); let c_session = client .request(session_req) @@ -264,11 +258,13 @@ impl IntoFuture for ConsulSession { .and_then(move |resp| { let status = resp.status(); if status == StatusCode::OK { - let body = resp.into_body() + let body = resp + .into_body() .concat2() .map_err(|e| ConsulError::Http(e)) .and_then(move |body| { - let resp: ConsulSessionResponse = try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); + let resp: ConsulSessionResponse = + try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); debug!(log, "new session"; "id"=>format!("{}", resp.id)); Ok(Some(resp.id)) }); @@ -276,12 +272,13 @@ impl IntoFuture for ConsulSession { } else { let body = resp.into_body().concat2().map_err(|e| ConsulError::Http(e)); // TODO make this into option - let sleep = Delay::new(Instant::now() + Duration::from_millis(1000)).map_err(|e| ConsulError::Timer(e)); + let sleep = Delay::new(Instant::now() + Duration::from_millis(1000)) + .map_err(|e| ConsulError::Timer(e)); let future = sleep.join(body).then(move |res| match res { Ok((_, body)) => Err::, _>(ConsulError::HttpStatus( - status, - format!("{:?}", String::from_utf8(body.to_vec())), - )), + status, + format!("{:?}", String::from_utf8(body.to_vec())), + )), Err(e) => Err(e), }); Box::new(future) @@ -326,11 +323,11 @@ impl IntoFuture for ConsulRenew { renew_req.headers_mut().insert( CONTENT_LENGTH, HeaderValue::from_str(&format!("{}", bodylen)).unwrap(), - ); + ); renew_req.headers_mut().insert( CONTENT_TYPE, HeaderValue::from_str(WWW_FORM_URLENCODED.as_str()).unwrap(), - ); + ); let renew_client = hyper::Client::new(); let future = renew_client.request(renew_req).then(move |res| match res { @@ -338,7 +335,8 @@ impl IntoFuture for ConsulRenew { Ok(resp) => { if resp.status() != StatusCode::OK { let status = resp.status().clone(); - let body = resp.into_body() + let body = resp + .into_body() .concat2() .map_err(|e| ConsulError::Http(e)) .and_then(move |body| { @@ -396,7 +394,8 @@ impl IntoFuture for ConsulAcquire { .concat2() .map_err(|e| ConsulError::Http(e)) .and_then(move |body| { - let acquired: bool = try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); + let acquired: bool = + try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); switch_leader(acquired, &log); // let should_set = { diff --git a/src/errors.rs b/src/errors.rs index af6061d..b5b5628 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -3,22 +3,13 @@ use std::net::SocketAddr; #[derive(Fail, Debug)] pub enum GeneralError { #[fail(display = "I/O error")] - Io( - #[cause] - ::std::io::Error - ), + Io(#[cause] ::std::io::Error), #[fail(display = "Error when creating timer: {}", _0)] - Timer( - #[cause] - ::tokio::timer::Error - ), + Timer(#[cause] ::tokio::timer::Error), #[fail(display = "getting system time")] - Time( - #[cause] - ::std::time::SystemTimeError - ), + Time(#[cause] ::std::time::SystemTimeError), #[fail(display = "Gave up connecting to {}", _0)] TcpOutOfTries(SocketAddr), diff --git a/src/main.rs b/src/main.rs index fcb4d47..4615d34 100644 --- a/src/main.rs +++ b/src/main.rs @@ -6,10 +6,10 @@ extern crate failure; extern crate failure_derive; #[macro_use] extern crate slog; +extern crate rand; extern crate slog_async; extern crate slog_scope; extern crate slog_term; -extern crate rand; // Options #[macro_use] @@ -51,11 +51,11 @@ pub mod management; pub mod metric; pub mod parser; pub mod peer; +pub mod raft; pub mod server; -pub mod udp; pub mod task; +pub mod udp; pub mod util; -pub mod raft; pub mod protocol_capnp { include!(concat!(env!("OUT_DIR"), "/schema/protocol_capnp.rs")); @@ -71,7 +71,7 @@ use std::time::{self, Duration, Instant, SystemTime}; use slog::{Drain, Level}; -use bytes::{Bytes}; +use bytes::Bytes; use futures::future::{empty, ok}; use futures::sync::mpsc; use futures::{Future, IntoFuture, Stream}; @@ -79,7 +79,7 @@ use futures::{Future, IntoFuture, Stream}; use tokio::runtime::current_thread::Runtime; use tokio::timer::Interval; -use udp::{start_sync_udp, start_async_udp}; +use udp::{start_async_udp, start_sync_udp}; use carbon::CarbonBackend; use config::{Command, Consul, Metrics, Network, System}; @@ -88,9 +88,11 @@ use errors::GeneralError; use management::{MgmtClient, MgmtServer}; use metric::Metric; use peer::{NativeProtocolServer, NativeProtocolSnapshot}; -use task::Task; -use util::{AggregateOptions, Aggregator, BackoffRetryBuilder, OwnStats, UpdateCounterOptions, try_resolve}; use raft::start_internal_raft; +use task::Task; +use util::{ + try_resolve, AggregateOptions, Aggregator, BackoffRetryBuilder, OwnStats, UpdateCounterOptions, +}; // floating type used all over the code, can be changed to f32, to use less memory at the price of // precision @@ -142,7 +144,7 @@ pub enum ConsensusKind { lazy_static! { pub static ref CONSENSUS_STATE: Mutex = - { Mutex::new(ConsensusState::Disabled) }; + { Mutex::new(ConsensusState::Disabled) }; } pub static IS_LEADER: AtomicBool = ATOMIC_BOOL_INIT; @@ -152,35 +154,38 @@ fn main() { let System { verbosity, - network: Network { - listen, - peer_listen, - mgmt_listen, - bufsize, - multimessage, - mm_packets, - mm_async, - buffer_flush, - greens, - async_sockets, - nodes, - snapshot_interval, - }, + network: + Network { + listen, + peer_listen, + mgmt_listen, + bufsize, + multimessage, + mm_packets, + mm_async, + buffer_flush, + greens, + async_sockets, + nodes, + snapshot_interval, + }, raft, - consul: Consul { - start_as: consul_start_as, - agent, - session_ttl: consul_session_ttl, - renew_time: consul_renew_time, - key_name: consul_key, - }, - metrics: Metrics { - // max_metrics, - mut count_updates, - update_counter_prefix, - update_counter_suffix, - update_counter_threshold, - }, + consul: + Consul { + start_as: consul_start_as, + agent, + session_ttl: consul_session_ttl, + renew_time: consul_renew_time, + key_name: consul_key, + }, + metrics: + Metrics { + // max_metrics, + mut count_updates, + update_counter_prefix, + update_counter_suffix, + update_counter_threshold, + }, carbon, n_threads, w_threads, @@ -247,8 +252,7 @@ fn main() { let mut runtime = Runtime::new().expect("creating runtime for counting worker"); let future = rx.for_each(move |task: Task| ok(task.run())); runtime.block_on(future).expect("worker thread failed"); - }) - .expect("starting counting worker thread"); + }).expect("starting counting worker thread"); } let stats_prefix = stats_prefix.trim_right_matches(".").to_string(); @@ -268,11 +272,11 @@ fn main() { nodes.clone(), Duration::from_millis(snapshot_interval as u64), &chans, - ).into_future() - .map_err(move |e| { - PEER_ERRORS.fetch_add(1, Ordering::Relaxed); - info!(snap_err_log, "error sending snapshot";"error"=>format!("{}", e)); - }); + ).into_future() + .map_err(move |e| { + PEER_ERRORS.fetch_add(1, Ordering::Relaxed); + info!(snap_err_log, "error sending snapshot";"error"=>format!("{}", e)); + }); runtime.spawn(snapshot); // settings safe for asap restart @@ -295,7 +299,6 @@ fn main() { // Init leader state before starting backend IS_LEADER.store(start_as_leader, Ordering::SeqCst); - let consensus_log = rlog.clone(); match consensus { @@ -314,19 +317,19 @@ fn main() { consensus.set_renew_time(Duration::from_millis(consul_renew_time as u64)); runtime.spawn(consensus.into_future().map_err(|_| ())); // TODO errors } - ConsensusKind::None => {IS_LEADER.store(true, Ordering::SeqCst);}, + ConsensusKind::None => { + IS_LEADER.store(true, Ordering::SeqCst); + } } info!(log, "starting management server"); let m_serv_log = rlog.clone(); let m_serv_err_log = rlog.clone(); let m_server = hyper::Server::bind(&mgmt_listen) - .serve(move || { - ok::<_, hyper::Error>(MgmtServer::new(m_serv_log.clone(), &mgmt_listen)) - }) - .map_err(move |e| { - warn!(m_serv_err_log, "management server gone with error: {:?}", e); - }); + .serve(move || ok::<_, hyper::Error>(MgmtServer::new(m_serv_log.clone(), &mgmt_listen))) + .map_err(move |e| { + warn!(m_serv_err_log, "management server gone with error: {:?}", e); + }); runtime.spawn(m_server); @@ -439,10 +442,14 @@ fn main() { let tlog = rlog.clone(); let flags = flush_flags.clone(); - let flush_timer = flush_timer.map_err(|e| GeneralError::Timer(e)).for_each( - move |_tick| { + let flush_timer = flush_timer + .map_err(|e| GeneralError::Timer(e)) + .for_each(move |_tick| { debug!(tlog, "buffer flush requested"); - flags.iter().map(|flag| flag.swap(true, Ordering::SeqCst)).last(); + flags + .iter() + .map(|flag| flag.swap(true, Ordering::SeqCst)) + .last(); Ok(()) }); let tlog = rlog.clone(); @@ -452,12 +459,33 @@ fn main() { } if multimessage { - start_sync_udp(log, listen, &chans, n_threads, bufsize, mm_packets, mm_async, task_queue_size, buffer_flush, flush_flags.clone()); + start_sync_udp( + log, + listen, + &chans, + n_threads, + bufsize, + mm_packets, + mm_async, + task_queue_size, + buffer_flush, + flush_flags.clone(), + ); } else { - start_async_udp(log, listen, &chans, n_threads, greens, async_sockets, bufsize, task_queue_size, flush_flags.clone()); + start_async_udp( + log, + listen, + &chans, + n_threads, + greens, + async_sockets, + bufsize, + task_queue_size, + flush_flags.clone(), + ); } - runtime.block_on(empty::<(), ()>()).expect( - "running runtime in main thread", - ); + runtime + .block_on(empty::<(), ()>()) + .expect("running runtime in main thread"); } diff --git a/src/management.rs b/src/management.rs index 39fbf92..5b0dd39 100644 --- a/src/management.rs +++ b/src/management.rs @@ -16,28 +16,16 @@ use {ConsensusState, CONSENSUS_STATE, IS_LEADER}; #[derive(Fail, Debug)] pub enum MgmtError { #[fail(display = "I/O error: {}", _0)] - Io( - #[cause] - ::std::io::Error - ), + Io(#[cause] ::std::io::Error), #[fail(display = "Http error: {}", _0)] - Http( - #[cause] - hyper::Error - ), + Http(#[cause] hyper::Error), #[fail(display = "JSON decoding error {}", _0)] - Decode( - #[cause] - serde_json::error::Error - ), + Decode(#[cause] serde_json::error::Error), #[fail(display = "JSON encoding error: {}", _0)] - Encode( - #[cause] - serde_json::error::Error - ), + Encode(#[cause] serde_json::error::Error), #[fail(display = "bad command")] BadCommand, @@ -130,9 +118,7 @@ pub struct MgmtServer { impl MgmtServer { pub fn new(log: Logger, address: &SocketAddr) -> Self { Self { - log: log.new( - o!("source"=>"management-server", "server"=>format!("{}", address)), - ), + log: log.new(o!("source"=>"management-server", "server"=>format!("{}", address))), } } } @@ -238,9 +224,8 @@ pub struct MgmtClient { impl MgmtClient { pub fn new(log: Logger, address: SocketAddr, command: MgmtCommand) -> Self { Self { - log: log.new( - o!("source"=>"management-client", "server"=>format!("{}", address.clone())), - ), + log: log + .new(o!("source"=>"management-client", "server"=>format!("{}", address.clone()))), address, command, } @@ -264,9 +249,9 @@ impl IntoFuture for MgmtClient { match command { MgmtCommand::Status => { *req.method_mut() = Method::GET; - *req.uri_mut() = format!("http://{}/status", address).parse().expect( - "creating url for management command ", - ); + *req.uri_mut() = format!("http://{}/status", address) + .parse() + .expect("creating url for management command "); let client = hyper::Client::new(); let clog = log.clone(); @@ -274,24 +259,29 @@ impl IntoFuture for MgmtClient { Err(e) => Box::new(err(MgmtError::Http(e))), Ok(resp) => { if resp.status() == StatusCode::OK { - let body = resp.into_body() + let body = resp + .into_body() .concat2() .map_err(|e| MgmtError::Http(e)) - .map(move |body| match serde_json::from_slice::( - &*body, - ) { - Ok(status) => { - println!("{:?}", status); - } - Err(e) => { - println!("Error parsing server response: {}", e.to_string()); + .map(move |body| { + match serde_json::from_slice::(&*body) { + Ok(status) => { + println!("{:?}", status); + } + Err(e) => { + println!( + "Error parsing server response: {}", + e.to_string() + ); + } } }); Box::new(body) as Box> } else { - Box::new(ok( - warn!(clog, "Bad status returned from server: {:?}", resp), - )) + Box::new(ok(warn!( + clog, + "Bad status returned from server: {:?}", resp + ))) } } }); @@ -299,9 +289,9 @@ impl IntoFuture for MgmtClient { } command @ MgmtCommand::ConsensusCommand(_, _) => { *req.method_mut() = Method::POST; - *req.uri_mut() = format!("http://{}/consensus", address).parse().expect( - "creating url for management command", - ); + *req.uri_mut() = format!("http://{}/consensus", address) + .parse() + .expect("creating url for management command"); let body = serde_json::to_vec_pretty(&command).unwrap(); *req.body_mut() = Body::from(body); @@ -311,24 +301,29 @@ impl IntoFuture for MgmtClient { Err(e) => Box::new(err(MgmtError::Http(e))), Ok(resp) => { if resp.status() == StatusCode::OK { - let body = resp.into_body() + let body = resp + .into_body() .concat2() .map_err(|e| MgmtError::Http(e)) - .map(move |body| match serde_json::from_slice::( - &*body, - ) { - Ok(status) => { - println!("New server state: {:?}", status); - } - Err(e) => { - println!("Error parsing server response: {}", e.to_string()); + .map(move |body| { + match serde_json::from_slice::(&*body) { + Ok(status) => { + println!("New server state: {:?}", status); + } + Err(e) => { + println!( + "Error parsing server response: {}", + e.to_string() + ); + } } }); Box::new(body) as Box> } else { - Box::new(ok( - warn!(clog, "Bad status returned from server: {:?}", resp), - )) + Box::new(ok(warn!( + clog, + "Bad status returned from server: {:?}", resp + ))) } } }); @@ -371,9 +366,7 @@ mod test { let c_serv_err_log = rlog.clone(); let s_addr = address.clone(); let server = hyper::Server::bind(&address) - .serve(move || { - ok::<_, hyper::Error>(MgmtServer::new(c_serv_log.clone(), &s_addr)) - }) + .serve(move || ok::<_, hyper::Error>(MgmtServer::new(c_serv_log.clone(), &s_addr))) .map_err(move |e| { warn!(c_serv_err_log, "management server gone with error: {:?}", e); }); @@ -394,19 +387,22 @@ mod test { // let server some time to settle // then test the status command let d = Delay::new(Instant::now() + Duration::from_secs(1)); - let delayed = d.map_err(|_| ()).and_then(move |_| { - client.into_future().map_err(|e| panic!(e)) - }); + let delayed = d + .map_err(|_| ()) + .and_then(move |_| client.into_future().map_err(|e| panic!(e))); runtime.spawn(delayed); // then send a status change command let d = Delay::new(Instant::now() + Duration::from_secs(2)); let delayed = d.map_err(|_| ()).and_then(move |_| { - let command = MgmtCommand::ConsensusCommand(ConsensusAction::Enable, LeaderAction::Enable); + let command = + MgmtCommand::ConsensusCommand(ConsensusAction::Enable, LeaderAction::Enable); let client = MgmtClient::new(log.clone(), address, command); - client.into_future().map_err(|e| panic!("{:?}", e)).map( - move |_| { + client + .into_future() + .map_err(|e| panic!("{:?}", e)) + .map(move |_| { // ensure state has changed let state = ServerStatus::new(); assert_eq!( @@ -416,8 +412,7 @@ mod test { leader_status: true, } ) - }, - ) + }) }); runtime.spawn(delayed); diff --git a/src/metric.rs b/src/metric.rs index de165fc..bd6fc3c 100644 --- a/src/metric.rs +++ b/src/metric.rs @@ -82,9 +82,27 @@ where impl Metric where - F: Add + AddAssign + Sub + SubAssign + Clone + Div + Mul + PartialOrd + PartialEq + Into + From + Copy + Debug + Sync, + F: Add + + AddAssign + + Sub + + SubAssign + + Clone + + Div + + Mul + + PartialOrd + + PartialEq + + Into + + From + + Copy + + Debug + + Sync, { - pub fn new(value: F, mtype: MetricType, timestamp: Option, sampling: Option) -> Result { + pub fn new( + value: F, + mtype: MetricType, + timestamp: Option, + sampling: Option, + ) -> Result { let mut metric = Metric { value, mtype, @@ -147,7 +165,9 @@ where Ok(()) } - pub fn from_capnp<'a>(reader: cmetric::Reader<'a>) -> Result<(Bytes, Metric), MetricError> { + pub fn from_capnp<'a>( + reader: cmetric::Reader<'a>, + ) -> Result<(Bytes, Metric), MetricError> { let name = reader.get_name().map_err(MetricError::Capnp)?.into(); let value = reader.get_value(); @@ -182,9 +202,10 @@ where let (sampling, up_counter) = match reader.get_meta() { Ok(reader) => ( if reader.has_sampling() { - reader.get_sampling().ok().map( - |reader| reader.get_sampling(), - ) + reader + .get_sampling() + .ok() + .map(|reader| reader.get_sampling()) } else { None }, @@ -226,8 +247,7 @@ where .map(|(idx, value)| { let value: f64 = (*value).into(); timer_builder.set(idx as u32, value); - }) - .last(); + }).last(); } } } @@ -270,7 +290,18 @@ where impl IntoIterator for Metric where - F: Debug + Add + AddAssign + Sub + SubAssign + Div + Mul + Clone + Copy + PartialOrd + PartialEq + Into, + F: Debug + + Add + + AddAssign + + Sub + + SubAssign + + Div + + Mul + + Clone + + Copy + + PartialOrd + + PartialEq + + Into, { type Item = (&'static str, Float); type IntoIter = MetricIter; @@ -281,7 +312,18 @@ where pub struct MetricIter where - F: Debug + Add + AddAssign + Sub + SubAssign + Div + Mul + Clone + Copy + PartialOrd + PartialEq + Into, + F: Debug + + Add + + AddAssign + + Sub + + SubAssign + + Div + + Mul + + Clone + + Copy + + PartialOrd + + PartialEq + + Into, { m: Metric, count: usize, @@ -291,7 +333,18 @@ where impl MetricIter where - F: Debug + Add + AddAssign + Sub + SubAssign + Div + Mul + Clone + Copy + PartialOrd + PartialEq + Into, + F: Debug + + Add + + AddAssign + + Sub + + SubAssign + + Div + + Mul + + Clone + + Copy + + PartialOrd + + PartialEq + + Into, { fn new(mut metric: Metric) -> Self { let sum = if let MetricType::Timer(ref mut agg) = metric.mtype { @@ -311,7 +364,18 @@ where impl Iterator for MetricIter where - F: Debug + Add + AddAssign + Sub + SubAssign + Div + Mul + Clone + Copy + PartialOrd + PartialEq + Into, + F: Debug + + Add + + AddAssign + + Sub + + SubAssign + + Div + + Mul + + Clone + + Copy + + PartialOrd + + PartialEq + + Into, { type Item = (&'static str, Float); diff --git a/src/parser.rs b/src/parser.rs index d2b0583..1930a59 100644 --- a/src/parser.rs +++ b/src/parser.rs @@ -13,9 +13,26 @@ use metric::MetricType; // to make his zero-copy and get better errors, parser only recognizes parts // of the metric: (name, value, type, sampling) -pub fn metric_parser<'a, F>() -> impl Parser, Option), Input = &'a [u8]> +pub fn metric_parser<'a, F>( +) -> impl Parser, Option), Input = &'a [u8]> where -F: FromStr + Add + AddAssign + Sub + SubAssign + Div + Mul + Neg + PartialOrd + Into + From + Debug + Default + Clone + Copy + PartialEq + Sync, + F: FromStr + + Add + + AddAssign + + Sub + + SubAssign + + Div + + Mul + + Neg + + PartialOrd + + Into + + From + + Debug + + Default + + Clone + + Copy + + PartialEq + + Sync, { // This will parse metric name and separator let name = take_while1(|c: u8| c != b':' && c != b'\n').skip(byte(b':')); @@ -43,9 +60,7 @@ F: FromStr + Add + AddAssign + Sub + SubAssign + Div().map_err(|_e| UnexpectedParse::Unexpected) - })? + .map(|v| v.parse::().map_err(|_e| UnexpectedParse::Unexpected))? }); ( name, @@ -54,19 +69,20 @@ F: FromStr + Add + AddAssign + Sub + SubAssign + Div((name, value, mtype, sampling)) - }) + Ok::<_, UnexpectedParse>((name, value, mtype, sampling)) + }) } #[cfg(test)] diff --git a/src/peer.rs b/src/peer.rs index 6867438..3b11185 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -23,16 +23,10 @@ use {Float, PEER_ERRORS}; #[derive(Fail, Debug)] pub enum PeerError { #[fail(display = "I/O error: {}", _0)] - Io( - #[cause] - ::std::io::Error - ), + Io(#[cause] ::std::io::Error), #[fail(display = "Error when creating timer: {}", _0)] - Timer( - #[cause] - ::tokio::timer::Error - ), + Timer(#[cause] ::tokio::timer::Error), #[fail(display = "error sending task to worker thread")] TaskSend, @@ -66,9 +60,7 @@ pub struct NativeProtocolServer { impl NativeProtocolServer { pub fn new(log: Logger, listen: SocketAddr, chans: Vec>) -> Self { Self { - log: log.new( - o!("source"=>"canproto-peer-server", "ip"=>format!("{}", listen.clone())), - ), + log: log.new(o!("source"=>"canproto-peer-server", "ip"=>format!("{}", listen.clone()))), listen, chans: chans, } @@ -87,10 +79,10 @@ impl IntoFuture for NativeProtocolServer { .incoming() .map_err(|e| PeerError::Io(e)) .for_each(move |conn| { - let peer_addr = conn.peer_addr().map(|addr| addr.to_string()).unwrap_or( - "[UNCONNECTED]" - .into(), - ); + let peer_addr = conn + .peer_addr() + .map(|addr| addr.to_string()) + .unwrap_or("[UNCONNECTED]".into()); let transport = ReadStream::new(conn, ReaderOptions::default()); let log = log.new(o!("remote"=>peer_addr)); @@ -103,14 +95,15 @@ impl IntoFuture for NativeProtocolServer { // decode incoming capnp data into message // FIXME unwraps let reader = reader.map_err(PeerError::Capnp)?; - let reader = reader.get_root::().map_err(PeerError::Capnp)?; + let reader = reader + .get_root::() + .map_err(PeerError::Capnp)?; let next_chan = chans.next().unwrap(); parse_and_send(reader, next_chan, log.clone()).map_err(|e| { warn!(log, "bad incoming message"; "error" => e.to_string()); PeerError::Metric(e) }) - }) - .for_each(|_| { + }).for_each(|_| { // Consume all messages from the stream Ok(()) }) @@ -119,7 +112,11 @@ impl IntoFuture for NativeProtocolServer { } } -fn parse_and_send(reader: cmsg::Reader, next_chan: Sender, log: Logger) -> Result<(), MetricError> { +fn parse_and_send( + reader: cmsg::Reader, + next_chan: Sender, + log: Logger, +) -> Result<(), MetricError> { match reader.which().map_err(MetricError::CapnpSchema)? { cmsg::Single(reader) => { let reader = reader.map_err(MetricError::Capnp)?; @@ -140,9 +137,9 @@ fn parse_and_send(reader: cmsg::Reader, next_chan: Sender, log: Logger) -> reader .iter() .map(|reader| { - Metric::::from_capnp(reader).map(|(name, metric)| metrics.push((name, metric))) - }) - .last(); + Metric::::from_capnp(reader) + .map(|(name, metric)| metrics.push((name, metric))) + }).last(); let future = next_chan .send(Task::AddMetrics(metrics)) .map(|_| ()) // drop next sender @@ -159,9 +156,9 @@ fn parse_and_send(reader: cmsg::Reader, next_chan: Sender, log: Logger) -> reader .iter() .map(|reader| { - Metric::::from_capnp(reader).map(|(name, metric)| metrics.push((name, metric))) - }) - .last(); + Metric::::from_capnp(reader) + .map(|(name, metric)| metrics.push((name, metric))) + }).last(); let future = next_chan .send(Task::AddSnapshot(metrics)) .map(|_| ()) // drop next sender @@ -183,7 +180,12 @@ pub struct NativeProtocolSnapshot { } impl NativeProtocolSnapshot { - pub fn new(log: &Logger, nodes: Vec, interval: Duration, chans: &Vec>) -> Self { + pub fn new( + log: &Logger, + nodes: Vec, + interval: Duration, + chans: &Vec>, + ) -> Self { Self { log: log.new(o!("source"=>"peer-client")), nodes, @@ -251,7 +253,11 @@ impl IntoFuture for NativeProtocolSnapshot { let mut snapshot_message = Builder::new_default(); { let builder = snapshot_message.init_root::<::protocol_capnp::message::Builder>(); - let mut multi_metric = builder.init_snapshot(metrics.len() as u32); + let flat_len = metrics + .iter() + .flat_map(|hmap| hmap.iter()) + .count(); + let mut multi_metric = builder.init_snapshot(flat_len as u32); metrics .into_iter() .flat_map(|hmap| hmap.into_iter()) @@ -352,9 +358,9 @@ mod test { let sender = TcpStream::connect(&address) .map_err(|e| { println!("connection err: {:?}", e); - }) - .and_then(move |conn| { - let codec = ::capnp_futures::serialize::Transport::new(conn, ReaderOptions::default()); + }).and_then(move |conn| { + let codec = + ::capnp_futures::serialize::Transport::new(conn, ReaderOptions::default()); let mut single_message = Builder::new_default(); { @@ -375,7 +381,8 @@ mod test { let mut snapshot_message = Builder::new_default(); { - let builder = snapshot_message.init_root::<::protocol_capnp::message::Builder>(); + let builder = + snapshot_message.init_root::<::protocol_capnp::message::Builder>(); let multi_metric = builder.init_snapshot(1); let mut new_metric = multi_metric.get(0); new_metric.set_name("complex.test.bioyino_snapshot"); @@ -384,14 +391,12 @@ mod test { codec .send(single_message) .and_then(|codec| { - codec.send(multi_message).and_then(|codec| { - codec.send(snapshot_message) - }) - }) - .map(|_| ()) + codec + .send(multi_message) + .and_then(|codec| codec.send(snapshot_message)) + }).map(|_| ()) .map_err(|e| println!("codec error: {:?}", e)) - }) - .map_err(move |e| debug!(log, "error sending snapshot: {:?}", e)); + }).map_err(move |e| debug!(log, "error sending snapshot: {:?}", e)); let d = Delay::new(Instant::now() + Duration::from_secs(1)); let delayed = d.map_err(|_| ()).and_then(|_| sender); diff --git a/src/raft.rs b/src/raft.rs index 860910b..2331428 100644 --- a/src/raft.rs +++ b/src/raft.rs @@ -1,8 +1,8 @@ -use std::collections::HashMap; use rand::random; +use std::collections::HashMap; //use slog::{Drain, Level, Logger}; -use slog::{Logger}; +use slog::Logger; use futures::future::lazy; use tokio::runtime::current_thread::Runtime; @@ -13,10 +13,10 @@ use raft_tokio::raft_consensus::state_machine::null::NullStateMachine; //use raft_tokio::raft_consensus::ServerId; ////use raft_tokio::raft::RaftPeerProtocol; -use raft_tokio::start_raft_tcp; -use raft_tokio::{Notifier}; -use util::{switch_leader, try_resolve, get_hostname}; use config::Raft; +use raft_tokio::start_raft_tcp; +use raft_tokio::Notifier; +use util::{get_hostname, switch_leader, try_resolve}; pub struct LeaderNotifier(Logger); @@ -42,18 +42,24 @@ pub(crate) fn start_internal_raft(options: Raft, runtime: &mut Runtime, logger: let mut this_id = None; if options.nodes.len() < 3 { - warn!(logger, "raft requires at least 3 nodes, this may work not as intended"); + warn!( + logger, + "raft requires at least 3 nodes, this may work not as intended" + ); } // resolve nodes and generate random ServerId - let mut nodes = options.nodes.iter().map(|node| { - let id = random::().into(); - let addr = try_resolve(node); - if addr == this { - this_id = Some(id) - } - (id, addr) - }).collect::>(); + let mut nodes = options + .nodes + .iter() + .map(|node| { + let id = random::().into(); + let addr = try_resolve(node); + if addr == this { + this_id = Some(id) + } + (id, addr) + }).collect::>(); //let id = this_id/.expect("list of nodes must contain own hostname"); use raft_tokio::raft_consensus::ServerId; @@ -69,8 +75,10 @@ pub(crate) fn start_internal_raft(options: Raft, runtime: &mut Runtime, logger: let options = options.get_raft_options(); // Create the runtime - let raft = lazy(move || {start_raft_tcp(id, nodes, raft_log, sm, notifier, options, logger); Ok(())}); + let raft = lazy(move || { + start_raft_tcp(id, nodes, raft_log, sm, notifier, options, logger); + Ok(()) + }); runtime.spawn(raft); - } diff --git a/src/server.rs b/src/server.rs index 6d02993..9f7ed4b 100644 --- a/src/server.rs +++ b/src/server.rs @@ -1,4 +1,4 @@ -use std::sync::atomic::{Ordering, AtomicBool}; +use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use {DROPS, INGRESS}; @@ -26,17 +26,18 @@ pub struct StatsdServer { } impl StatsdServer { - pub fn new(socket: UdpSocket, - chans: Vec>, - buf: BytesMut, - buf_queue_size: usize, - bufsize: usize, - next: usize, - readbuf: BytesMut, - chunks: usize, - flush_flags: Arc>, - thread_idx: usize, - ) -> Self { + pub fn new( + socket: UdpSocket, + chans: Vec>, + buf: BytesMut, + buf_queue_size: usize, + bufsize: usize, + next: usize, + readbuf: BytesMut, + chunks: usize, + flush_flags: Arc>, + thread_idx: usize, + ) -> Self { Self { socket, chans, @@ -47,7 +48,7 @@ impl StatsdServer { readbuf, chunks, flush_flags, - thread_idx + thread_idx, } } } @@ -68,7 +69,7 @@ impl IntoFuture for StatsdServer { readbuf, chunks, flush_flags, - thread_idx + thread_idx, } = self; let future = socket @@ -82,8 +83,10 @@ impl IntoFuture for StatsdServer { buf.put(&received[0..size]); - - let flush = flush_flags.get(thread_idx).unwrap().swap(false, Ordering::SeqCst); + let flush = flush_flags + .get(thread_idx) + .unwrap() + .swap(false, Ordering::SeqCst); if buf.remaining_mut() < bufsize || chunks == 0 || flush { let (chan, next) = if next >= chans.len() { (chans[0].clone(), 1) @@ -94,22 +97,23 @@ impl IntoFuture for StatsdServer { spawn( chan.send(Task::Parse(buf.freeze())) - .map_err(|_| { DROPS.fetch_add(1, Ordering::Relaxed); }) - .and_then(move |_| { - StatsdServer::new( - socket, - chans, - newbuf, - buf_queue_size, - bufsize, - next, - received, - buf_queue_size * bufsize, - flush_flags, - thread_idx - ).into_future() - }), - ); + .map_err(|_| { + DROPS.fetch_add(1, Ordering::Relaxed); + }).and_then(move |_| { + StatsdServer::new( + socket, + chans, + newbuf, + buf_queue_size, + bufsize, + next, + received, + buf_queue_size * bufsize, + flush_flags, + thread_idx, + ).into_future() + }), + ); } else { spawn( StatsdServer::new( @@ -122,9 +126,9 @@ impl IntoFuture for StatsdServer { received, chunks - 1, flush_flags, - thread_idx + thread_idx, ).into_future(), - ); + ); } Ok(()) }); diff --git a/src/task.rs b/src/task.rs index d75bb6c..0b26393 100644 --- a/src/task.rs +++ b/src/task.rs @@ -11,7 +11,10 @@ use metric::Metric; use parser::metric_parser; use util::AggregateOptions; -use {Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, LONG_CACHE, PARSE_ERRORS, PEER_ERRORS, SHORT_CACHE}; +use { + Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, LONG_CACHE, PARSE_ERRORS, PEER_ERRORS, + SHORT_CACHE, +}; #[derive(Debug)] pub struct AggregateData { @@ -50,20 +53,16 @@ impl Task { pub fn run(self) { match self { Task::Parse(buf) => parse_and_insert(buf), - Task::AddMetric(name, metric) => { - SHORT_CACHE.with(move |c| { - let mut short = c.borrow_mut(); - update_metric(&mut short, name, metric); - }) - } - Task::AddMetrics(mut list) => { - SHORT_CACHE.with(move |c| { - let mut short = c.borrow_mut(); - list.drain(..) - .map(|(name, metric)| update_metric(&mut short, name, metric)) - .last(); - }) - } + Task::AddMetric(name, metric) => SHORT_CACHE.with(move |c| { + let mut short = c.borrow_mut(); + update_metric(&mut short, name, metric); + }), + Task::AddMetrics(mut list) => SHORT_CACHE.with(move |c| { + let mut short = c.borrow_mut(); + list.drain(..) + .map(|(name, metric)| update_metric(&mut short, name, metric)) + .last(); + }), Task::AddSnapshot(mut list) => { LONG_CACHE.with(move |c| { // snapshots go to long cache to avoid being duplicated to other nodes @@ -106,12 +105,12 @@ impl Task { }); } Task::Aggregate(AggregateData { - mut buf, - name, - metric, - options, - mut response, - }) => { + mut buf, + name, + metric, + options, + mut response, + }) => { let upd = if let Some(options) = options.update_counter { if metric.update_counter > options.threshold { // + 2 is for dots @@ -144,20 +143,20 @@ impl Task { buf.extend_from_slice(suffix.as_bytes()); let name = buf.take().freeze(); (name, value) - }) - .chain(upd) + }).chain(upd) .map(|data| { response .start_send(data) - .map_err(|_| { AGG_ERRORS.fetch_add(1, Ordering::Relaxed); }) - .map(|_| ()) + .map_err(|_| { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()) .unwrap_or(()); - }) - .last(); + }).last(); response .poll_complete() - .map_err(|_| { AGG_ERRORS.fetch_add(1, Ordering::Relaxed); }) - .map(|_| ()) + .map_err(|_| { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()) .unwrap_or_else(|_| ()); } } diff --git a/src/udp.rs b/src/udp.rs index 7fee023..a8bed6b 100644 --- a/src/udp.rs +++ b/src/udp.rs @@ -1,38 +1,38 @@ -use std::net::SocketAddr; use std::io; -use std::thread; -use std::sync::atomic::{Ordering, AtomicBool}; +use std::net::SocketAddr; +use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; +use std::thread; use std::ptr::null_mut; -use {INGRESS, DROPS}; -use task::Task; use server::StatsdServer; +use task::Task; +use {DROPS, INGRESS}; -use tokio::net::UdpSocket; -use std::os::unix::io::AsRawFd; -use tokio::runtime::current_thread::Runtime; -use bytes::{BytesMut, BufMut}; -use futures::IntoFuture; +use bytes::{BufMut, BytesMut}; use futures::future::empty; use futures::sync::mpsc::Sender; +use futures::IntoFuture; use net2::unix::UnixUdpBuilderExt; use net2::UdpBuilder; use slog::Logger; +use std::os::unix::io::AsRawFd; +use tokio::net::UdpSocket; +use tokio::runtime::current_thread::Runtime; - -pub(crate) fn start_sync_udp(log: Logger, - listen: SocketAddr, - chans: &Vec>, - n_threads: usize, - bufsize: usize, - mm_packets: usize, - mm_async: bool, - task_queue_size: usize, - buffer_flush: u64, - flush_flags: Arc>) -{ +pub(crate) fn start_sync_udp( + log: Logger, + listen: SocketAddr, + chans: &Vec>, + n_threads: usize, + bufsize: usize, + mm_packets: usize, + mm_async: bool, + task_queue_size: usize, + buffer_flush: u64, + flush_flags: Arc>, +) { info!(log, "multimessage enabled, starting in sync UDP mode"; "socket-is-blocking"=>mm_async, "packets"=>mm_packets); // It is crucial for recvmmsg to have one socket per many threads @@ -78,7 +78,7 @@ pub(crate) fn start_sync_udp(log: Logger, iov_base: buf.as_mut_ptr() as *mut c_void, iov_len: bufsize as size_t, }, - ); + ); let m = mmsghdr { msg_hdr: msghdr { msg_name: null_mut(), @@ -111,22 +111,19 @@ pub(crate) fn start_sync_udp(log: Logger, // So we set our chunk size ourselves and send buffer when this value // exhausted, but we only allocate when buffer becomes empty let mut chunks = task_queue_size as isize; - let flags = if mm_async { - MSG_WAITFORONE - } else { - 0 - }; + let flags = if mm_async { MSG_WAITFORONE } else { 0 }; loop { let timeout = if buffer_flush > 0 { &mut timespec { - tv_sec: buffer_flush as i64 / 1000, - tv_nsec: (buffer_flush as i64 % 1000) * 1000000, + tv_sec: buffer_flush as i64 / 1000, + tv_nsec: (buffer_flush as i64 % 1000) * 1_000_000, } } else { null_mut() }; - let res = unsafe { recvmmsg(fd as c_int, vp, vlen as c_uint, flags, timeout) }; + let res = + unsafe { recvmmsg(fd as c_int, vp, vlen as c_uint, flags, timeout) }; if res >= 0 { let end = res as usize; @@ -161,8 +158,7 @@ pub(crate) fn start_sync_udp(log: Logger, .map_err(|_| { warn!(log, "error sending buffer(queue full?)"); DROPS.fetch_add(res as usize, Ordering::Relaxed); - }) - .unwrap_or(()); + }).unwrap_or(()); chunks = task_queue_size as isize; } } else { @@ -177,20 +173,21 @@ pub(crate) fn start_sync_udp(log: Logger, } } } - }) - .expect("starting multimsg thread"); + }).expect("starting multimsg thread"); } } -pub(crate) fn start_async_udp(log: Logger, - listen: SocketAddr, - chans: &Vec>, - n_threads: usize, - greens: usize, - async_sockets: usize, - bufsize: usize, - task_queue_size: usize, - flush_flags: Arc>) { +pub(crate) fn start_async_udp( + log: Logger, + listen: SocketAddr, + chans: &Vec>, + n_threads: usize, + greens: usize, + async_sockets: usize, + bufsize: usize, + task_queue_size: usize, + flush_flags: Arc>, +) { info!(log, "multimessage is disabled, starting in async UDP mode"); // Create a pool of listener sockets let mut sockets = Vec::new(); @@ -228,7 +225,9 @@ pub(crate) fn start_async_udp(log: Logger, let chans = chans.clone(); // create UDP listener let socket = socket.try_clone().expect("cloning socket"); - let socket = UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()).expect("adding socket to event loop"); + let socket = + UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()) + .expect("adding socket to event loop"); let server = StatsdServer::new( socket, @@ -240,17 +239,16 @@ pub(crate) fn start_async_udp(log: Logger, readbuf, task_queue_size * bufsize, flush_flags.clone(), - i + i, ); runtime.spawn(server.into_future()); } } - runtime.block_on(empty::<(), ()>()).expect( - "starting runtime for async UDP", - ); - }) - .expect("creating UDP reader thread"); + runtime + .block_on(empty::<(), ()>()) + .expect("starting runtime for async UDP"); + }).expect("creating UDP reader thread"); } } diff --git a/src/util.rs b/src/util.rs index bc14faa..7136c04 100644 --- a/src/util.rs +++ b/src/util.rs @@ -1,9 +1,9 @@ +use libc; use std::collections::HashMap; -use std::sync::atomic::Ordering; -use std::time::{Duration, Instant}; use std::ffi::CStr; use std::net::SocketAddr; -use libc; +use std::sync::atomic::Ordering; +use std::time::{Duration, Instant}; use bytes::{BufMut, Bytes, BytesMut}; use futures::future::Either; @@ -32,14 +32,14 @@ pub fn try_resolve(s: &str) -> SocketAddr { let port = port.parse().expect("bad port value"); let first_ip = resolver::resolve_host(host) - .expect("failed resolving backend name") + .expect(&format!("failed resolving {:}", &host)) .next() .expect("at least one IP address required"); SocketAddr::new(first_ip, port) }) } -/// Get hostname. Copypyasted from some crate +/// Get hostname. Copypasted from some crate pub fn get_hostname() -> Option { let len = 255; let mut buf = Vec::::with_capacity(len); @@ -103,7 +103,8 @@ impl OwnStats { let name = buf.take().freeze(); let metric = Metric::new($value, MetricType::Counter, None, None).unwrap(); let log = self.log.clone(); - let sender = self.chan + let sender = self + .chan .clone() .send(Task::AddMetric(name, metric)) .map(|_| ()) @@ -179,7 +180,11 @@ pub struct Aggregator { } impl Aggregator { - pub fn new(options: AggregateOptions, chans: Vec>, tx: UnboundedSender<(Bytes, Float)>) -> Self { + pub fn new( + options: AggregateOptions, + chans: Vec>, + tx: UnboundedSender<(Bytes, Float)>, + ) -> Self { Self { options, chans, tx } } } @@ -196,9 +201,9 @@ impl IntoFuture for Aggregator { // TODO: change oneshots to single channel // to do that, task must run in new tokio, then we will not have to pass handle to it //handle.spawn(chan.send(Task::Rotate(tx)).then(|_| Ok(()))); - chan.send(Task::Rotate(tx)).map_err(|_| ()).and_then(|_| { - rx.and_then(|m| Ok(m)).map_err(|_| ()) - }) + chan.send(Task::Rotate(tx)) + .map_err(|_| ()) + .and_then(|_| rx.and_then(|m| Ok(m)).map_err(|_| ())) }); if options.is_leader { @@ -219,7 +224,7 @@ impl IntoFuture for Aggregator { // }) }); - let aggregate = accumulate.and_then(move |accumulated| { + let aggregate = accumulate.and_then(move |accumulated| { accumulated .into_iter() .inspect(|_| { EGRESS.fetch_add(1, Ordering::Relaxed); }) @@ -245,7 +250,7 @@ impl IntoFuture for Aggregator { .last(); Ok(()) }); - Box::new(aggregate) + Box::new(aggregate) } else { // only get metrics from threads let not_leader = futures_unordered(metrics).for_each(|_| Ok(())); @@ -275,16 +280,16 @@ impl Default for BackoffRetryBuilder { impl BackoffRetryBuilder { pub fn spawn(self, action: F) -> BackoffRetry - where + where F: IntoFuture + Clone, - { - let inner = Either::A(action.clone().into_future()); - BackoffRetry { - action, - inner: inner, - options: self, - } + { + let inner = Either::A(action.clone().into_future()); + BackoffRetry { + action, + inner: inner, + options: self, } + } } /// TCP client that is able to reconnect with customizable settings @@ -296,7 +301,7 @@ pub struct BackoffRetry { impl Future for BackoffRetry where -F: IntoFuture + Clone, + F: IntoFuture + Clone, { type Item = F::Item; type Error = Option; @@ -305,21 +310,19 @@ F: IntoFuture + Clone, loop { let (rotate_f, rotate_t) = match self.inner { // we are polling a future currently - Either::A(ref mut future) => { - match future.poll() { - Ok(Async::Ready(item)) => { - return Ok(Async::Ready(item)); - } - Ok(Async::NotReady) => return Ok(Async::NotReady), - Err(e) => { - if self.options.retries == 0 { - return Err(Some(e)); - } else { - (true, false) - } + Either::A(ref mut future) => match future.poll() { + Ok(Async::Ready(item)) => { + return Ok(Async::Ready(item)); + } + Ok(Async::NotReady) => return Ok(Async::NotReady), + Err(e) => { + if self.options.retries == 0 { + return Err(Some(e)); + } else { + (true, false) } } - } + }, Either::B(ref mut timer) => { match timer.poll() { // we are waiting for the delay From 9069f243344fc4955cafaaccc4153c82c9861a8e Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Thu, 18 Oct 2018 15:53:05 +0300 Subject: [PATCH 09/24] Better UDP flushing support --- config.toml | 37 ++++++++++++++++++----- src/config.rs | 22 +++++++++++--- src/main.rs | 26 +++++++++++++---- src/raft.rs | 25 +++++++++++----- src/server.rs | 22 +++++--------- src/udp.rs | 81 ++++++++++++++++++++++++++------------------------- 6 files changed, 133 insertions(+), 80 deletions(-) diff --git a/config.toml b/config.toml index 8359338..cf083a7 100644 --- a/config.toml +++ b/config.toml @@ -1,4 +1,6 @@ -# This is an example config showing all the options, filled with default values +# This is an example config showing all the possible options +# Required options are filled with default values +# Non-required options are commented with defaul values in comments verbosity = "warn" @@ -24,7 +26,7 @@ stats-interval = 10000 stats-prefix = "resources.monitoring.bioyino" # What consensus to use: "consul", "internal" or "none" -consensus = "internal" +consensus = "none" [metrics] # Should we provide metrics that update more than update-counter-threshold times diring aggregation interval @@ -87,9 +89,22 @@ mm-packets = 100 # This means recvmmsg will receive 0..mm-packets datagrams instead of waiting for mm-packets mm-async = false +# Multimessage mode assumes early return by timeout, but ONLY when received +# a packet after timeout expiration. +# Basically this should be changed in very rare and specific cases. +# 0 means this value will be equal to buffer-flush-time +# mm-timeout = 0 + # To avoid packets staying in queue forever, this option can be used to flush # incoming data buffer forcing it to be sent even if it's not full -buffer-flush = 0 +buffer-flush-time = 0 + +# Same as buffer-flush-time, but riggers on buffer length. Please, notice that multimessage +# mode can only consider timer, so this check is only possible every mm-packets packets. +# zero value means automatic management depending on memory allocator internal logic, +# which on tests was found to reach 30Mb +# if in multimessage mode this value is lower that mm-packets*bufsize, it will be set to this value +buffer-flush-length = 65536 # Nmber of green threads for single-message mode greens = 4 @@ -105,11 +120,16 @@ snapshot-interval = 1000 # Settings for internal Raft [raft] +# NOT IMPLEMENTD YET Defer start of raft consensus to avoid node becoming leader too early +# Such situation is very likely possible when restarting current leader node +# and leads to losing metrics +#start-delay = 29000 + # Timeouts tuned according to the Raft paper and typical network latency. # Better not to change if unsure -heartbeat-timeout = 250 -election-timeout-min = 500 -election-timeout-max = 750 +#heartbeat-timeout = 250 +#election-timeout-min = 500 +#election-timeout-max = 750 # The name of the current node is taken from hostname by default # After that all hostnames are resolved using DNS. If node name cannot @@ -117,8 +137,9 @@ election-timeout-max = 750 # parameter in a format similar to one in node list. # this-node = -# A list of other raft nodes in form of hostname:port or IP:port -nodes = [] +# A map of other raft nodes. Keys are in form of hostname:port or IP:port +# values are integers +nodes = {} [consul] # Start in disabled leader finding mode diff --git a/src/config.rs b/src/config.rs index ffbcacb..95aeec3 100644 --- a/src/config.rs +++ b/src/config.rs @@ -1,3 +1,4 @@ +use std::collections::HashMap; use std::fs::File; use std::io::Read; use std::net::SocketAddr; @@ -92,6 +93,9 @@ pub(crate) struct Metrics { /// Minimal update count to be reported pub update_counter_threshold: u32, + // TODO + // /// Whether we should spam parsing errors in logs + // pub log_parse_errors: bool, } impl Default for Metrics { @@ -171,8 +175,14 @@ pub(crate) struct Network { /// Number of multimessage packets to receive at once if in multimessage mode pub mm_async: bool, + /// A timeout to return from multimessage mode syscall + pub mm_timeout: u64, + /// A timer to flush incoming buffer making sure metrics are not stuck there - pub buffer_flush: u64, + pub buffer_flush_time: u64, + + /// A length of incoming buffer to flush it making sure metrics are not stuck there + pub buffer_flush_length: usize, /// Nmber of green threads for single-message mode pub greens: usize, @@ -197,7 +207,9 @@ impl Default for Network { multimessage: false, mm_packets: 100, mm_async: false, - buffer_flush: 3000, + mm_timeout: 0, + buffer_flush_length: 0, + buffer_flush_time: 0, greens: 4, async_sockets: 4, nodes: Vec::new(), @@ -240,6 +252,8 @@ impl Default for Consul { #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] pub(crate) struct Raft { + // /// Delay raft after start (ms) + // pub start_delay: u64, /// Raft heartbeat timeout (ms) pub heartbeat_timeout: u64, @@ -253,7 +267,7 @@ pub(crate) struct Raft { pub this_node: Option, /// List of Raft nodes, may include this_node - pub nodes: Vec, + pub nodes: HashMap, } impl Default for Raft { @@ -263,7 +277,7 @@ impl Default for Raft { election_timeout_min: 500, election_timeout_max: 750, this_node: None, - nodes: Vec::new(), + nodes: HashMap::new(), } } } diff --git a/src/main.rs b/src/main.rs index 4615d34..6c88539 100644 --- a/src/main.rs +++ b/src/main.rs @@ -163,7 +163,9 @@ fn main() { multimessage, mm_packets, mm_async, - buffer_flush, + mm_timeout, + buffer_flush_time, + buffer_flush_length, greens, async_sockets, nodes, @@ -303,6 +305,9 @@ fn main() { match consensus { ConsensusKind::Internal => { + let mut con_state = CONSENSUS_STATE.lock().unwrap(); + info!(log, "starting internal consensus"; "initial_state"=>format!("{:?}", con_state)); + *con_state = ConsensusState::Enabled; start_internal_raft(raft, &mut runtime, consensus_log); } ConsensusKind::Consul => { @@ -436,8 +441,16 @@ fn main() { } } - if buffer_flush > 0 { - let dur = Duration::from_millis(buffer_flush); + let reserve_min = bufsize * mm_packets; + let buffer_flush_length = if buffer_flush_length < reserve_min { + debug!(log, "buffer-flush-len is lower than mm-packets*bufsize"; "new-value"=>reserve_min, "old-value"=>buffer_flush_length); + reserve_min + } else { + buffer_flush_length + }; + + if buffer_flush_time > 0 { + let dur = Duration::from_millis(buffer_flush_time); let flush_timer = Interval::new(Instant::now() + dur, dur); let tlog = rlog.clone(); @@ -467,8 +480,9 @@ fn main() { bufsize, mm_packets, mm_async, - task_queue_size, - buffer_flush, + mm_timeout, + buffer_flush_time, + buffer_flush_length, flush_flags.clone(), ); } else { @@ -480,7 +494,7 @@ fn main() { greens, async_sockets, bufsize, - task_queue_size, + buffer_flush_length, flush_flags.clone(), ); } diff --git a/src/raft.rs b/src/raft.rs index 2331428..0ec08b4 100644 --- a/src/raft.rs +++ b/src/raft.rs @@ -1,7 +1,7 @@ -use rand::random; use std::collections::HashMap; -//use slog::{Drain, Level, Logger}; +use rand::random; + use slog::Logger; use futures::future::lazy; @@ -10,14 +10,16 @@ use tokio::runtime::current_thread::Runtime; use raft_tokio::raft_consensus::persistent_log::mem::MemLog; use raft_tokio::raft_consensus::state::ConsensusState; use raft_tokio::raft_consensus::state_machine::null::NullStateMachine; -//use raft_tokio::raft_consensus::ServerId; +use raft_tokio::raft_consensus::ServerId; ////use raft_tokio::raft::RaftPeerProtocol; use config::Raft; +use raft_tokio::raft::{BiggerIdSolver, ConnectionSolver}; use raft_tokio::start_raft_tcp; use raft_tokio::Notifier; use util::{get_hostname, switch_leader, try_resolve}; +#[derive(Clone)] pub struct LeaderNotifier(Logger); impl Notifier for LeaderNotifier { @@ -32,6 +34,13 @@ impl Notifier for LeaderNotifier { } } +// we reuse the type to avoid creating a new one +impl ConnectionSolver for LeaderNotifier { + fn solve(&self, is_client: bool, local_id: ServerId, remote_id: ServerId) -> bool { + return BiggerIdSolver.solve(is_client, local_id, remote_id); + } +} + pub(crate) fn start_internal_raft(options: Raft, runtime: &mut Runtime, logger: Logger) { let this = if let Some(name) = options.this_node.clone() { try_resolve(&name) @@ -52,13 +61,12 @@ pub(crate) fn start_internal_raft(options: Raft, runtime: &mut Runtime, logger: let mut nodes = options .nodes .iter() - .map(|node| { - let id = random::().into(); + .map(|(node, id)| { let addr = try_resolve(node); if addr == this { - this_id = Some(id) + this_id = Some(ServerId::from(*id)) } - (id, addr) + (ServerId::from(*id), addr) }).collect::>(); //let id = this_id/.expect("list of nodes must contain own hostname"); @@ -72,11 +80,12 @@ pub(crate) fn start_internal_raft(options: Raft, runtime: &mut Runtime, logger: let raft_log = MemLog::new(); let sm = NullStateMachine; let notifier = LeaderNotifier(logger.clone()); + let solver = notifier.clone(); let options = options.get_raft_options(); // Create the runtime let raft = lazy(move || { - start_raft_tcp(id, nodes, raft_log, sm, notifier, options, logger); + start_raft_tcp(id, nodes, raft_log, sm, notifier, options, logger, solver); Ok(()) }); diff --git a/src/server.rs b/src/server.rs index 9f7ed4b..edd83f8 100644 --- a/src/server.rs +++ b/src/server.rs @@ -16,11 +16,10 @@ pub struct StatsdServer { socket: UdpSocket, chans: Vec>, buf: BytesMut, - buf_queue_size: usize, + buffer_flush_length: usize, bufsize: usize, next: usize, readbuf: BytesMut, - chunks: usize, flush_flags: Arc>, thread_idx: usize, } @@ -30,11 +29,10 @@ impl StatsdServer { socket: UdpSocket, chans: Vec>, buf: BytesMut, - buf_queue_size: usize, + buffer_flush_length: usize, bufsize: usize, next: usize, readbuf: BytesMut, - chunks: usize, flush_flags: Arc>, thread_idx: usize, ) -> Self { @@ -42,11 +40,10 @@ impl StatsdServer { socket, chans, buf, - buf_queue_size, + buffer_flush_length, bufsize, next, readbuf, - chunks, flush_flags, thread_idx, } @@ -63,11 +60,10 @@ impl IntoFuture for StatsdServer { socket, chans, mut buf, - buf_queue_size, + buffer_flush_length, bufsize, next, readbuf, - chunks, flush_flags, thread_idx, } = self; @@ -87,13 +83,13 @@ impl IntoFuture for StatsdServer { .get(thread_idx) .unwrap() .swap(false, Ordering::SeqCst); - if buf.remaining_mut() < bufsize || chunks == 0 || flush { + if buf.remaining_mut() < bufsize || buf.len() >= buffer_flush_length || flush { let (chan, next) = if next >= chans.len() { (chans[0].clone(), 1) } else { (chans[next].clone(), next + 1) }; - let newbuf = BytesMut::with_capacity(buf_queue_size * bufsize); + let newbuf = BytesMut::with_capacity(buffer_flush_length); spawn( chan.send(Task::Parse(buf.freeze())) @@ -104,11 +100,10 @@ impl IntoFuture for StatsdServer { socket, chans, newbuf, - buf_queue_size, + buffer_flush_length, bufsize, next, received, - buf_queue_size * bufsize, flush_flags, thread_idx, ).into_future() @@ -120,11 +115,10 @@ impl IntoFuture for StatsdServer { socket, chans, buf, - buf_queue_size, + buffer_flush_length, bufsize, next, received, - chunks - 1, flush_flags, thread_idx, ).into_future(), diff --git a/src/udp.rs b/src/udp.rs index a8bed6b..20d7bb4 100644 --- a/src/udp.rs +++ b/src/udp.rs @@ -1,11 +1,10 @@ use std::io; use std::net::SocketAddr; +use std::ptr::null_mut; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::thread; -use std::ptr::null_mut; - use server::StatsdServer; use task::Task; use {DROPS, INGRESS}; @@ -29,8 +28,9 @@ pub(crate) fn start_sync_udp( bufsize: usize, mm_packets: usize, mm_async: bool, - task_queue_size: usize, - buffer_flush: u64, + mm_timeout: u64, + buffer_flush_time: u64, + buffer_flush_length: usize, flush_flags: Arc>, ) { info!(log, "multimessage enabled, starting in sync UDP mode"; "socket-is-blocking"=>mm_async, "packets"=>mm_packets); @@ -44,6 +44,12 @@ pub(crate) fn start_sync_udp( let sck = socket.bind(listen).unwrap(); sck.set_nonblocking(mm_async).unwrap(); + let mm_timeout = if mm_timeout == 0 { + buffer_flush_time + } else { + mm_timeout + }; + for i in 0..n_threads { let chans = chans.clone(); let log = log.new(o!("source"=>"mudp_thread")); @@ -99,76 +105,71 @@ pub(crate) fn start_sync_udp( let vp = v.as_mut_ptr(); let vlen = v.len(); - // This is the buffer we fill with metrics and periodically send to - // tasks - // To avoid allocations we make it bigger than multimsg message count - // Also, it can be the huge value already allocated here, so for even less + // This is the buffer we fill with metrics and periodically send to tasks + // Due to implementation of BytesMut it's real size is bigger, so for even less // allocations, we split the filled part and leave the rest for future bytes - let mut b = BytesMut::with_capacity(bufsize * mm_packets * task_queue_size); + let mut b = BytesMut::with_capacity(buffer_flush_length); - // We cannot count on allocator to allocate a value close to capacity - // it can be much more than that and stall our buffer for too long - // So we set our chunk size ourselves and send buffer when this value - // exhausted, but we only allocate when buffer becomes empty - let mut chunks = task_queue_size as isize; let flags = if mm_async { MSG_WAITFORONE } else { 0 }; + loop { - let timeout = if buffer_flush > 0 { - &mut timespec { - tv_sec: buffer_flush as i64 / 1000, - tv_nsec: (buffer_flush as i64 % 1000) * 1_000_000, + // timeout is mutable and changed by every recvmmsg call, so it MUST be inside loop + // creating timeout as &mut fails because it's supposedly not dropped + let mut timeout = if mm_timeout > 0 { + timespec { + tv_sec: (mm_timeout / 1000u64) as i64, + tv_nsec: ((mm_timeout % 1000u64) * 1_000_000u64) as i64, } } else { - null_mut() + timespec { + tv_sec: 0, + tv_nsec: 0, + } }; let res = - unsafe { recvmmsg(fd as c_int, vp, vlen as c_uint, flags, timeout) }; - - if res >= 0 { - let end = res as usize; + unsafe { recvmmsg(fd as c_int, vp, vlen as c_uint, flags, if mm_timeout > 0 {&mut timeout} else {null_mut()}) }; + if res == 0 { + // skip this shit + } else if res > 0 { + let messages = res as usize; // Check if we can fit all packets into buffer let mut total_bytes = 0; - for i in 0..end { + for i in 0..messages { total_bytes += v[i].msg_len as usize; } - // newlines - total_bytes += end - 1; // if we cannot, allocate more if b.remaining_mut() < total_bytes { - b.reserve(bufsize * mm_packets * task_queue_size) + b.reserve(buffer_flush_length) } // put packets into buffer - for i in 0..end { + for i in 0..messages { let len = v[i].msg_len as usize; - b.put(&message_vec[i][0..len]); - chunks -= end as isize; } // when it's time to send bytes, send them let flush = flush_flags.get(i).unwrap().swap(false, Ordering::SeqCst); - if chunks <= 0 || flush { + if flush || b.len() >= buffer_flush_length { let mut chan = ichans.next().unwrap().clone(); - INGRESS.fetch_add(res as usize, Ordering::Relaxed); + INGRESS.fetch_add(messages as usize, Ordering::Relaxed); chan.try_send(Task::Parse(b.take().freeze())) .map_err(|_| { warn!(log, "error sending buffer(queue full?)"); - DROPS.fetch_add(res as usize, Ordering::Relaxed); + DROPS.fetch_add(messages as usize, Ordering::Relaxed); }).unwrap_or(()); - chunks = task_queue_size as isize; } } else { let errno = unsafe { *__errno_location() }; if errno == EAGAIN { } else { warn!(log, "UDP receive error"; - "code"=> format!("{}",res), + "code"=> format!("{}", res), "error"=>format!("{}", io::Error::last_os_error()) - ) + ); } } } @@ -185,10 +186,11 @@ pub(crate) fn start_async_udp( greens: usize, async_sockets: usize, bufsize: usize, - task_queue_size: usize, + buffer_flush_length: usize, flush_flags: Arc>, ) { info!(log, "multimessage is disabled, starting in async UDP mode"); + // Create a pool of listener sockets let mut sockets = Vec::new(); for _ in 0..async_sockets { @@ -218,7 +220,7 @@ pub(crate) fn start_async_udp( for _ in 0..greens { // start a listener for all sockets for socket in sockets.iter() { - let buf = BytesMut::with_capacity(task_queue_size * bufsize); + let buf = BytesMut::with_capacity(buffer_flush_length); let mut readbuf = BytesMut::with_capacity(bufsize); unsafe { readbuf.set_len(bufsize) } @@ -233,11 +235,10 @@ pub(crate) fn start_async_udp( socket, chans.clone(), buf, - task_queue_size, + buffer_flush_length, bufsize, i, readbuf, - task_queue_size * bufsize, flush_flags.clone(), i, ); From 6f99004bf3c27b2994e16eea19427056ce78a1c2 Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Thu, 18 Oct 2018 16:51:30 +0300 Subject: [PATCH 10/24] Add raft delay --- Cargo.toml | 2 +- config.toml | 19 +++++++++---------- src/config.rs | 6 ++++-- src/main.rs | 31 +++++++++++++++++++++++++------ src/raft.rs | 8 ++++---- src/task.rs | 1 + 6 files changed, 44 insertions(+), 23 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 427ccbb..e11e80d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -24,7 +24,7 @@ mime="^0.3" serde="^1.0" serde_derive="^1.0" serde_json="^1.0" -slog="^2.2" +slog="^2.4" slog-term="^2.4" slog-async="^2.3" slog-scope="^4.0" diff --git a/config.toml b/config.toml index cf083a7..824572f 100644 --- a/config.toml +++ b/config.toml @@ -10,13 +10,11 @@ n-threads = 4 # Number of aggregating and counting threads, use 0(not recommended) to use all CPU cores w-threads = 4 -# Queue size for single counting thread before packet is dropped +# Queue size for single counting thread before task is dropped task-queue-size = 1024 -# if server should start in leader enabled state -# NOTE: this is importnt to set this value to true, when working in standalone mode -# since leader state only changes on command -start-as-leader = false +# If server should become leader from it's very start +start-as-leader = true # How often to gather own stats, in ms. Use 0 to disable (stats are still gathered and printed to log, # but not included in metric dump @@ -120,10 +118,10 @@ snapshot-interval = 1000 # Settings for internal Raft [raft] -# NOT IMPLEMENTD YET Defer start of raft consensus to avoid node becoming leader too early -# Such situation is very likely possible when restarting current leader node -# and leads to losing metrics -#start-delay = 29000 +# Defer start of raft consensus to avoid node becoming leader too early +# Such situation is very likely when restarting current leader node +# and means losing metrics in most cases +#start-delay = 0 # Timeouts tuned according to the Raft paper and typical network latency. # Better not to change if unsure @@ -142,7 +140,8 @@ snapshot-interval = 1000 nodes = {} [consul] -# Start in disabled leader finding mode +# Start in disabled leader finding mode. This only works while consul is bootstrapping. +# Can be helpful when there is a danger of agent being inaccessible. start-as = "disabled" # Consul agent address diff --git a/src/config.rs b/src/config.rs index 95aeec3..b2e5a97 100644 --- a/src/config.rs +++ b/src/config.rs @@ -252,8 +252,9 @@ impl Default for Consul { #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] pub(crate) struct Raft { - // /// Delay raft after start (ms) - // pub start_delay: u64, + /// Delay raft after start (ms) + pub start_delay: u64, + /// Raft heartbeat timeout (ms) pub heartbeat_timeout: u64, @@ -273,6 +274,7 @@ pub(crate) struct Raft { impl Default for Raft { fn default() -> Self { Self { + start_delay: 0, heartbeat_timeout: 250, election_timeout_min: 500, election_timeout_max: 750, diff --git a/src/main.rs b/src/main.rs index 6c88539..e98d23d 100644 --- a/src/main.rs +++ b/src/main.rs @@ -77,7 +77,7 @@ use futures::sync::mpsc; use futures::{Future, IntoFuture, Stream}; use tokio::runtime::current_thread::Runtime; -use tokio::timer::Interval; +use tokio::timer::{Delay, Interval}; use udp::{start_async_udp, start_sync_udp}; @@ -305,12 +305,27 @@ fn main() { match consensus { ConsensusKind::Internal => { - let mut con_state = CONSENSUS_STATE.lock().unwrap(); - info!(log, "starting internal consensus"; "initial_state"=>format!("{:?}", con_state)); - *con_state = ConsensusState::Enabled; - start_internal_raft(raft, &mut runtime, consensus_log); + if start_as_leader { + warn!(log, "Starting as leader with enabled consensus. More that one leader is possible before consensus settle up."); + } + let d = Delay::new(Instant::now() + Duration::from_millis(raft.start_delay)); + let log = log.clone(); + let delayed = d + .map_err(|_| ()) + .and_then(move |_| { + let mut con_state = CONSENSUS_STATE.lock().unwrap(); + info!(log, "starting internal consensus"; "initial_state"=>format!("{:?}", con_state)); + *con_state = ConsensusState::Enabled; + start_internal_raft(raft, consensus_log); + Ok(()) + }); + + runtime.spawn(delayed); } ConsensusKind::Consul => { + if start_as_leader { + warn!(log, "Starting as leader with enabled consensus. More that one leader is possible before consensus settle up."); + } { let mut con_state = CONSENSUS_STATE.lock().unwrap(); info!(log, "starting consul consensus"; "initial_state"=>format!("{:?}", con_state)); @@ -323,7 +338,11 @@ fn main() { runtime.spawn(consensus.into_future().map_err(|_| ())); // TODO errors } ConsensusKind::None => { - IS_LEADER.store(true, Ordering::SeqCst); + if !start_as_leader { + // starting as non-leader in this mode can be useful for agent mode + // so we don't disorient user with warnings + info!(log, "Starting as non-leader with disabled consensus. No metrics will be sent until leader is switched on by command"); + } } } diff --git a/src/raft.rs b/src/raft.rs index 0ec08b4..b3dd34c 100644 --- a/src/raft.rs +++ b/src/raft.rs @@ -5,7 +5,7 @@ use rand::random; use slog::Logger; use futures::future::lazy; -use tokio::runtime::current_thread::Runtime; +use tokio::runtime::current_thread::spawn; use raft_tokio::raft_consensus::persistent_log::mem::MemLog; use raft_tokio::raft_consensus::state::ConsensusState; @@ -41,7 +41,7 @@ impl ConnectionSolver for LeaderNotifier { } } -pub(crate) fn start_internal_raft(options: Raft, runtime: &mut Runtime, logger: Logger) { +pub(crate) fn start_internal_raft(options: Raft, logger: Logger) { let this = if let Some(name) = options.this_node.clone() { try_resolve(&name) } else { @@ -83,11 +83,11 @@ pub(crate) fn start_internal_raft(options: Raft, runtime: &mut Runtime, logger: let solver = notifier.clone(); let options = options.get_raft_options(); - // Create the runtime + // Create the raft runtime let raft = lazy(move || { start_raft_tcp(id, nodes, raft_log, sm, notifier, options, logger, solver); Ok(()) }); - runtime.spawn(raft); + spawn(raft); } diff --git a/src/task.rs b/src/task.rs index 0b26393..b0c998a 100644 --- a/src/task.rs +++ b/src/task.rs @@ -111,6 +111,7 @@ impl Task { options, mut response, }) => { + //println!("AGGG {:?} {:?}", buf, name); let upd = if let Some(options) = options.update_counter { if metric.update_counter > options.threshold { // + 2 is for dots From 99a6b9d827b29b95f94df98489d3ab9e0a60c33d Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Thu, 18 Oct 2018 20:23:41 +0300 Subject: [PATCH 11/24] Add parallel aggregation option --- Cargo.toml | 2 +- config.toml | 8 +++++++ src/config.rs | 5 +++++ src/main.rs | 7 +++++++ src/task.rs | 1 - src/util.rs | 58 ++++++++++++++++++++++++++++----------------------- 6 files changed, 53 insertions(+), 28 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index e11e80d..26fe434 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -5,7 +5,7 @@ authors = ["Sergey Noskov aka Albibek "] description = "StatsD-compatible, high-performance, fault-tolerant metric aggregator" [dependencies] -lazy_static="^1.0" +lazy_static="^1.1" clap="^2.31" failure="^0.1" failure_derive="^0.1" diff --git a/config.toml b/config.toml index 824572f..3aae66b 100644 --- a/config.toml +++ b/config.toml @@ -39,6 +39,14 @@ update-counter-suffix = "" # Minimal update counter to be reported update-counter-threshold = 200 +# Aggregation can be made in two ways giving a tradeoff between the aggregation speed and +# the probability of loosing metrics +# 1. In a specially spawned thread (fast-aggregation=false). In this case only one thread will be used to do all the counting. +# This is slow, but worker threads will keep parsing decreasing probabiilty of metric drops. +# 2. In worker threads (fast-aggregation=true). In this case metrics will be aggregated in worker threads, so +# any heavy counting would potentially block task and make some unparsed buffers to be skipped +# fast-aggregation = true + [carbon] # IP and port of the carbon-protocol backend to send aggregated data to diff --git a/src/config.rs b/src/config.rs index b2e5a97..9b39ae5 100644 --- a/src/config.rs +++ b/src/config.rs @@ -93,6 +93,10 @@ pub(crate) struct Metrics { /// Minimal update count to be reported pub update_counter_threshold: u32, + + /// Aggregate faster at the price or probably loosing some incoming metrics + pub fast_aggregation: bool, + // TODO // /// Whether we should spam parsing errors in logs // pub log_parse_errors: bool, @@ -106,6 +110,7 @@ impl Default for Metrics { update_counter_prefix: "resources.monitoring.bioyino.updates".to_string(), update_counter_suffix: String::new(), update_counter_threshold: 200, + fast_aggregation: true, } } } diff --git a/src/main.rs b/src/main.rs index e98d23d..8452f3c 100644 --- a/src/main.rs +++ b/src/main.rs @@ -104,6 +104,9 @@ pub type Cache = HashMap>; thread_local!(static LONG_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); thread_local!(static SHORT_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); +// options +//pub static SHOW_PARSE_ERRORS: AtomicBool = ATOMIC_BOOL_INIT; + // statistic counters pub static PARSE_ERRORS: AtomicUsize = ATOMIC_USIZE_INIT; pub static AGG_ERRORS: AtomicUsize = ATOMIC_USIZE_INIT; @@ -187,6 +190,7 @@ fn main() { update_counter_prefix, update_counter_suffix, update_counter_threshold, + fast_aggregation, }, carbon, n_threads, @@ -242,6 +246,8 @@ fn main() { let update_counter_suffix: Bytes = update_counter_suffix.into(); let log = rlog.new(o!("thread" => "main")); + // Init task options before initializing task threads + // Start counting threads info!(log, "starting counting threads"); let mut chans = Vec::with_capacity(w_threads); @@ -402,6 +408,7 @@ fn main() { } else { None }, + fast_aggregation, }; if is_leader { diff --git a/src/task.rs b/src/task.rs index b0c998a..0b26393 100644 --- a/src/task.rs +++ b/src/task.rs @@ -111,7 +111,6 @@ impl Task { options, mut response, }) => { - //println!("AGGG {:?} {:?}", buf, name); let upd = if let Some(options) = options.update_counter { if metric.update_counter > options.threshold { // + 2 is for dots diff --git a/src/util.rs b/src/util.rs index 7136c04..918d6a9 100644 --- a/src/util.rs +++ b/src/util.rs @@ -167,6 +167,7 @@ pub struct UpdateCounterOptions { pub struct AggregateOptions { pub is_leader: bool, pub update_counter: Option, + pub fast_aggregation: bool, } pub struct Aggregator { @@ -196,7 +197,7 @@ impl IntoFuture for Aggregator { fn into_future(self) -> Self::Future { let Self { options, chans, tx } = self; - let metrics = chans.into_iter().map(|chan| { + let metrics = chans.clone().into_iter().map(|chan| { let (tx, rx) = oneshot::channel(); // TODO: change oneshots to single channel // to do that, task must run in new tokio, then we will not have to pass handle to it @@ -225,31 +226,36 @@ impl IntoFuture for Aggregator { }); let aggregate = accumulate.and_then(move |accumulated| { - accumulated - .into_iter() - .inspect(|_| { EGRESS.fetch_add(1, Ordering::Relaxed); }) - .map(move |(name, metric)| { - let buf = BytesMut::with_capacity(1024); - let task = Task::Aggregate(AggregateData { - buf, - name: Bytes::from(name), - metric, - options: options.clone(), - response: tx.clone(), - }); - // as of now we just run each task in the current thread - // there is a reason we should not in general run the task in the counting workers: - // workers will block on heavy computation and may cause metrics goind to them over - // network to be dropped because of backpressure - // at the same time counting aggregation is not urgent because of current backend(carbon/graphite) - // nature where one can send metrics with any timestamp - // TODO: at some day counting workers will probably work in work-stealing mode, - // after that we probably will be able to run task in common mode - task.run(); - }) - .last(); - Ok(()) - }); + accumulated + .into_iter() + .inspect(|_| { + EGRESS.fetch_add(1, Ordering::Relaxed); + }).enumerate() + .map(move |(num, (name, metric))| { + let buf = BytesMut::with_capacity(1024); + let task = Task::Aggregate(AggregateData { + buf, + name: Bytes::from(name), + metric, + options: options.clone(), + response: tx.clone(), + }); + if options.fast_aggregation { + spawn( + chans[num % chans.len()] + .clone() + .send(task) + .map(|_| ()) + .map_err(|_| { + DROPS.fetch_add(1, Ordering::Relaxed); + }), + ); + } else { + task.run(); + } + }).last(); + Ok(()) + }); Box::new(aggregate) } else { // only get metrics from threads From 8d8ce8580c1e10113daacd32836ee0b1d1832159 Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Thu, 18 Oct 2018 20:45:26 +0300 Subject: [PATCH 12/24] Make internal stats be per second --- src/util.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/util.rs b/src/util.rs index 918d6a9..d881094 100644 --- a/src/util.rs +++ b/src/util.rs @@ -123,7 +123,7 @@ impl OwnStats { add_metric!(PEER_ERRORS, peer_errors, "peer-error"); add_metric!(DROPS, drops, "drop"); if self.interval > 0 { - let s_interval = self.interval as f64; + let s_interval = self.interval as f64 / 1000f64; info!(self.log, "stats"; "egress" => format!("{:2}", egress / s_interval), From 37fe6c2003ba8663c4ff1155864fbcf08570d310 Mon Sep 17 00:00:00 2001 From: Albibek Date: Sun, 28 Oct 2018 16:32:26 +0300 Subject: [PATCH 13/24] Fix parsing metrics with sampling --- Cargo.toml | 2 +- src/main.rs | 96 ++++++++++++++++++++++++++++++++------------------- src/parser.rs | 69 +++++++++++++++++++++++++----------- 3 files changed, 110 insertions(+), 57 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 26fe434..ebdb22a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -18,7 +18,7 @@ tokio-codec="^0.1" bytes = { version = "^0.4", features = [ "serde" ] } resolve="^0.1" net2="^0.2" -combine="^3.3" +combine="^3.6" hyper="^0.12" mime="^0.3" serde="^1.0" diff --git a/src/main.rs b/src/main.rs index 8452f3c..ff01776 100644 --- a/src/main.rs +++ b/src/main.rs @@ -147,12 +147,38 @@ pub enum ConsensusKind { lazy_static! { pub static ref CONSENSUS_STATE: Mutex = - { Mutex::new(ConsensusState::Disabled) }; + { Mutex::new(ConsensusState::Disabled) }; } pub static IS_LEADER: AtomicBool = ATOMIC_BOOL_INIT; fn main() { + { + use std::io; + use std::ffi::CString; + use std::ptr::{null_mut, null}; + use libc::*; + + let domain = CString::new("ya.ru").unwrap().into_raw(); + let mut result: *mut addrinfo = null_mut(); + + unsafe { + getaddrinfo(domain, null_mut(), null(), &mut result); + } + + //let errno = unsafe { *__errno_location() }; + println!("{:?}", io::Error::last_os_error()); + let mut cur = result; + while cur != null_mut() { + unsafe{ + println!("LEN {:?}", (*result).ai_addrlen); + println!("DATA {:?}", (*(*result).ai_addr).sa_data); + cur = (*result).ai_next; + } + } + + } + let (system, command) = System::load(); let System { @@ -174,32 +200,32 @@ fn main() { nodes, snapshot_interval, }, - raft, - consul: - Consul { - start_as: consul_start_as, - agent, - session_ttl: consul_session_ttl, - renew_time: consul_renew_time, - key_name: consul_key, - }, - metrics: - Metrics { - // max_metrics, - mut count_updates, - update_counter_prefix, - update_counter_suffix, - update_counter_threshold, - fast_aggregation, - }, - carbon, - n_threads, - w_threads, - stats_interval: s_interval, - task_queue_size, - start_as_leader, - stats_prefix, - consensus, + raft, + consul: + Consul { + start_as: consul_start_as, + agent, + session_ttl: consul_session_ttl, + renew_time: consul_renew_time, + key_name: consul_key, + }, + metrics: + Metrics { + // max_metrics, + mut count_updates, + update_counter_prefix, + update_counter_suffix, + update_counter_threshold, + fast_aggregation, + }, + carbon, + n_threads, + w_threads, + stats_interval: s_interval, + task_queue_size, + start_as_leader, + stats_prefix, + consensus, } = system.clone(); let verbosity = Level::from_str(&verbosity).expect("bad verbosity"); @@ -280,11 +306,11 @@ fn main() { nodes.clone(), Duration::from_millis(snapshot_interval as u64), &chans, - ).into_future() - .map_err(move |e| { - PEER_ERRORS.fetch_add(1, Ordering::Relaxed); - info!(snap_err_log, "error sending snapshot";"error"=>format!("{}", e)); - }); + ).into_future() + .map_err(move |e| { + PEER_ERRORS.fetch_add(1, Ordering::Relaxed); + info!(snap_err_log, "error sending snapshot";"error"=>format!("{}", e)); + }); runtime.spawn(snapshot); // settings safe for asap restart @@ -324,7 +350,7 @@ fn main() { *con_state = ConsensusState::Enabled; start_internal_raft(raft, consensus_log); Ok(()) - }); + }); runtime.spawn(delayed); } @@ -510,7 +536,7 @@ fn main() { buffer_flush_time, buffer_flush_length, flush_flags.clone(), - ); + ); } else { start_async_udp( log, @@ -522,7 +548,7 @@ fn main() { bufsize, buffer_flush_length, flush_flags.clone(), - ); + ); } runtime diff --git a/src/parser.rs b/src/parser.rs index 1930a59..a8a7fbf 100644 --- a/src/parser.rs +++ b/src/parser.rs @@ -4,10 +4,11 @@ use std::str::from_utf8; use std::str::FromStr; use combine::byte::{byte, bytes, newline}; +use combine::parser::byte::digit; use combine::combinator::{eof, skip_many}; use combine::error::UnexpectedParse; -use combine::parser::range::{take_while, take_while1}; -use combine::{optional, Parser}; +use combine::parser::range::{take_while1, recognize}; +use combine::{optional, Parser, skip_many1}; use metric::MetricType; @@ -16,23 +17,23 @@ use metric::MetricType; pub fn metric_parser<'a, F>( ) -> impl Parser, Option), Input = &'a [u8]> where - F: FromStr - + Add - + AddAssign - + Sub - + SubAssign - + Div - + Mul - + Neg - + PartialOrd - + Into - + From - + Debug - + Default - + Clone - + Copy - + PartialEq - + Sync, +F: FromStr ++ Add ++ AddAssign ++ Sub ++ SubAssign ++ Div ++ Mul ++ Neg ++ PartialOrd ++ Into ++ From ++ Debug ++ Default ++ Clone ++ Copy ++ PartialEq ++ Sync, { // This will parse metric name and separator let name = take_while1(|c: u8| c != b':' && c != b'\n').skip(byte(b':')); @@ -57,7 +58,15 @@ where // we can add more types here // .or(byte(b'h').map(|_| MetricType::Histrogram)) ; - let sampling = (bytes(b"|@"), take_while(|c: u8| c != b'\n')).and_then(|(_, value)| { + + let unsigned_float = skip_many1(digit()).and(optional( + (byte(b'.'), skip_many1(digit())) + )).and(optional( + (byte(b'e'), optional(byte(b'+').or(byte(b'-'))), skip_many1(digit())) + )); + + let sampling = (bytes(b"|@"), recognize(unsigned_float)).and_then(|(_, value)| { + // TODO replace from_utf8 with handmade parser removing recognize from_utf8(value) .map_err(|_e| UnexpectedParse::Unexpected) .map(|v| v.parse::().map_err(|_e| UnexpectedParse::Unexpected))? @@ -69,7 +78,7 @@ where mtype, optional(sampling), skip_many(newline()).or(eof()), - ) + ) .and_then(|(name, sign, mut value, mtype, sampling, _)| { let mtype = if let MetricType::Gauge(_) = mtype { MetricType::Gauge(sign) @@ -143,6 +152,24 @@ mod tests { assert_eq!(rest.len(), 0); } + #[test] + fn parse_metric_without_newline_sampling() { + let data = b"gorets:+1000|g|@0.4e-3gorets:-1000|g|@0.5"; + let mut parser = metric_parser::(); + let (v, rest) = parser.parse(data).unwrap(); + assert_eq!(v.0, b"gorets"); + assert_eq!(v.1, 1000f64); + assert_eq!(v.2, MetricType::Gauge(Some(1i8))); + assert_eq!(v.3, Some(0.0004f32)); + //assert_neq!(rest.len(), 0) + let (v, rest) = parser.parse(rest).unwrap(); + assert_eq!(v.0, b"gorets"); + assert_eq!(v.1, 1000f64); + assert_eq!(v.2, MetricType::Gauge(Some(-1i8))); + assert_eq!(v.3, Some(0.5f32)); + assert_eq!(rest.len(), 0) + } + #[test] fn parse_metric_short() { let data = b"gorets:1|c"; From 76dbd4f3b583cf96e00bb87a7e91158d4636dbaf Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Wed, 31 Oct 2018 10:53:42 +0300 Subject: [PATCH 14/24] Move aggregate task to new tokio --- src/carbon.rs | 22 +++++++++++---- src/config.rs | 1 - src/main.rs | 44 ++++++++++++++++++------------ src/task.rs | 25 ++++++++--------- src/util.rs | 75 +++++++++++++++++++++++++++++++++++++++++---------- 5 files changed, 116 insertions(+), 51 deletions(-) diff --git a/src/carbon.rs b/src/carbon.rs index e0324f5..f424f29 100644 --- a/src/carbon.rs +++ b/src/carbon.rs @@ -8,6 +8,7 @@ use failure::Error; use ftoa; use futures::stream; use futures::{Future, IntoFuture, Sink, Stream}; +use slog::Logger; use tokio::net::TcpStream; use tokio_codec::{Decoder, Encoder}; @@ -18,12 +19,17 @@ use {Float, AGG_ERRORS}; #[derive(Clone)] pub struct CarbonBackend { addr: SocketAddr, - metrics: Arc>, + log: Logger, } impl CarbonBackend { - pub(crate) fn new(addr: SocketAddr, ts: Duration, metrics: Arc>) -> Self { + pub(crate) fn new( + addr: SocketAddr, + ts: Duration, + metrics: Arc>, + log: Logger, + ) -> Self { let ts: Bytes = ts.as_secs().to_string().into(); let buf = BytesMut::with_capacity(metrics.len() * 200); // 200 is an approximate for full metric name + value @@ -47,7 +53,7 @@ impl CarbonBackend { (acc, buf) }); let metrics = Arc::new(metrics); - let self_ = Self { addr, metrics }; + let self_ = Self { addr, metrics, log }; self_ } } @@ -58,16 +64,22 @@ impl IntoFuture for CarbonBackend { type Future = Box>; fn into_future(self) -> Self::Future { - let Self { addr, metrics } = self; + let Self { addr, metrics, log } = self; let conn = TcpStream::connect(&addr).map_err(|e| GeneralError::Io(e)); + let elog = log.clone(); let future = conn.and_then(move |conn| { + info!(log, "carbon backend sending metrics"); let writer = CarbonCodec::new().framed(conn); let metric_stream = stream::iter_ok::<_, ()>(SharedIter::new(metrics)); metric_stream .map_err(|_| GeneralError::CarbonBackend) .forward(writer.sink_map_err(|_| GeneralError::CarbonBackend)) - .map(|_| ()) + .map(move |_| info!(log, "carbon backend finished")) + .map_err(move |e| { + info!(elog, "carbon backend error"); + e + }) }); Box::new(future) diff --git a/src/config.rs b/src/config.rs index 9b39ae5..1db1192 100644 --- a/src/config.rs +++ b/src/config.rs @@ -96,7 +96,6 @@ pub(crate) struct Metrics { /// Aggregate faster at the price or probably loosing some incoming metrics pub fast_aggregation: bool, - // TODO // /// Whether we should spam parsing errors in logs // pub log_parse_errors: bool, diff --git a/src/main.rs b/src/main.rs index ff01776..0319c80 100644 --- a/src/main.rs +++ b/src/main.rs @@ -337,22 +337,32 @@ fn main() { match consensus { ConsensusKind::Internal => { - if start_as_leader { - warn!(log, "Starting as leader with enabled consensus. More that one leader is possible before consensus settle up."); - } - let d = Delay::new(Instant::now() + Duration::from_millis(raft.start_delay)); let log = log.clone(); - let delayed = d - .map_err(|_| ()) - .and_then(move |_| { - let mut con_state = CONSENSUS_STATE.lock().unwrap(); - info!(log, "starting internal consensus"; "initial_state"=>format!("{:?}", con_state)); - *con_state = ConsensusState::Enabled; - start_internal_raft(raft, consensus_log); - Ok(()) + let flog = log.clone(); + thread::Builder::new() + .name("bioyino_raft".into()) + .spawn(move || { + let mut runtime = Runtime::new().expect("creating runtime for raft thread"); + if start_as_leader { + warn!(log, "Starting as leader with enabled consensus. More that one leader is possible before consensus settle up."); + } + let d = Delay::new(Instant::now() + Duration::from_millis(raft.start_delay)); + let log = log.clone(); + let delayed = d + .map_err(|_| ()) + .and_then(move |_| { + let mut con_state = CONSENSUS_STATE.lock().unwrap(); + *con_state = ConsensusState::Enabled; + info!(log, "starting internal consensus"; "initial_state"=>format!("{:?}", *con_state)); + start_internal_raft(raft, consensus_log); + Ok(()) }); - runtime.spawn(delayed); + runtime.spawn(delayed); + runtime.block_on(empty::<(), ()>()).expect("raft thread failed"); + + info!(flog, "consensus thread stopped"); + }).expect("starting counting worker thread"); } ConsensusKind::Consul => { if start_as_leader { @@ -440,7 +450,7 @@ fn main() { if is_leader { info!(carbon_log, "leader sending metrics"); let (backend_tx, backend_rx) = mpsc::unbounded(); - let aggregator = Aggregator::new(options, tchans, backend_tx).into_future(); + let aggregator = Aggregator::new(options, tchans, backend_tx, carbon_log.clone()).into_future(); runtime.spawn(aggregator); @@ -448,7 +458,7 @@ fn main() { .inspect(|_| { EGRESS.fetch_add(1, Ordering::Relaxed); }) .collect() .and_then(|metrics| { - let backend = CarbonBackend::new(backend_addr, ts, Arc::new(metrics)); + let backend = CarbonBackend::new(backend_addr, ts, Arc::new(metrics), carbon_log.clone()); let retrier = BackoffRetryBuilder { delay: backend_opts.connect_delay, @@ -467,7 +477,7 @@ fn main() { } else { info!(carbon_log, "not leader, removing metrics"); let (backend_tx, _) = mpsc::unbounded(); - let aggregator = Aggregator::new(options, tchans, backend_tx).into_future(); + let aggregator = Aggregator::new(options, tchans, backend_tx, carbon_log.clone()).into_future(); runtime .block_on(aggregator.then(|_| Ok::<(), ()>(()))) .unwrap_or_else(|e| { @@ -510,7 +520,7 @@ fn main() { let flush_timer = flush_timer .map_err(|e| GeneralError::Timer(e)) .for_each(move |_tick| { - debug!(tlog, "buffer flush requested"); + info!(tlog, "buffer flush requested"); flags .iter() .map(|flag| flag.swap(true, Ordering::SeqCst)) diff --git a/src/task.rs b/src/task.rs index 0b26393..4c7a96b 100644 --- a/src/task.rs +++ b/src/task.rs @@ -5,7 +5,8 @@ use bytes::{BufMut, Bytes, BytesMut}; use combine::Parser; use futures::sync::mpsc::UnboundedSender; use futures::sync::oneshot; -use futures::Sink; +use futures::{Future, Sink}; +use tokio::runtime::current_thread::spawn; use metric::Metric; use parser::metric_parser; @@ -109,7 +110,7 @@ impl Task { name, metric, options, - mut response, + response, }) => { let upd = if let Some(options) = options.update_counter { if metric.update_counter > options.threshold { @@ -145,19 +146,15 @@ impl Task { (name, value) }).chain(upd) .map(|data| { - response - .start_send(data) - .map_err(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }).map(|_| ()) - .unwrap_or(()); + spawn( + response + .clone() + .send(data) + .map_err(|_| { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()), + ); }).last(); - response - .poll_complete() - .map_err(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }).map(|_| ()) - .unwrap_or_else(|_| ()); } } } diff --git a/src/util.rs b/src/util.rs index d881094..95b3f8f 100644 --- a/src/util.rs +++ b/src/util.rs @@ -108,9 +108,7 @@ impl OwnStats { .clone() .send(Task::AddMetric(name, metric)) .map(|_| ()) - .map_err(move |_| { - warn!(log, "stats future could not send metric to task") - }); + .map_err(move |_| warn!(log, "stats future could not send metric to task")); spawn(sender); } }; @@ -178,6 +176,7 @@ pub struct Aggregator { // a channel(supposedly from a backend) where we pass aggregated metrics to // TODO this probably needs to be generic stream tx: UnboundedSender<(Bytes, Float)>, + log: Logger, } impl Aggregator { @@ -185,8 +184,14 @@ impl Aggregator { options: AggregateOptions, chans: Vec>, tx: UnboundedSender<(Bytes, Float)>, + log: Logger, ) -> Self { - Self { options, chans, tx } + Self { + options, + chans, + tx, + log, + } } } @@ -196,7 +201,13 @@ impl IntoFuture for Aggregator { type Future = Box>; fn into_future(self) -> Self::Future { - let Self { options, chans, tx } = self; + let Self { + options, + chans, + tx, + log, + } = self; + info!(log, "leader accumulating metrics"); let metrics = chans.clone().into_iter().map(|chan| { let (tx, rx) = oneshot::channel(); // TODO: change oneshots to single channel @@ -208,24 +219,27 @@ impl IntoFuture for Aggregator { }); if options.is_leader { - let accumulate = futures_unordered(metrics)//.for_each(|| { - .fold(HashMap::new(), move |mut acc: Cache, metrics| { + let accumulate = + futures_unordered(metrics).fold(HashMap::new(), move |mut acc: Cache, metrics| { metrics .into_iter() .map(|(name, metric)| { if acc.contains_key(&name) { - acc.get_mut(&name).unwrap() - .aggregate(metric).unwrap_or_else(|_| {AGG_ERRORS.fetch_add(1, Ordering::Relaxed);}); + acc.get_mut(&name) + .unwrap() + .aggregate(metric) + .unwrap_or_else(|_| { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + }); } else { acc.insert(name, metric); } - }) - .last(); + }).last(); Ok(acc) - // }) - }); + }); let aggregate = accumulate.and_then(move |accumulated| { + debug!(log, "leader aggregating metrics"); accumulated .into_iter() .inspect(|_| { @@ -255,7 +269,40 @@ impl IntoFuture for Aggregator { } }).last(); Ok(()) - }); + //}); + /* + * TODO: this was an expermient for multithreade aggregation in separate threadpol with rayon + * it worked, but needs more work to be in prod + let aggregate = accumulate.and_then(move |accumulated| { + info!(log, "leader aggregating metrics"); + use rayon::iter::IntoParallelIterator; + use rayon::iter::ParallelIterator; + use rayon::ThreadPoolBuilder; + + let pool = ThreadPoolBuilder::new() + .thread_name(|i| format!("bioyino_crb{}", i).into()) + .num_threads(8) + .build() + .unwrap(); + pool.install(|| { + accumulated + .into_par_iter() + .inspect(|_| { + EGRESS.fetch_add(1, Ordering::Relaxed); + }).for_each(move |(name, metric)| { + let buf = BytesMut::with_capacity(1024); + let task = Task::Aggregate(AggregateData { + buf, + name: Bytes::from(name), + metric, + options: options.clone(), + response: tx.clone(), + }); + task.run(); + }); //.last(); + }); + Ok(()) + */ }); Box::new(aggregate) } else { // only get metrics from threads From b253702b21436be72e99280326e3d4d5cf4b6fcf Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Wed, 31 Oct 2018 13:47:31 +0300 Subject: [PATCH 15/24] Remove experimental code --- src/main.rs | 50 +++++++++++++++++++++++++------------------------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/src/main.rs b/src/main.rs index 0319c80..5fa92ba 100644 --- a/src/main.rs +++ b/src/main.rs @@ -153,31 +153,31 @@ lazy_static! { pub static IS_LEADER: AtomicBool = ATOMIC_BOOL_INIT; fn main() { - { - use std::io; - use std::ffi::CString; - use std::ptr::{null_mut, null}; - use libc::*; - - let domain = CString::new("ya.ru").unwrap().into_raw(); - let mut result: *mut addrinfo = null_mut(); - - unsafe { - getaddrinfo(domain, null_mut(), null(), &mut result); - } - - //let errno = unsafe { *__errno_location() }; - println!("{:?}", io::Error::last_os_error()); - let mut cur = result; - while cur != null_mut() { - unsafe{ - println!("LEN {:?}", (*result).ai_addrlen); - println!("DATA {:?}", (*(*result).ai_addr).sa_data); - cur = (*result).ai_next; - } - } - - } + // { + //use std::io; + //use std::ffi::CString; + //use std::ptr::{null_mut, null}; + //use libc::*; + + //let domain = CString::new("ya.ru").unwrap().into_raw(); + //let mut result: *mut addrinfo = null_mut(); + + //unsafe { + //getaddrinfo(domain, null_mut(), null(), &mut result); + //} + + ////let errno = unsafe { *__errno_location() }; + //println!("{:?}", io::Error::last_os_error()); + //let mut cur = result; + //while cur != null_mut() { + //unsafe{ + //println!("LEN {:?}", (*result).ai_addrlen); + //println!("DATA {:?}", (*(*result).ai_addr).sa_data); + //cur = (*result).ai_next; + //} + //} + + //} let (system, command) = System::load(); From aba8fa2f10c9ec31d4d7c59c6b3a86667bd44802 Mon Sep 17 00:00:00 2001 From: Albibek Date: Tue, 6 Nov 2018 00:27:10 +0300 Subject: [PATCH 16/24] Better buffer handling on receive side --- src/server.rs | 83 +++++++++++++++---------- src/task.rs | 30 +++++---- src/udp.rs | 166 +++++++++++++++++++++++++++++++------------------- src/util.rs | 134 ++++++++++++++++++++-------------------- 4 files changed, 238 insertions(+), 175 deletions(-) diff --git a/src/server.rs b/src/server.rs index edd83f8..4715173 100644 --- a/src/server.rs +++ b/src/server.rs @@ -1,5 +1,9 @@ use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; +use std::collections::HashMap; +use std::collections::hash_map::DefaultHasher; +use std::hash::{Hasher, Hash}; +use std::net::SocketAddr; use {DROPS, INGRESS}; @@ -15,9 +19,10 @@ use task::Task; pub struct StatsdServer { socket: UdpSocket, chans: Vec>, - buf: BytesMut, + bufmap: HashMap, buffer_flush_length: usize, bufsize: usize, + recv_counter: usize, next: usize, readbuf: BytesMut, flush_flags: Arc>, @@ -28,20 +33,22 @@ impl StatsdServer { pub fn new( socket: UdpSocket, chans: Vec>, - buf: BytesMut, + bufmap: HashMap, buffer_flush_length: usize, bufsize: usize, + recv_counter: usize, next: usize, readbuf: BytesMut, flush_flags: Arc>, thread_idx: usize, - ) -> Self { + ) -> Self { Self { socket, chans, - buf, + bufmap, buffer_flush_length, bufsize, + recv_counter, next, readbuf, flush_flags, @@ -59,9 +66,10 @@ impl IntoFuture for StatsdServer { let Self { socket, chans, - mut buf, + mut bufmap, buffer_flush_length, bufsize, + mut recv_counter, next, readbuf, flush_flags, @@ -71,58 +79,69 @@ impl IntoFuture for StatsdServer { let future = socket .recv_dgram(readbuf) .map_err(|e| println!("error receiving UDP packet {:?}", e)) - .and_then(move |(socket, received, size, _addr)| { + .and_then(move |(socket, received, size, addr)| { INGRESS.fetch_add(1, Ordering::Relaxed); if size == 0 { return Ok(()); } - buf.put(&received[0..size]); + { + let buf = bufmap.entry(addr).or_insert(BytesMut::with_capacity(buffer_flush_length)); + recv_counter += size; + buf.put(&received[0..size]); + } let flush = flush_flags .get(thread_idx) .unwrap() .swap(false, Ordering::SeqCst); - if buf.remaining_mut() < bufsize || buf.len() >= buffer_flush_length || flush { - let (chan, next) = if next >= chans.len() { - (chans[0].clone(), 1) - } else { - (chans[next].clone(), next + 1) - }; - let newbuf = BytesMut::with_capacity(buffer_flush_length); + if recv_counter >= buffer_flush_length || flush { - spawn( - chan.send(Task::Parse(buf.freeze())) + bufmap.drain().map(|(addr, buf)|{ + let mut hasher = DefaultHasher::new(); + addr.hash(&mut hasher); + let ahash = hasher.finish(); + let (chan, next) = if next >= chans.len() { + (chans[0].clone(), 1) + } else { + (chans[next].clone(), next + 1) + }; + + spawn( + chan.send(Task::Parse(ahash, buf.freeze())) .map_err(|_| { DROPS.fetch_add(1, Ordering::Relaxed); - }).and_then(move |_| { - StatsdServer::new( - socket, - chans, - newbuf, - buffer_flush_length, - bufsize, - next, - received, - flush_flags, - thread_idx, - ).into_future() - }), + }).map(|_|())) + }).last(); + + spawn(StatsdServer::new( + socket, + chans, + bufmap, + buffer_flush_length, + bufsize, + 0, + next, + received, + flush_flags, + thread_idx, + ).into_future() ); } else { spawn( StatsdServer::new( socket, chans, - buf, + bufmap, buffer_flush_length, bufsize, + recv_counter, next, received, flush_flags, thread_idx, - ).into_future(), - ); + ).into_future(), + ); } Ok(()) }); diff --git a/src/task.rs b/src/task.rs index 4c7a96b..fce4abb 100644 --- a/src/task.rs +++ b/src/task.rs @@ -3,6 +3,7 @@ use std::sync::atomic::Ordering; use bytes::{BufMut, Bytes, BytesMut}; use combine::Parser; +use combine::error::UnexpectedParse; use futures::sync::mpsc::UnboundedSender; use futures::sync::oneshot; use futures::{Future, Sink}; @@ -28,7 +29,7 @@ pub struct AggregateData { #[derive(Debug)] pub enum Task { - Parse(Bytes), + Parse(u64, Bytes), AddMetric(Bytes, Metric), AddMetrics(Vec<(Bytes, Metric)>), AddSnapshot(Vec<(Bytes, Metric)>), @@ -53,7 +54,7 @@ fn update_metric(cache: &mut Cache, name: Bytes, metric: Metric) { impl Task { pub fn run(self) { match self { - Task::Parse(buf) => parse_and_insert(buf), + Task::Parse(addr, buf) => parse_and_insert(addr, buf), Task::AddMetric(name, metric) => SHORT_CACHE.with(move |c| { let mut short = c.borrow_mut(); update_metric(&mut short, name, metric); @@ -145,16 +146,16 @@ impl Task { let name = buf.take().freeze(); (name, value) }).chain(upd) - .map(|data| { - spawn( - response - .clone() - .send(data) - .map_err(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }).map(|_| ()), + .map(|data| { + spawn( + response + .clone() + .send(data) + .map_err(|_| { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()), ); - }).last(); + }).last(); } } } @@ -172,7 +173,7 @@ fn cut_bad(buf: &mut Bytes) -> Option { } } -fn parse_and_insert(mut buf: Bytes) { +fn parse_and_insert(_addr: u64, mut buf: Bytes) { // Cloned buf is shallow copy, so input and buf are the same bytes. // We are going to parse the whole slice, so for parser we use input as readonly // while buf follows the parser progress and is cut to get only names @@ -219,6 +220,9 @@ fn parse_and_insert(mut buf: Bytes) { break; } } + Err(UnexpectedParse::Eoi) => { + break; + } Err(_e) => { if let Some(pos) = cut_bad(&mut buf) { input = input.split_at(pos + 1).1; @@ -241,7 +245,7 @@ mod tests { let mut data = Bytes::new(); data.extend_from_slice( b"trash\ngorets1:+1000|g\nTRASH\ngorets2:-1000|g|@0.5\nMORETrasH\nFUUU", - ); + ); parse_and_insert(data); diff --git a/src/udp.rs b/src/udp.rs index 20d7bb4..ccf2a6b 100644 --- a/src/udp.rs +++ b/src/udp.rs @@ -4,6 +4,9 @@ use std::ptr::null_mut; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::thread; +use std::collections::HashMap; +use std::collections::hash_map::DefaultHasher; +use std::hash::Hasher; use server::StatsdServer; use task::Task; @@ -32,8 +35,8 @@ pub(crate) fn start_sync_udp( buffer_flush_time: u64, buffer_flush_length: usize, flush_flags: Arc>, -) { - info!(log, "multimessage enabled, starting in sync UDP mode"; "socket-is-blocking"=>mm_async, "packets"=>mm_packets); + ) { + info!(log, "multimessage enabled, starting in sync UDP mode"; "socket-is-blocking"=>!mm_async, "packets"=>mm_packets); // It is crucial for recvmmsg to have one socket per many threads // to avoid drops because at lease two threads have to work on socket @@ -61,58 +64,82 @@ pub(crate) fn start_sync_udp( .spawn(move || { let fd = sck.as_raw_fd(); { - // <--- this limits `use::libc::*` scope + // <--- this limits the use of `use::libc::*` scope use libc::*; + let chlen = chans.len(); let mut ichans = chans.iter().cycle(); - // a vector to avoid dropping iovec structures - let mut iovecs = Vec::with_capacity(mm_packets); + // store mmsghdr array so Rust won't free it's memory + let mut mheaders: Vec = Vec::with_capacity(mm_packets); - // a vector to avoid dropping message buffers - let mut message_vec = Vec::new(); + // allocate space for address information + // addr len is 16 bytes for ipv6 address + 4 bytes for port totalling 20 bytes + // the structure is only used for advanced information and not copied anywhere + // so it can be initialized before main cycle + let mut addrs: Vec<[u8;20]> = Vec::with_capacity(mm_packets); + addrs.resize(mm_packets, [0; 20]); - let mut v: Vec = Vec::with_capacity(mm_packets); - for _ in 0..mm_packets { - let mut buf = Vec::with_capacity(bufsize); - buf.resize(bufsize, 0); - - let mut iov = Vec::with_capacity(1); - iov.resize( - 1, - iovec { - iov_base: buf.as_mut_ptr() as *mut c_void, - iov_len: bufsize as size_t, - }, - ); + for i in 0..mm_packets { let m = mmsghdr { msg_hdr: msghdr { - msg_name: null_mut(), - msg_namelen: 0 as socklen_t, - msg_iov: iov.as_mut_ptr(), - msg_iovlen: iov.len() as size_t, - msg_control: null_mut(), - msg_controllen: 0, - msg_flags: 0, + msg_name: addrs[i].as_mut_ptr() as *mut c_void, + msg_namelen: 20 as socklen_t, + msg_iov: null_mut(), // this will change later + msg_iovlen: 0, // this will change later + msg_control: null_mut(), // we won't need this + msg_controllen: 0, // and this + msg_flags: 0, // and of couse this }, msg_len: 0, }; - v.push(m); - iovecs.push(iov); - message_vec.push(buf); + mheaders.push(m); } - let vp = v.as_mut_ptr(); - let vlen = v.len(); - - // This is the buffer we fill with metrics and periodically send to tasks - // Due to implementation of BytesMut it's real size is bigger, so for even less - // allocations, we split the filled part and leave the rest for future bytes - let mut b = BytesMut::with_capacity(buffer_flush_length); + let mhptr = mheaders.as_mut_ptr(); + let mhlen = mheaders.len(); let flags = if mm_async { MSG_WAITFORONE } else { 0 }; + // this will store resulting per-source buffers + let mut bufmap = HashMap::new(); + let mut total_received = 0; + let min_bytes = mm_packets*mm_packets*bufsize; + let rowsize = bufsize*mm_packets; + + let mut recv_buffer = Vec::new(); + recv_buffer.reserve_exact(min_bytes); + + // prepare scatter-gather buffers (iovecs) + // We allocate (mm_packets x mm_packets*bufsize) matrix to guarantee fitting of all + // the messages into memory. For doing this se have to consider 2 edge cases here. + // We know that recvmmsg places all messages from a single source to + // the same iovecs bucket. That is the first case is when all data come from + // single address, so we will have row filled with bytes. The second case is when + // all data come from different addresses, so buffers are filled in + // columns. The default value - 1500 does not consider IP fragmentation here, so the ideal + // value would be maximum IP packet size (~ 65535 - 8 = 65507), but this is the + // rare case in modern networks, at least at datacenters, which are our + // main use case. + + recv_buffer.resize(min_bytes, 0); + // we don't want rust to forget about intermediate iovecs so we put them into + // separate vector + let mut chunks = Vec::with_capacity(mm_packets); + + for i in 0..mm_packets { + let mut chunk = iovec { + iov_base: recv_buffer[i*rowsize..i*rowsize+rowsize].as_mut_ptr() as *mut c_void, + iov_len: rowsize + }; + chunks.push(chunk); + // put the result to mheaders + mheaders[i].msg_hdr.msg_iov = &mut chunks[i]; + mheaders[i].msg_hdr.msg_iovlen = 1; + } + loop { + debug!(log, "recvmsg start"); // timeout is mutable and changed by every recvmmsg call, so it MUST be inside loop // creating timeout as &mut fails because it's supposedly not dropped let mut timeout = if mm_timeout > 0 { @@ -128,39 +155,52 @@ pub(crate) fn start_sync_udp( }; let res = - unsafe { recvmmsg(fd as c_int, vp, vlen as c_uint, flags, if mm_timeout > 0 {&mut timeout} else {null_mut()}) }; + unsafe { recvmmsg(fd as c_int, mhptr, mhlen as c_uint, flags, if mm_timeout > 0 {&mut timeout} else {null_mut()}) }; if res == 0 { // skip this shit } else if res > 0 { let messages = res as usize; - // Check if we can fit all packets into buffer - let mut total_bytes = 0; + // we've received some messages for i in 0..messages { - total_bytes += v[i].msg_len as usize; - } + let mlen = mheaders[i].msg_len as usize; + total_received += mlen; - // if we cannot, allocate more - if b.remaining_mut() < total_bytes { - b.reserve(buffer_flush_length) - } + // create address entry in messagemap + let mut entry = bufmap.entry(addrs[i]).or_insert(BytesMut::with_capacity(buffer_flush_length)); + // and put it's buffer there + entry.put(&recv_buffer[i*rowsize..i*rowsize+mlen]); - // put packets into buffer - for i in 0..messages { - let len = v[i].msg_len as usize; - b.put(&message_vec[i][0..len]); + // reset addres to be used in next cycle + addrs[i] = [0; 20]; + mheaders[i].msg_hdr.msg_namelen = 20; } + INGRESS.fetch_add(total_received, Ordering::Relaxed); + + let consistent_parsing = true; // when it's time to send bytes, send them let flush = flush_flags.get(i).unwrap().swap(false, Ordering::SeqCst); - if flush || b.len() >= buffer_flush_length { - let mut chan = ichans.next().unwrap().clone(); - INGRESS.fetch_add(messages as usize, Ordering::Relaxed); - chan.try_send(Task::Parse(b.take().freeze())) - .map_err(|_| { - warn!(log, "error sending buffer(queue full?)"); - DROPS.fetch_add(messages as usize, Ordering::Relaxed); - }).unwrap_or(()); + if flush || total_received >= buffer_flush_length { + total_received = 0; + bufmap.drain().map(|(addr, mut buf)|{ + // in some ideal world we want all values from the same host to be parsed by the + // same thread, but this could cause load unbalancing between + // threads TODO: make it an option in config + let mut hasher = DefaultHasher::new(); + hasher.write(&addr); + let ahash = hasher.finish(); + let mut chan = if consistent_parsing { + chans[ahash as usize % chlen].clone() + } else { + ichans.next().unwrap().clone() + }; + chan.try_send(Task::Parse(ahash, buf.take().freeze())) + .map_err(|_| { + warn!(log, "error sending buffer(queue full?)"); + DROPS.fetch_add(messages as usize, Ordering::Relaxed); + }).unwrap_or(()); + }).last(); } } else { let errno = unsafe { *__errno_location() }; @@ -188,7 +228,7 @@ pub(crate) fn start_async_udp( bufsize: usize, buffer_flush_length: usize, flush_flags: Arc>, -) { + ) { info!(log, "multimessage is disabled, starting in async UDP mode"); // Create a pool of listener sockets @@ -220,7 +260,6 @@ pub(crate) fn start_async_udp( for _ in 0..greens { // start a listener for all sockets for socket in sockets.iter() { - let buf = BytesMut::with_capacity(buffer_flush_length); let mut readbuf = BytesMut::with_capacity(bufsize); unsafe { readbuf.set_len(bufsize) } @@ -229,19 +268,20 @@ pub(crate) fn start_async_udp( let socket = socket.try_clone().expect("cloning socket"); let socket = UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()) - .expect("adding socket to event loop"); + .expect("adding socket to event loop"); let server = StatsdServer::new( socket, chans.clone(), - buf, + HashMap::new(), buffer_flush_length, bufsize, + 0, i, readbuf, flush_flags.clone(), i, - ); + ); runtime.spawn(server.into_future()); } diff --git a/src/util.rs b/src/util.rs index 95b3f8f..c19481f 100644 --- a/src/util.rs +++ b/src/util.rs @@ -185,7 +185,7 @@ impl Aggregator { chans: Vec>, tx: UnboundedSender<(Bytes, Float)>, log: Logger, - ) -> Self { + ) -> Self { Self { options, chans, @@ -207,7 +207,6 @@ impl IntoFuture for Aggregator { tx, log, } = self; - info!(log, "leader accumulating metrics"); let metrics = chans.clone().into_iter().map(|chan| { let (tx, rx) = oneshot::channel(); // TODO: change oneshots to single channel @@ -219,6 +218,7 @@ impl IntoFuture for Aggregator { }); if options.is_leader { + info!(log, "leader accumulating metrics"); let accumulate = futures_unordered(metrics).fold(HashMap::new(), move |mut acc: Cache, metrics| { metrics @@ -245,65 +245,65 @@ impl IntoFuture for Aggregator { .inspect(|_| { EGRESS.fetch_add(1, Ordering::Relaxed); }).enumerate() - .map(move |(num, (name, metric))| { - let buf = BytesMut::with_capacity(1024); - let task = Task::Aggregate(AggregateData { - buf, - name: Bytes::from(name), - metric, - options: options.clone(), - response: tx.clone(), - }); - if options.fast_aggregation { - spawn( - chans[num % chans.len()] - .clone() - .send(task) - .map(|_| ()) - .map_err(|_| { - DROPS.fetch_add(1, Ordering::Relaxed); - }), + .map(move |(num, (name, metric))| { + let buf = BytesMut::with_capacity(1024); + let task = Task::Aggregate(AggregateData { + buf, + name: Bytes::from(name), + metric, + options: options.clone(), + response: tx.clone(), + }); + if options.fast_aggregation { + spawn( + chans[num % chans.len()] + .clone() + .send(task) + .map(|_| ()) + .map_err(|_| { + DROPS.fetch_add(1, Ordering::Relaxed); + }), ); - } else { - task.run(); - } - }).last(); + } else { + task.run(); + } + }).last(); Ok(()) - //}); - /* - * TODO: this was an expermient for multithreade aggregation in separate threadpol with rayon - * it worked, but needs more work to be in prod - let aggregate = accumulate.and_then(move |accumulated| { - info!(log, "leader aggregating metrics"); - use rayon::iter::IntoParallelIterator; - use rayon::iter::ParallelIterator; - use rayon::ThreadPoolBuilder; + //}); + /* + * TODO: this was an expermient for multithreade aggregation in separate threadpol with rayon + * it worked, but needs more work to be in prod + let aggregate = accumulate.and_then(move |accumulated| { + info!(log, "leader aggregating metrics"); + use rayon::iter::IntoParallelIterator; + use rayon::iter::ParallelIterator; + use rayon::ThreadPoolBuilder; - let pool = ThreadPoolBuilder::new() - .thread_name(|i| format!("bioyino_crb{}", i).into()) - .num_threads(8) - .build() - .unwrap(); - pool.install(|| { - accumulated - .into_par_iter() - .inspect(|_| { - EGRESS.fetch_add(1, Ordering::Relaxed); - }).for_each(move |(name, metric)| { - let buf = BytesMut::with_capacity(1024); - let task = Task::Aggregate(AggregateData { - buf, - name: Bytes::from(name), - metric, - options: options.clone(), - response: tx.clone(), - }); - task.run(); - }); //.last(); - }); - Ok(()) - */ }); - Box::new(aggregate) + let pool = ThreadPoolBuilder::new() + .thread_name(|i| format!("bioyino_crb{}", i).into()) + .num_threads(8) + .build() + .unwrap(); + pool.install(|| { + accumulated + .into_par_iter() + .inspect(|_| { + EGRESS.fetch_add(1, Ordering::Relaxed); + }).for_each(move |(name, metric)| { + let buf = BytesMut::with_capacity(1024); + let task = Task::Aggregate(AggregateData { + buf, + name: Bytes::from(name), + metric, + options: options.clone(), + response: tx.clone(), + }); + task.run(); + }); //.last(); + }); + Ok(()) + */ }); + Box::new(aggregate) } else { // only get metrics from threads let not_leader = futures_unordered(metrics).for_each(|_| Ok(())); @@ -333,16 +333,16 @@ impl Default for BackoffRetryBuilder { impl BackoffRetryBuilder { pub fn spawn(self, action: F) -> BackoffRetry - where + where F: IntoFuture + Clone, - { - let inner = Either::A(action.clone().into_future()); - BackoffRetry { - action, - inner: inner, - options: self, + { + let inner = Either::A(action.clone().into_future()); + BackoffRetry { + action, + inner: inner, + options: self, + } } - } } /// TCP client that is able to reconnect with customizable settings @@ -354,7 +354,7 @@ pub struct BackoffRetry { impl Future for BackoffRetry where - F: IntoFuture + Clone, +F: IntoFuture + Clone, { type Item = F::Item; type Error = Option; From 7f5d9f92da75e54454c10d64d3b878008b203fba Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Tue, 6 Nov 2018 10:50:12 +0300 Subject: [PATCH 17/24] Fix ingress stats, add persistent parsing in async mode --- src/server.rs | 69 ++++++++++++++++++++------------- src/udp.rs | 104 +++++++++++++++++++++++++++++--------------------- 2 files changed, 103 insertions(+), 70 deletions(-) diff --git a/src/server.rs b/src/server.rs index 4715173..e552868 100644 --- a/src/server.rs +++ b/src/server.rs @@ -1,9 +1,9 @@ -use std::sync::atomic::{AtomicBool, Ordering}; -use std::sync::Arc; -use std::collections::HashMap; use std::collections::hash_map::DefaultHasher; -use std::hash::{Hasher, Hash}; +use std::collections::HashMap; +use std::hash::{Hash, Hasher}; use std::net::SocketAddr; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::Arc; use {DROPS, INGRESS}; @@ -41,7 +41,7 @@ impl StatsdServer { readbuf: BytesMut, flush_flags: Arc>, thread_idx: usize, - ) -> Self { + ) -> Self { Self { socket, chans, @@ -70,7 +70,7 @@ impl IntoFuture for StatsdServer { buffer_flush_length, bufsize, mut recv_counter, - next, + mut next, readbuf, flush_flags, thread_idx, @@ -86,7 +86,9 @@ impl IntoFuture for StatsdServer { } { - let buf = bufmap.entry(addr).or_insert(BytesMut::with_capacity(buffer_flush_length)); + let buf = bufmap + .entry(addr) + .or_insert(BytesMut::with_capacity(buffer_flush_length)); recv_counter += size; buf.put(&received[0..size]); } @@ -95,26 +97,39 @@ impl IntoFuture for StatsdServer { .get(thread_idx) .unwrap() .swap(false, Ordering::SeqCst); - if recv_counter >= buffer_flush_length || flush { - bufmap.drain().map(|(addr, buf)|{ - let mut hasher = DefaultHasher::new(); - addr.hash(&mut hasher); - let ahash = hasher.finish(); - let (chan, next) = if next >= chans.len() { - (chans[0].clone(), 1) - } else { - (chans[next].clone(), next + 1) - }; + // TODO make it a config option + let consistent_parsing = true; + if recv_counter >= buffer_flush_length || flush { + bufmap + .drain() + .map(|(addr, buf)| { + let mut hasher = DefaultHasher::new(); + addr.hash(&mut hasher); + let ahash = hasher.finish(); + let chan = if consistent_parsing { + let chlen = chans.len(); + chans[ahash as usize % chlen].clone() + } else { + if next >= chans.len() { + next = 1; + chans[0].clone() + } else { + next = next + 1; + chans[next].clone() + } + }; - spawn( - chan.send(Task::Parse(ahash, buf.freeze())) - .map_err(|_| { - DROPS.fetch_add(1, Ordering::Relaxed); - }).map(|_|())) - }).last(); + spawn( + chan.send(Task::Parse(ahash, buf.freeze())) + .map_err(|_| { + DROPS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()), + ) + }).last(); - spawn(StatsdServer::new( + spawn( + StatsdServer::new( socket, chans, bufmap, @@ -125,7 +140,7 @@ impl IntoFuture for StatsdServer { received, flush_flags, thread_idx, - ).into_future() + ).into_future(), ); } else { spawn( @@ -140,8 +155,8 @@ impl IntoFuture for StatsdServer { received, flush_flags, thread_idx, - ).into_future(), - ); + ).into_future(), + ); } Ok(()) }); diff --git a/src/udp.rs b/src/udp.rs index ccf2a6b..25807b7 100644 --- a/src/udp.rs +++ b/src/udp.rs @@ -1,12 +1,12 @@ +use std::collections::hash_map::DefaultHasher; +use std::collections::HashMap; +use std::hash::Hasher; use std::io; use std::net::SocketAddr; use std::ptr::null_mut; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::thread; -use std::collections::HashMap; -use std::collections::hash_map::DefaultHasher; -use std::hash::Hasher; use server::StatsdServer; use task::Task; @@ -35,7 +35,7 @@ pub(crate) fn start_sync_udp( buffer_flush_time: u64, buffer_flush_length: usize, flush_flags: Arc>, - ) { +) { info!(log, "multimessage enabled, starting in sync UDP mode"; "socket-is-blocking"=>!mm_async, "packets"=>mm_packets); // It is crucial for recvmmsg to have one socket per many threads @@ -77,7 +77,7 @@ pub(crate) fn start_sync_udp( // addr len is 16 bytes for ipv6 address + 4 bytes for port totalling 20 bytes // the structure is only used for advanced information and not copied anywhere // so it can be initialized before main cycle - let mut addrs: Vec<[u8;20]> = Vec::with_capacity(mm_packets); + let mut addrs: Vec<[u8; 20]> = Vec::with_capacity(mm_packets); addrs.resize(mm_packets, [0; 20]); for i in 0..mm_packets { @@ -85,11 +85,11 @@ pub(crate) fn start_sync_udp( msg_hdr: msghdr { msg_name: addrs[i].as_mut_ptr() as *mut c_void, msg_namelen: 20 as socklen_t, - msg_iov: null_mut(), // this will change later - msg_iovlen: 0, // this will change later - msg_control: null_mut(), // we won't need this - msg_controllen: 0, // and this - msg_flags: 0, // and of couse this + msg_iov: null_mut(), // this will change later + msg_iovlen: 0, // this will change later + msg_control: null_mut(), // we won't need this + msg_controllen: 0, // and this + msg_flags: 0, // and of couse this }, msg_len: 0, }; @@ -104,8 +104,8 @@ pub(crate) fn start_sync_udp( // this will store resulting per-source buffers let mut bufmap = HashMap::new(); let mut total_received = 0; - let min_bytes = mm_packets*mm_packets*bufsize; - let rowsize = bufsize*mm_packets; + let min_bytes = mm_packets * mm_packets * bufsize; + let rowsize = bufsize * mm_packets; let mut recv_buffer = Vec::new(); recv_buffer.reserve_exact(min_bytes); @@ -129,8 +129,9 @@ pub(crate) fn start_sync_udp( for i in 0..mm_packets { let mut chunk = iovec { - iov_base: recv_buffer[i*rowsize..i*rowsize+rowsize].as_mut_ptr() as *mut c_void, - iov_len: rowsize + iov_base: recv_buffer[i * rowsize..i * rowsize + rowsize].as_mut_ptr() + as *mut c_void, + iov_len: rowsize, }; chunks.push(chunk); // put the result to mheaders @@ -154,8 +155,19 @@ pub(crate) fn start_sync_udp( } }; - let res = - unsafe { recvmmsg(fd as c_int, mhptr, mhlen as c_uint, flags, if mm_timeout > 0 {&mut timeout} else {null_mut()}) }; + let res = unsafe { + recvmmsg( + fd as c_int, + mhptr, + mhlen as c_uint, + flags, + if mm_timeout > 0 { + &mut timeout + } else { + null_mut() + }, + ) + }; if res == 0 { // skip this shit @@ -163,44 +175,51 @@ pub(crate) fn start_sync_udp( let messages = res as usize; // we've received some messages for i in 0..messages { - let mlen = mheaders[i].msg_len as usize; + let mlen = mheaders[i].msg_len as usize; total_received += mlen; + INGRESS.fetch_add(mlen, Ordering::Relaxed); + // create address entry in messagemap - let mut entry = bufmap.entry(addrs[i]).or_insert(BytesMut::with_capacity(buffer_flush_length)); + let mut entry = bufmap + .entry(addrs[i]) + .or_insert(BytesMut::with_capacity(buffer_flush_length)); // and put it's buffer there - entry.put(&recv_buffer[i*rowsize..i*rowsize+mlen]); + entry.put(&recv_buffer[i * rowsize..i * rowsize + mlen]); // reset addres to be used in next cycle addrs[i] = [0; 20]; mheaders[i].msg_hdr.msg_namelen = 20; } - INGRESS.fetch_add(total_received, Ordering::Relaxed); - let consistent_parsing = true; // when it's time to send bytes, send them let flush = flush_flags.get(i).unwrap().swap(false, Ordering::SeqCst); if flush || total_received >= buffer_flush_length { total_received = 0; - bufmap.drain().map(|(addr, mut buf)|{ - // in some ideal world we want all values from the same host to be parsed by the - // same thread, but this could cause load unbalancing between - // threads TODO: make it an option in config - let mut hasher = DefaultHasher::new(); - hasher.write(&addr); - let ahash = hasher.finish(); - let mut chan = if consistent_parsing { - chans[ahash as usize % chlen].clone() - } else { - ichans.next().unwrap().clone() - }; - chan.try_send(Task::Parse(ahash, buf.take().freeze())) - .map_err(|_| { - warn!(log, "error sending buffer(queue full?)"); - DROPS.fetch_add(messages as usize, Ordering::Relaxed); - }).unwrap_or(()); - }).last(); + bufmap + .drain() + .map(|(addr, mut buf)| { + // in some ideal world we want all values from the same host to be parsed by the + // same thread, but this could cause load unbalancing between + // threads TODO: make it an option in config + let mut hasher = DefaultHasher::new(); + hasher.write(&addr); + let ahash = hasher.finish(); + let mut chan = if consistent_parsing { + chans[ahash as usize % chlen].clone() + } else { + ichans.next().unwrap().clone() + }; + chan.try_send(Task::Parse(ahash, buf.take().freeze())) + .map_err(|_| { + warn!(log, "error sending buffer(queue full?)"); + DROPS.fetch_add( + messages as usize, + Ordering::Relaxed, + ); + }).unwrap_or(()); + }).last(); } } else { let errno = unsafe { *__errno_location() }; @@ -228,7 +247,7 @@ pub(crate) fn start_async_udp( bufsize: usize, buffer_flush_length: usize, flush_flags: Arc>, - ) { +) { info!(log, "multimessage is disabled, starting in async UDP mode"); // Create a pool of listener sockets @@ -260,7 +279,6 @@ pub(crate) fn start_async_udp( for _ in 0..greens { // start a listener for all sockets for socket in sockets.iter() { - let mut readbuf = BytesMut::with_capacity(bufsize); unsafe { readbuf.set_len(bufsize) } let chans = chans.clone(); @@ -268,7 +286,7 @@ pub(crate) fn start_async_udp( let socket = socket.try_clone().expect("cloning socket"); let socket = UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()) - .expect("adding socket to event loop"); + .expect("adding socket to event loop"); let server = StatsdServer::new( socket, @@ -281,7 +299,7 @@ pub(crate) fn start_async_udp( readbuf, flush_flags.clone(), i, - ); + ); runtime.spawn(server.into_future()); } From eb53406f4e548f5c91bc0a6479132e58a897cc0b Mon Sep 17 00:00:00 2001 From: Albibek Date: Thu, 8 Nov 2018 00:02:17 +0300 Subject: [PATCH 18/24] Add consistent_parsing option --- config.toml | 5 +++ src/config.rs | 48 ++++++++++++---------- src/main.rs | 109 +++++++++++++++++++++----------------------------- src/server.rs | 41 +++++++++---------- src/task.rs | 5 ++- src/udp.rs | 33 +++++++-------- src/util.rs | 32 +++++++++++++++ 7 files changed, 149 insertions(+), 124 deletions(-) diff --git a/config.toml b/config.toml index 3aae66b..e35af9f 100644 --- a/config.toml +++ b/config.toml @@ -47,6 +47,11 @@ update-counter-threshold = 200 # any heavy counting would potentially block task and make some unparsed buffers to be skipped # fast-aggregation = true +# Process buffers from different hosts separately, this gives more guarantee to parse +# metrics from different hosts correctly. Can play bad if lots of metrics is received from a single host, set +# it to false if you have such use case +# consistent-parsing = true + [carbon] # IP and port of the carbon-protocol backend to send aggregated data to diff --git a/src/config.rs b/src/config.rs index 1db1192..4e62d17 100644 --- a/src/config.rs +++ b/src/config.rs @@ -55,6 +55,9 @@ pub(crate) struct System { /// Consensus kind to use pub consensus: ConsensusKind, + + /// Consistent parsing + pub consistent_parsing: bool, } impl Default for System { @@ -73,6 +76,7 @@ impl Default for System { start_as_leader: false, stats_prefix: "resources.monitoring.bioyino".to_string(), consensus: ConsensusKind::None, + consistent_parsing: true, } } } @@ -312,35 +316,35 @@ impl System { let app = app_from_crate!() .arg( Arg::with_name("config") - .help("configuration file path") - .long("config") - .short("c") - .required(true) - .takes_value(true) - .default_value("/etc/bioyino/bioyino.toml"), - ).arg( - Arg::with_name("verbosity") + .help("configuration file path") + .long("config") + .short("c") + .required(true) + .takes_value(true) + .default_value("/etc/bioyino/bioyino.toml"), + ).arg( + Arg::with_name("verbosity") .short("v") .help("logging level") .takes_value(true), - ).subcommand( - SubCommand::with_name("query") - .about("send a management command to running bioyino server") - .arg( - Arg::with_name("host") + ).subcommand( + SubCommand::with_name("query") + .about("send a management command to running bioyino server") + .arg( + Arg::with_name("host") .short("h") .default_value("127.0.0.1:8137"), - ).subcommand(SubCommand::with_name("status").about("get server state")) - .subcommand( - SubCommand::with_name("consensus") + ).subcommand(SubCommand::with_name("status").about("get server state")) + .subcommand( + SubCommand::with_name("consensus") .arg(Arg::with_name("action").index(1)) .arg( Arg::with_name("leader_action") - .index(2) - .default_value("unchanged"), - ), - ), - ).get_matches(); + .index(2) + .default_value("unchanged"), + ), + ), + ).get_matches(); let config = value_t!(app.value_of("config"), String).expect("config file must be string"); let mut file = File::open(&config).expect(&format!("opening config file at {}", &config)); @@ -365,7 +369,7 @@ impl System { ( system, Command::Query(MgmtCommand::ConsensusCommand(c_action, l_action), server), - ) + ) } else { // shold be unreachable unreachable!("clap bug?") diff --git a/src/main.rs b/src/main.rs index 5fa92ba..9da6332 100644 --- a/src/main.rs +++ b/src/main.rs @@ -71,7 +71,7 @@ use std::time::{self, Duration, Instant, SystemTime}; use slog::{Drain, Level}; -use bytes::Bytes; +use bytes::{Bytes, BytesMut}; use futures::future::{empty, ok}; use futures::sync::mpsc; use futures::{Future, IntoFuture, Stream}; @@ -103,6 +103,7 @@ pub type Float = f64; pub type Cache = HashMap>; thread_local!(static LONG_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); thread_local!(static SHORT_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); +thread_local!(static BUFFER_CACHE: RefCell> = RefCell::new(HashMap::with_capacity(8192))); // options //pub static SHOW_PARSE_ERRORS: AtomicBool = ATOMIC_BOOL_INIT; @@ -146,41 +147,16 @@ pub enum ConsensusKind { } lazy_static! { - pub static ref CONSENSUS_STATE: Mutex = - { Mutex::new(ConsensusState::Disabled) }; + pub static ref CONSENSUS_STATE: Mutex = { Mutex::new(ConsensusState::Disabled) }; } pub static IS_LEADER: AtomicBool = ATOMIC_BOOL_INIT; fn main() { - // { - //use std::io; - //use std::ffi::CString; - //use std::ptr::{null_mut, null}; - //use libc::*; - - //let domain = CString::new("ya.ru").unwrap().into_raw(); - //let mut result: *mut addrinfo = null_mut(); - - //unsafe { - //getaddrinfo(domain, null_mut(), null(), &mut result); - //} - - ////let errno = unsafe { *__errno_location() }; - //println!("{:?}", io::Error::last_os_error()); - //let mut cur = result; - //while cur != null_mut() { - //unsafe{ - //println!("LEN {:?}", (*result).ai_addrlen); - //println!("DATA {:?}", (*(*result).ai_addr).sa_data); - //cur = (*result).ai_next; - //} - //} - - //} let (system, command) = System::load(); + let mut config = system.clone(); let System { verbosity, network: @@ -194,7 +170,7 @@ fn main() { mm_async, mm_timeout, buffer_flush_time, - buffer_flush_length, + buffer_flush_length :_, greens, async_sockets, nodes, @@ -226,7 +202,8 @@ fn main() { start_as_leader, stats_prefix, consensus, - } = system.clone(); + consistent_parsing: _, + } = system; let verbosity = Level::from_str(&verbosity).expect("bad verbosity"); @@ -270,6 +247,17 @@ fn main() { let update_counter_prefix: Bytes = update_counter_prefix.into(); let update_counter_suffix: Bytes = update_counter_suffix.into(); + + let reserve_min = bufsize * mm_packets * mm_packets; + + config.network.buffer_flush_length = if config.network.buffer_flush_length < reserve_min { + debug!(rlog, "buffer-flush-len is lower than mm-packets*mm-packets*bufsize"; "new-value"=>reserve_min, "old-value"=>config.network.buffer_flush_length); + reserve_min + } else { + config.network.buffer_flush_length + }; + + let config = Arc::new(config); let log = rlog.new(o!("thread" => "main")); // Init task options before initializing task threads @@ -340,29 +328,29 @@ fn main() { let log = log.clone(); let flog = log.clone(); thread::Builder::new() - .name("bioyino_raft".into()) - .spawn(move || { - let mut runtime = Runtime::new().expect("creating runtime for raft thread"); - if start_as_leader { - warn!(log, "Starting as leader with enabled consensus. More that one leader is possible before consensus settle up."); - } - let d = Delay::new(Instant::now() + Duration::from_millis(raft.start_delay)); - let log = log.clone(); - let delayed = d - .map_err(|_| ()) - .and_then(move |_| { - let mut con_state = CONSENSUS_STATE.lock().unwrap(); - *con_state = ConsensusState::Enabled; - info!(log, "starting internal consensus"; "initial_state"=>format!("{:?}", *con_state)); - start_internal_raft(raft, consensus_log); - Ok(()) - }); - - runtime.spawn(delayed); - runtime.block_on(empty::<(), ()>()).expect("raft thread failed"); - - info!(flog, "consensus thread stopped"); - }).expect("starting counting worker thread"); + .name("bioyino_raft".into()) + .spawn(move || { + let mut runtime = Runtime::new().expect("creating runtime for raft thread"); + if start_as_leader { + warn!(log, "Starting as leader with enabled consensus. More that one leader is possible before consensus settle up."); + } + let d = Delay::new(Instant::now() + Duration::from_millis(raft.start_delay)); + let log = log.clone(); + let delayed = d + .map_err(|_| ()) + .and_then(move |_| { + let mut con_state = CONSENSUS_STATE.lock().unwrap(); + *con_state = ConsensusState::Enabled; + info!(log, "starting internal consensus"; "initial_state"=>format!("{:?}", *con_state)); + start_internal_raft(raft, consensus_log); + Ok(()) + }); + + runtime.spawn(delayed); + runtime.block_on(empty::<(), ()>()).expect("raft thread failed"); + + info!(flog, "consensus thread stopped"); + }).expect("starting counting worker thread"); } ConsensusKind::Consul => { if start_as_leader { @@ -405,6 +393,7 @@ fn main() { let dur = Duration::from_millis(carbon.interval); let carbon_timer = Interval::new(Instant::now() + dur, dur); + let carbon_config = config.carbon.clone(); let carbon_timer = carbon_timer.map_err(|e| GeneralError::Timer(e)).for_each( move |_tick| { let ts = SystemTime::now().duration_since(time::UNIX_EPOCH).map_err( @@ -419,7 +408,7 @@ fn main() { let update_counter_prefix = update_counter_prefix.clone(); let update_counter_suffix = update_counter_suffix.clone(); - let backend_opts = system.carbon.clone(); + let backend_opts = carbon_config.clone(); thread::Builder::new() .name("bioyino_carbon".into()) .spawn(move || { @@ -503,13 +492,6 @@ fn main() { } } - let reserve_min = bufsize * mm_packets; - let buffer_flush_length = if buffer_flush_length < reserve_min { - debug!(log, "buffer-flush-len is lower than mm-packets*bufsize"; "new-value"=>reserve_min, "old-value"=>buffer_flush_length); - reserve_min - } else { - buffer_flush_length - }; if buffer_flush_time > 0 { let dur = Duration::from_millis(buffer_flush_time); @@ -538,13 +520,12 @@ fn main() { log, listen, &chans, + config.clone(), n_threads, bufsize, mm_packets, mm_async, mm_timeout, - buffer_flush_time, - buffer_flush_length, flush_flags.clone(), ); } else { @@ -552,11 +533,11 @@ fn main() { log, listen, &chans, + config.clone(), n_threads, greens, async_sockets, bufsize, - buffer_flush_length, flush_flags.clone(), ); } diff --git a/src/server.rs b/src/server.rs index e552868..5c7aa19 100644 --- a/src/server.rs +++ b/src/server.rs @@ -14,13 +14,14 @@ use tokio::executor::current_thread::spawn; use tokio::net::UdpSocket; use task::Task; +use config::System; #[derive(Debug)] pub struct StatsdServer { socket: UdpSocket, chans: Vec>, bufmap: HashMap, - buffer_flush_length: usize, + config: Arc, bufsize: usize, recv_counter: usize, next: usize, @@ -30,23 +31,23 @@ pub struct StatsdServer { } impl StatsdServer { - pub fn new( + pub(crate) fn new( socket: UdpSocket, chans: Vec>, bufmap: HashMap, - buffer_flush_length: usize, + config: Arc, bufsize: usize, recv_counter: usize, next: usize, readbuf: BytesMut, flush_flags: Arc>, thread_idx: usize, - ) -> Self { + ) -> Self { Self { socket, chans, bufmap, - buffer_flush_length, + config, bufsize, recv_counter, next, @@ -67,7 +68,7 @@ impl IntoFuture for StatsdServer { socket, chans, mut bufmap, - buffer_flush_length, + config, bufsize, mut recv_counter, mut next, @@ -88,7 +89,7 @@ impl IntoFuture for StatsdServer { { let buf = bufmap .entry(addr) - .or_insert(BytesMut::with_capacity(buffer_flush_length)); + .or_insert(BytesMut::with_capacity(config.network.buffer_flush_length)); recv_counter += size; buf.put(&received[0..size]); } @@ -98,16 +99,14 @@ impl IntoFuture for StatsdServer { .unwrap() .swap(false, Ordering::SeqCst); - // TODO make it a config option - let consistent_parsing = true; - if recv_counter >= buffer_flush_length || flush { + if recv_counter >= config.network.buffer_flush_length || flush { bufmap .drain() .map(|(addr, buf)| { let mut hasher = DefaultHasher::new(); addr.hash(&mut hasher); let ahash = hasher.finish(); - let chan = if consistent_parsing { + let chan = if config.consistent_parsing { let chlen = chans.len(); chans[ahash as usize % chlen].clone() } else { @@ -122,10 +121,10 @@ impl IntoFuture for StatsdServer { spawn( chan.send(Task::Parse(ahash, buf.freeze())) - .map_err(|_| { - DROPS.fetch_add(1, Ordering::Relaxed); - }).map(|_| ()), - ) + .map_err(|_| { + DROPS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()), + ) }).last(); spawn( @@ -133,30 +132,30 @@ impl IntoFuture for StatsdServer { socket, chans, bufmap, - buffer_flush_length, + config, bufsize, 0, next, received, flush_flags, thread_idx, - ).into_future(), - ); + ).into_future(), + ); } else { spawn( StatsdServer::new( socket, chans, bufmap, - buffer_flush_length, + config, bufsize, recv_counter, next, received, flush_flags, thread_idx, - ).into_future(), - ); + ).into_future(), + ); } Ok(()) }); diff --git a/src/task.rs b/src/task.rs index fce4abb..e8154d8 100644 --- a/src/task.rs +++ b/src/task.rs @@ -15,7 +15,7 @@ use util::AggregateOptions; use { Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, LONG_CACHE, PARSE_ERRORS, PEER_ERRORS, - SHORT_CACHE, + SHORT_CACHE, BUFFER_CACHE, }; #[derive(Debug)] @@ -174,6 +174,9 @@ fn cut_bad(buf: &mut Bytes) -> Option { } fn parse_and_insert(_addr: u64, mut buf: Bytes) { + //let prev_buf = BUFFER_CACHE.with(|c| { + // c.get_mut + //}); // Cloned buf is shallow copy, so input and buf are the same bytes. // We are going to parse the whole slice, so for parser we use input as readonly // while buf follows the parser progress and is cut to get only names diff --git a/src/udp.rs b/src/udp.rs index 25807b7..caf6323 100644 --- a/src/udp.rs +++ b/src/udp.rs @@ -10,6 +10,7 @@ use std::thread; use server::StatsdServer; use task::Task; +use config::System; use {DROPS, INGRESS}; use bytes::{BufMut, BytesMut}; @@ -27,15 +28,14 @@ pub(crate) fn start_sync_udp( log: Logger, listen: SocketAddr, chans: &Vec>, + config: Arc, n_threads: usize, bufsize: usize, mm_packets: usize, mm_async: bool, mm_timeout: u64, - buffer_flush_time: u64, - buffer_flush_length: usize, flush_flags: Arc>, -) { + ) { info!(log, "multimessage enabled, starting in sync UDP mode"; "socket-is-blocking"=>!mm_async, "packets"=>mm_packets); // It is crucial for recvmmsg to have one socket per many threads @@ -48,7 +48,7 @@ pub(crate) fn start_sync_udp( sck.set_nonblocking(mm_async).unwrap(); let mm_timeout = if mm_timeout == 0 { - buffer_flush_time + config.network.buffer_flush_time } else { mm_timeout }; @@ -59,6 +59,7 @@ pub(crate) fn start_sync_udp( let sck = sck.try_clone().unwrap(); let flush_flags = flush_flags.clone(); + let config = config.clone(); thread::Builder::new() .name(format!("bioyino_mudp{}", i).into()) .spawn(move || { @@ -131,7 +132,7 @@ pub(crate) fn start_sync_udp( let mut chunk = iovec { iov_base: recv_buffer[i * rowsize..i * rowsize + rowsize].as_mut_ptr() as *mut c_void, - iov_len: rowsize, + iov_len: rowsize, }; chunks.push(chunk); // put the result to mheaders @@ -166,7 +167,7 @@ pub(crate) fn start_sync_udp( } else { null_mut() }, - ) + ) }; if res == 0 { @@ -183,7 +184,7 @@ pub(crate) fn start_sync_udp( // create address entry in messagemap let mut entry = bufmap .entry(addrs[i]) - .or_insert(BytesMut::with_capacity(buffer_flush_length)); + .or_insert(BytesMut::with_capacity(config.network.buffer_flush_length)); // and put it's buffer there entry.put(&recv_buffer[i * rowsize..i * rowsize + mlen]); @@ -192,10 +193,9 @@ pub(crate) fn start_sync_udp( mheaders[i].msg_hdr.msg_namelen = 20; } - let consistent_parsing = true; // when it's time to send bytes, send them let flush = flush_flags.get(i).unwrap().swap(false, Ordering::SeqCst); - if flush || total_received >= buffer_flush_length { + if flush || total_received >= config.network.buffer_flush_length { total_received = 0; bufmap .drain() @@ -206,7 +206,7 @@ pub(crate) fn start_sync_udp( let mut hasher = DefaultHasher::new(); hasher.write(&addr); let ahash = hasher.finish(); - let mut chan = if consistent_parsing { + let mut chan = if config.consistent_parsing { chans[ahash as usize % chlen].clone() } else { ichans.next().unwrap().clone() @@ -217,7 +217,7 @@ pub(crate) fn start_sync_udp( DROPS.fetch_add( messages as usize, Ordering::Relaxed, - ); + ); }).unwrap_or(()); }).last(); } @@ -241,13 +241,13 @@ pub(crate) fn start_async_udp( log: Logger, listen: SocketAddr, chans: &Vec>, + config: Arc, n_threads: usize, greens: usize, async_sockets: usize, bufsize: usize, - buffer_flush_length: usize, flush_flags: Arc>, -) { + ) { info!(log, "multimessage is disabled, starting in async UDP mode"); // Create a pool of listener sockets @@ -269,6 +269,7 @@ pub(crate) fn start_async_udp( let chans = chans.clone(); let flush_flags = flush_flags.clone(); + let config = config.clone(); thread::Builder::new() .name(format!("bioyino_udp{}", i).into()) .spawn(move || { @@ -286,20 +287,20 @@ pub(crate) fn start_async_udp( let socket = socket.try_clone().expect("cloning socket"); let socket = UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()) - .expect("adding socket to event loop"); + .expect("adding socket to event loop"); let server = StatsdServer::new( socket, chans.clone(), HashMap::new(), - buffer_flush_length, + config.clone(), bufsize, 0, i, readbuf, flush_flags.clone(), i, - ); + ); runtime.spawn(server.into_future()); } diff --git a/src/util.rs b/src/util.rs index c19481f..3aa7ace 100644 --- a/src/util.rs +++ b/src/util.rs @@ -39,6 +39,38 @@ pub fn try_resolve(s: &str) -> SocketAddr { }) } +// TODO impl this correctly and use instead of try_resolve +// PROFIT: gives libnss-aware behaviour +/* + fn _try_resolve_nss(name: &str) { + use std::io; + use std::ffi::CString; + use std::ptr::{null_mut, null}; + use libc::*; + + let domain= CString::new(Vec::from(name)).unwrap().into_raw(); + let mut result: *mut addrinfo = null_mut(); + + let success = unsafe { + getaddrinfo(domain, null_mut(), null(), &mut result) + }; + + if success != 0 { +// let errno = unsafe { *__errno_location() }; +println!("{:?}", io::Error::last_os_error()); +} else { +let mut cur = result; +while cur != null_mut() { +unsafe{ +println!("LEN {:?}", (*result).ai_addrlen); +println!("DATA {:?}", (*(*result).ai_addr).sa_data); +cur = (*result).ai_next; +} +} +} +} +*/ + /// Get hostname. Copypasted from some crate pub fn get_hostname() -> Option { let len = 255; From 89c87399c7e09518ed776c51b1bdc1fd55b89171 Mon Sep 17 00:00:00 2001 From: Albibek Date: Fri, 9 Nov 2018 01:25:22 +0300 Subject: [PATCH 19/24] Change buffer handling in parse task --- config.toml | 3 + src/config.rs | 19 +-- src/main.rs | 18 +-- src/peer.rs | 94 ++++++------ src/server.rs | 2 +- src/task.rs | 415 ++++++++++++++++++++++++++++---------------------- src/udp.rs | 2 +- src/util.rs | 22 ++- 8 files changed, 320 insertions(+), 255 deletions(-) diff --git a/config.toml b/config.toml index e35af9f..88b3727 100644 --- a/config.toml +++ b/config.toml @@ -52,6 +52,9 @@ update-counter-threshold = 200 # it to false if you have such use case # consistent-parsing = true +# Log all buffers being dropped due to parsing errors. Can be very spammy. +# log-parse-errors = false + [carbon] # IP and port of the carbon-protocol backend to send aggregated data to diff --git a/src/config.rs b/src/config.rs index 4e62d17..489ca5c 100644 --- a/src/config.rs +++ b/src/config.rs @@ -15,7 +15,7 @@ use {ConsensusKind, ConsensusState}; #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] -pub(crate) struct System { +pub struct System { /// Logging level pub verbosity: String, @@ -83,7 +83,7 @@ impl Default for System { #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] -pub(crate) struct Metrics { +pub struct Metrics { // TODO: Maximum metric array size, 0 for unlimited // max_metrics: usize, /// Should we provide metrics with top update numbers @@ -100,9 +100,9 @@ pub(crate) struct Metrics { /// Aggregate faster at the price or probably loosing some incoming metrics pub fast_aggregation: bool, - // TODO - // /// Whether we should spam parsing errors in logs - // pub log_parse_errors: bool, + + /// Whether we should spam parsing errors in logs + pub log_parse_errors: bool, } impl Default for Metrics { @@ -114,13 +114,14 @@ impl Default for Metrics { update_counter_suffix: String::new(), update_counter_threshold: 200, fast_aggregation: true, + log_parse_errors: false, } } } #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] -pub(crate) struct Carbon { +pub struct Carbon { // TODO: will be used when multiple backends support is implemented ///// Enable sending to carbon protocol backend //pub enabled: bool, @@ -160,7 +161,7 @@ impl Default for Carbon { #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] -pub(crate) struct Network { +pub struct Network { /// Address and UDP port to listen for statsd metrics on pub listen: SocketAddr, @@ -228,7 +229,7 @@ impl Default for Network { #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] -pub(crate) struct Consul { +pub struct Consul { /// Start in disabled leader finding mode pub start_as: ConsensusState, @@ -259,7 +260,7 @@ impl Default for Consul { #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] -pub(crate) struct Raft { +pub struct Raft { /// Delay raft after start (ms) pub start_delay: u64, diff --git a/src/main.rs b/src/main.rs index 9da6332..26a49bb 100644 --- a/src/main.rs +++ b/src/main.rs @@ -61,7 +61,6 @@ pub mod protocol_capnp { include!(concat!(env!("OUT_DIR"), "/schema/protocol_capnp.rs")); } -use std::cell::RefCell; use std::collections::HashMap; use std::str::FromStr; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering, ATOMIC_BOOL_INIT, ATOMIC_USIZE_INIT}; @@ -71,7 +70,7 @@ use std::time::{self, Duration, Instant, SystemTime}; use slog::{Drain, Level}; -use bytes::{Bytes, BytesMut}; +use bytes::{Bytes}; use futures::future::{empty, ok}; use futures::sync::mpsc; use futures::{Future, IntoFuture, Stream}; @@ -89,7 +88,7 @@ use management::{MgmtClient, MgmtServer}; use metric::Metric; use peer::{NativeProtocolServer, NativeProtocolSnapshot}; use raft::start_internal_raft; -use task::Task; +use task::{Task, TaskRunner}; use util::{ try_resolve, AggregateOptions, Aggregator, BackoffRetryBuilder, OwnStats, UpdateCounterOptions, }; @@ -101,12 +100,6 @@ pub type Float = f64; // a type to store pre-aggregated data pub type Cache = HashMap>; -thread_local!(static LONG_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); -thread_local!(static SHORT_CACHE: RefCell>> = RefCell::new(HashMap::with_capacity(8192))); -thread_local!(static BUFFER_CACHE: RefCell> = RefCell::new(HashMap::with_capacity(8192))); - -// options -//pub static SHOW_PARSE_ERRORS: AtomicBool = ATOMIC_BOOL_INIT; // statistic counters pub static PARSE_ERRORS: AtomicUsize = ATOMIC_USIZE_INIT; @@ -193,6 +186,7 @@ fn main() { update_counter_suffix, update_counter_threshold, fast_aggregation, + log_parse_errors: _, }, carbon, n_threads, @@ -268,11 +262,15 @@ fn main() { for i in 0..w_threads { let (tx, rx) = mpsc::channel(task_queue_size); chans.push(tx); + let tlog = log.clone(); + let cf = config.clone(); thread::Builder::new() .name(format!("bioyino_cnt{}", i).into()) .spawn(move || { + let runner = TaskRunner::new(tlog, cf, 8192); let mut runtime = Runtime::new().expect("creating runtime for counting worker"); - let future = rx.for_each(move |task: Task| ok(task.run())); + let future = rx.fold(runner, move |mut runner, task: Task| {runner.run(task); Ok(runner)}).map(|_|()).map_err(|_|()); + // let future = rx.for_each(|task: Task| ok(runner.run(task))); runtime.block_on(future).expect("worker thread failed"); }).expect("starting counting worker thread"); } diff --git a/src/peer.rs b/src/peer.rs index 3b11185..e7f2106 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -116,7 +116,7 @@ fn parse_and_send( reader: cmsg::Reader, next_chan: Sender, log: Logger, -) -> Result<(), MetricError> { + ) -> Result<(), MetricError> { match reader.which().map_err(MetricError::CapnpSchema)? { cmsg::Single(reader) => { let reader = reader.map_err(MetricError::Capnp)?; @@ -185,7 +185,7 @@ impl NativeProtocolSnapshot { nodes: Vec, interval: Duration, chans: &Vec>, - ) -> Self { + ) -> Self { Self { log: log.new(o!("source"=>"peer-client")), nodes, @@ -220,17 +220,17 @@ impl IntoFuture for NativeProtocolSnapshot { spawn(chan.send(Task::TakeSnapshot(tx)).then(|_| Ok(()))); rx }) - .collect::>(); + .collect::>(); let get_metrics = join_all(metrics) .map_err(|_| { PEER_ERRORS.fetch_add(1, Ordering::Relaxed); PeerError::TaskSend }) - .and_then(move |mut metrics| { - metrics.retain(|m| m.len() > 0); - Ok(metrics) - }); + .and_then(move |mut metrics| { + metrics.retain(|m| m.len() > 0); + Ok(metrics) + }); // All nodes have to receive the same metrics // so we don't parallel connections and metrics fetching @@ -268,20 +268,20 @@ impl IntoFuture for NativeProtocolSnapshot { c_metric.set_name(name); metric.fill_capnp(&mut c_metric); }) - .last(); + .last(); } codec.send(snapshot_message).map(|_| ()).map_err(move |e| { debug!(elog, "codec error"; "error"=>e.to_string()); PeerError::Capnp(e) }) }) - .map_err(move |e| { - PEER_ERRORS.fetch_add(1, Ordering::Relaxed); - debug!(dlog, "error sending snapshot: {}", e) - }) - .then(|_| Ok(())) // we don't want to faill the whole timer cycle because of one send error + .map_err(move |e| { + PEER_ERRORS.fetch_add(1, Ordering::Relaxed); + debug!(dlog, "error sending snapshot: {}", e) + }) + .then(|_| Ok(())) // we don't want to fail the whole timer cycle because of one send error }) - .collect::>(); + .collect::>(); join_all(clients).map(|_| ()) }) }); @@ -293,34 +293,24 @@ impl IntoFuture for NativeProtocolSnapshot { mod test { use std::net::SocketAddr; - use {slog, slog_async, slog_term}; + use std::time::{SystemTime, UNIX_EPOCH}; + use std::sync::Arc; use bytes::Bytes; use capnp::message::Builder; - use futures::future::ok; use futures::sync::mpsc::{self, Receiver}; - use metric::{Metric, MetricType}; - use slog::Drain; use slog::Logger; - use std::time::{SystemTime, UNIX_EPOCH}; use tokio::runtime::current_thread::Runtime; use tokio::timer::Delay; - use {LONG_CACHE, SHORT_CACHE}; + use util::prepare_log; + use metric::{Metric, MetricType}; + use task::TaskRunner; + use config::System; use super::*; - fn prepare_log() -> Logger { - // Set logging - let decorator = slog_term::TermDecorator::new().build(); - let drain = slog_term::FullFormat::new(decorator).build().fuse(); - let filter = slog::LevelFilter::new(drain, slog::Level::Trace).fuse(); - let drain = slog_async::Async::new(filter).build().fuse(); - let rlog = slog::Logger::root(drain, o!("program"=>"test")); - return rlog; - } - fn prepare_runtime_with_server() -> (Runtime, Receiver, Logger, SocketAddr) { - let rlog = prepare_log(); + fn prepare_runtime_with_server(log: Logger) -> (Runtime, Receiver, SocketAddr) { let mut chans = Vec::new(); let (tx, rx) = mpsc::channel(5); chans.push(tx); @@ -329,32 +319,47 @@ mod test { let mut runtime = Runtime::new().expect("creating runtime for main thread"); let c_peer_listen = address.clone(); - let c_serv_log = rlog.clone(); - let peer_server = NativeProtocolServer::new(rlog.clone(), c_peer_listen, chans) + let c_serv_log = log.clone(); + let peer_server = NativeProtocolServer::new(log.clone(), c_peer_listen, chans) .into_future() .map_err(move |e| { warn!(c_serv_log, "shot server gone with error: {:?}", e); }); runtime.spawn(peer_server); - (runtime, rx, rlog, address) + (runtime, rx, address) } #[test] fn test_peer_protocol_capnp() { let test_timeout = Instant::now() + Duration::from_secs(3); - let (mut runtime, rx, rlog, address) = prepare_runtime_with_server(); - let future = rx.for_each(move |task: Task| ok(task.run()).and_then(|_| Ok(()))); - runtime.spawn(future); + let log = prepare_log("test_peer_capnp"); + + let mut config = System::default(); + config.metrics.log_parse_errors = true; + let runner = TaskRunner::new(log.clone(), Arc::new(config), 16); let ts = SystemTime::now().duration_since(UNIX_EPOCH).unwrap(); let ts = ts.as_secs() as u64; let outmetric = Metric::new(42f64, MetricType::Gauge(None), ts.into(), None).unwrap(); - let log = rlog.clone(); let metric = outmetric.clone(); + let (mut runtime, rx, address) = prepare_runtime_with_server(log.clone()); + + let future = rx.fold(runner, move |mut runner, task: Task| {runner.run(task); Ok(runner)}).and_then(move |runner| { + let single_name: Bytes = "complex.test.bioyino_single".into(); + let multi_name: Bytes = "complex.test.bioyino_multi".into(); + let shot_name: Bytes = "complex.test.bioyino_snapshot".into(); + assert_eq!(runner.get_long_entry(&shot_name), Some(&outmetric)); + assert_eq!(runner.get_short_entry(&single_name), Some(&outmetric)); + assert_eq!(runner.get_short_entry(&multi_name), Some(&outmetric)); + + Ok(()) + }).map_err(|_|panic!("error in the future")); + runtime.spawn(future); + let sender = TcpStream::connect(&address) .map_err(|e| { println!("connection err: {:?}", e); @@ -395,7 +400,7 @@ mod test { .send(multi_message) .and_then(|codec| codec.send(snapshot_message)) }).map(|_| ()) - .map_err(|e| println!("codec error: {:?}", e)) + .map_err(|e| println!("codec error: {:?}", e)) }).map_err(move |e| debug!(log, "error sending snapshot: {:?}", e)); let d = Delay::new(Instant::now() + Duration::from_secs(1)); @@ -404,16 +409,5 @@ mod test { let test_delay = Delay::new(test_timeout); runtime.block_on(test_delay).expect("runtime"); - - let single_name: Bytes = "complex.test.bioyino_single".into(); - let multi_name: Bytes = "complex.test.bioyino_multi".into(); - let shot_name: Bytes = "complex.test.bioyino_snapshot".into(); - LONG_CACHE.with(|c| { - assert_eq!(c.borrow().get(&shot_name), Some(&outmetric)); - }); - SHORT_CACHE.with(|c| { - assert_eq!(c.borrow().get(&single_name), Some(&outmetric)); - assert_eq!(c.borrow().get(&multi_name), Some(&outmetric)); - }); } } diff --git a/src/server.rs b/src/server.rs index 5c7aa19..ef1b74b 100644 --- a/src/server.rs +++ b/src/server.rs @@ -120,7 +120,7 @@ impl IntoFuture for StatsdServer { }; spawn( - chan.send(Task::Parse(ahash, buf.freeze())) + chan.send(Task::Parse(ahash, buf)) .map_err(|_| { DROPS.fetch_add(1, Ordering::Relaxed); }).map(|_| ()), diff --git a/src/task.rs b/src/task.rs index e8154d8..fbe74f4 100644 --- a/src/task.rs +++ b/src/task.rs @@ -1,5 +1,7 @@ +use std::collections::HashMap; use std::collections::hash_map::Entry; use std::sync::atomic::Ordering; +use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; use combine::Parser; @@ -8,15 +10,14 @@ use futures::sync::mpsc::UnboundedSender; use futures::sync::oneshot; use futures::{Future, Sink}; use tokio::runtime::current_thread::spawn; +use slog::Logger; use metric::Metric; use parser::metric_parser; use util::AggregateOptions; +use config::System; -use { - Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, LONG_CACHE, PARSE_ERRORS, PEER_ERRORS, - SHORT_CACHE, BUFFER_CACHE, -}; +use { Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, PARSE_ERRORS, PEER_ERRORS }; #[derive(Debug)] pub struct AggregateData { @@ -29,7 +30,7 @@ pub struct AggregateData { #[derive(Debug)] pub enum Task { - Parse(u64, Bytes), + Parse(u64, BytesMut), AddMetric(Bytes, Metric), AddMetrics(Vec<(Bytes, Metric)>), AddSnapshot(Vec<(Bytes, Metric)>), @@ -51,191 +52,247 @@ fn update_metric(cache: &mut Cache, name: Bytes, metric: Metric) { }; } -impl Task { - pub fn run(self) { - match self { - Task::Parse(addr, buf) => parse_and_insert(addr, buf), - Task::AddMetric(name, metric) => SHORT_CACHE.with(move |c| { - let mut short = c.borrow_mut(); - update_metric(&mut short, name, metric); - }), - Task::AddMetrics(mut list) => SHORT_CACHE.with(move |c| { - let mut short = c.borrow_mut(); - list.drain(..) - .map(|(name, metric)| update_metric(&mut short, name, metric)) - .last(); - }), - Task::AddSnapshot(mut list) => { - LONG_CACHE.with(move |c| { - // snapshots go to long cache to avoid being duplicated to other nodes - let mut long = c.borrow_mut(); +fn cut_bad(log: Option, buf: &mut Bytes) -> Option { + PARSE_ERRORS.fetch_add(1, Ordering::Relaxed); + match buf.iter().position(|&c| c == 10u8) { + Some(pos) if pos <= buf.len() - 1 => { + if let Some(log) = log { + warn!(log, "dropping buffer: {:?}", &buf[0..pos+1]); + } + buf.advance(pos + 1); + Some(pos) + } + Some(_) => None, + None => None, + } +} + +#[derive(Debug)] +pub struct TaskRunner { + long: HashMap>, + short: HashMap>, + buffers: HashMap, + config: Arc, + log: Logger, +} + +impl TaskRunner { + pub fn new(log: Logger, config: Arc, cap: usize) -> Self { + Self { + long: HashMap::with_capacity(cap), + short: HashMap::with_capacity(cap), + buffers: HashMap::with_capacity(cap), + config, + log, + } + } + + pub fn run(&mut self, task: Task) { + match task { + Task::Parse(addr, buf) => { + let log = if self.config.metrics.log_parse_errors { + Some(self.log.clone()) + } else { + None + }; + let mut buf = { + let len = buf.len(); + let (_, ref mut prev_buf) = self.buffers.entry(addr) + .and_modify(|(times, _)| { + *times += 1; + }) + .or_insert((0, BytesMut::with_capacity(len))); + prev_buf.reserve(buf.len()); + prev_buf.put(buf); + prev_buf.clone().freeze() + }; + + let parsed = self.parse_and_insert(log, buf); + self.buffers.entry(addr) + .and_modify(|(_, buf)| { + buf.advance(parsed); + }); + } + Task::AddMetric(name, metric) => + update_metric(&mut self.short, name, metric), + Task::AddMetrics(mut list) => { list.drain(..) - .map(|(name, metric)| update_metric(&mut long, name, metric)) + .map(|(name, metric)| update_metric(&mut self.short, name, metric)) .last(); - }) + } + Task::AddSnapshot(mut list) => { + // snapshots go to long cache to avoid being duplicated to other nodes + list.drain(..) + .map(|(name, metric)| update_metric(&mut self.long, name, metric)) + .last(); } Task::TakeSnapshot(channel) => { - let mut short = SHORT_CACHE.with(|c| { - let short = c.borrow().clone(); - c.borrow_mut().clear(); - short - }); + let mut short = HashMap::with_capacity(self.short.len()); + ::std::mem::swap(&mut short, &mut self.short); - channel.send(short.clone()).unwrap_or_else(|_| { - PEER_ERRORS.fetch_add(1, Ordering::Relaxed); - // TODO debug log - // println!("shapshot not sent"); - }); + // self.short now contains empty hashmap + // join a copy of data in short cache to long cache + short.iter() + .map(|(name, metric)| update_metric(&mut self.long, name.clone(), metric.clone())) + .last(); - // Aggregate short cache into long - LONG_CACHE.with(|c| { - let mut long = c.borrow_mut(); - short - .drain() - .map(|(name, metric)| update_metric(&mut long, name, metric)) - .last(); + // TODO: we could do this without additional copying if borrowck would allow us + // this. Maybe NLL would help us with that in Rust 2018: + //self.short + //.drain() + //.map(|(name, metric)| update_metric(&mut self.long, name, metric)) + // .last(); + + // give a copy of snapshot to requestor + channel.send(short).unwrap_or_else(|_| { + PEER_ERRORS.fetch_add(1, Ordering::Relaxed); + debug!(self.log, "shapshot not sent"); }); } Task::Rotate(channel) => { - LONG_CACHE.with(|c| { - let rotated = c.borrow().clone(); - c.borrow_mut().clear(); - channel.send(rotated).unwrap_or_else(|_| { - println!("rotated data not sent"); - DROPS.fetch_add(1, Ordering::Relaxed); - }); + let rotated = self.long.clone(); + self.long.clear(); + let log = self.log.clone(); + channel.send(rotated).unwrap_or_else(|_| { + debug!(log, "rotated data not sent"); + DROPS.fetch_add(1, Ordering::Relaxed); }); - } - Task::Aggregate(AggregateData { - mut buf, - name, - metric, - options, - response, - }) => { - let upd = if let Some(options) = options.update_counter { - if metric.update_counter > options.threshold { - // + 2 is for dots - let cut_len = options.prefix.len() + name.len() + options.suffix.len() + 2; - buf.reserve(cut_len); - if options.prefix.len() > 0 { - buf.put_slice(&options.prefix); - buf.put_slice(b"."); - } - - buf.put_slice(&name); - if options.suffix.len() > 0 { - buf.put_slice(b"."); - buf.put_slice(&options.suffix); - } - - let counter = buf.take().freeze(); - Some((counter, metric.update_counter.into())) - } else { - None - } - } else { - None - }; - metric - .into_iter() - .map(move |(suffix, value)| { - buf.extend_from_slice(&name); - buf.extend_from_slice(suffix.as_bytes()); - let name = buf.take().freeze(); - (name, value) - }).chain(upd) - .map(|data| { - spawn( - response - .clone() - .send(data) - .map_err(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }).map(|_| ()), - ); - }).last(); + self.buffers.retain(|_, (ref mut times, _)| { + *times += 1; + *times < 5 + }); } - } - } -} -fn cut_bad(buf: &mut Bytes) -> Option { - PARSE_ERRORS.fetch_add(1, Ordering::Relaxed); - match buf.iter().position(|&c| c == 10u8) { - Some(pos) if pos <= buf.len() - 1 => { - buf.advance(pos + 1); - Some(pos) + Task::Aggregate(data) => aggregate_task(data), } - Some(_) => None, - None => None, } -} -fn parse_and_insert(_addr: u64, mut buf: Bytes) { - //let prev_buf = BUFFER_CACHE.with(|c| { - // c.get_mut - //}); - // Cloned buf is shallow copy, so input and buf are the same bytes. - // We are going to parse the whole slice, so for parser we use input as readonly - // while buf follows the parser progress and is cut to get only names - // so they are zero-copied - let mut input: &[u8] = &(buf.clone()); - let mut parser = metric_parser::(); - loop { - let buflen = buf.len(); - match parser.parse(&input) { - Ok(((name, value, mtype, sampling), rest)) => { - // name is always at the beginning of the buf - let name = buf.split_to(name.len()); - buf.advance(buflen - rest.len() - name.len()); - input = rest; - - // check if name is valid UTF-8 - if let Err(_) = ::std::str::from_utf8(&name) { - if let Some(pos) = cut_bad(&mut buf) { - input = input.split_at(pos + 1).1; - continue; - } else { - break; - } - } + fn parse_and_insert(&mut self, log: Option, mut buf: Bytes) -> usize { + // Cloned buf is shallow copy, so input and buf are the same bytes. + // We are going to parse the whole slice, so for parser we use input as readonly + // while buf follows the parser progress and is cut to get only names + // so they are zero-copied + let mut input: &[u8] = &(buf.clone()); + let mut parser = metric_parser::(); + loop { + let buflen = buf.len(); + match parser.parse(&input) { + Ok(((name, value, mtype, sampling), rest)) => { + // name is always at the beginning of the buf + let name = buf.split_to(name.len()); + buf.advance(buflen - rest.len() - name.len()); + input = rest; - let metric = match Metric::::new(value, mtype, None, sampling) { - Ok(metric) => metric, - Err(_) => { - if let Some(pos) = cut_bad(&mut buf) { + // check if name is valid UTF-8 + if let Err(_) = ::std::str::from_utf8(&name) { + if let Some(pos) = cut_bad(log.clone(), &mut buf) { input = input.split_at(pos + 1).1; continue; } else { - break; + return rest.len() + //break; } } - }; - INGRESS_METRICS.fetch_add(1, Ordering::Relaxed); - SHORT_CACHE.with(|c| { - let mut short = c.borrow_mut(); - update_metric(&mut short, name, metric); - }); - if rest.len() == 0 { - break; + let metric = match Metric::::new(value, mtype, None, sampling) { + Ok(metric) => metric, + Err(_) => { + if let Some(pos) = cut_bad(log.clone(), &mut buf) { + input = input.split_at(pos + 1).1; + continue; + } else { + return rest.len() + //break; + } + } + }; + + INGRESS_METRICS.fetch_add(1, Ordering::Relaxed); + update_metric(&mut self.short, name, metric); + if rest.len() == 0 { + return 0 + //break; + } } - } - Err(UnexpectedParse::Eoi) => { - break; - } - Err(_e) => { - if let Some(pos) = cut_bad(&mut buf) { - input = input.split_at(pos + 1).1; - continue; - } else { - break; + Err(UnexpectedParse::Eoi) => { + return buflen; + //break; + } + Err(_e) => { + if let Some(pos) = cut_bad(log.clone(), &mut buf) { + input = input.split_at(pos + 1).1; + continue; + } else { + return input.len(); + //break; + } } } } } + + // used in tests in peer.rs + pub fn get_long_entry(&self, e: &Bytes) -> Option<&Metric> { + self.long.get(e) + } + pub fn get_short_entry(&self, e: &Bytes) -> Option<&Metric> { + self.short.get(e) + } + +} + +pub fn aggregate_task(data: AggregateData) { + let AggregateData { + mut buf, + name, + metric, + options, + response, + } = data; + let upd = if let Some(options) = options.update_counter { + if metric.update_counter > options.threshold { + // + 2 is for dots + let cut_len = options.prefix.len() + name.len() + options.suffix.len() + 2; + buf.reserve(cut_len); + if options.prefix.len() > 0 { + buf.put_slice(&options.prefix); + buf.put_slice(b"."); + } + + buf.put_slice(&name); + if options.suffix.len() > 0 { + buf.put_slice(b"."); + buf.put_slice(&options.suffix); + } + + let counter = buf.take().freeze(); + Some((counter, metric.update_counter.into())) + } else { + None + } + } else { + None + }; + + metric + .into_iter() + .map(move |(suffix, value)| { + buf.extend_from_slice(&name); + buf.extend_from_slice(suffix.as_bytes()); + let name = buf.take().freeze(); + (name, value) + }).chain(upd) + .map(|data| { + spawn( + response + .clone() + .send(data) + .map_err(|_| { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()), + ); + }).last(); } #[cfg(test)] @@ -243,6 +300,8 @@ mod tests { use super::*; use metric::MetricType; + use util::prepare_log; + #[test] fn parse_trashed_metric_buf() { let mut data = Bytes::new(); @@ -250,21 +309,21 @@ mod tests { b"trash\ngorets1:+1000|g\nTRASH\ngorets2:-1000|g|@0.5\nMORETrasH\nFUUU", ); - parse_and_insert(data); - - SHORT_CACHE.with(|c| { - let c = c.borrow(); - let key: Bytes = "gorets1".into(); - let metric = c.get(&key).unwrap().clone(); - assert_eq!(metric.value, 1000f64); - assert_eq!(metric.mtype, MetricType::Gauge(Some(1i8))); - assert_eq!(metric.sampling, None); - - let key: Bytes = "gorets2".into(); - let metric = c.get(&key).unwrap().clone(); - assert_eq!(metric.value, 1000f64); - assert_eq!(metric.mtype, MetricType::Gauge(Some(-1i8))); - assert_eq!(metric.sampling, Some(0.5f32)); - }); + let mut config = System::default(); + config.metrics.log_parse_errors = true; + let mut runner = TaskRunner::new(prepare_log("parse_thrashed"), Arc::new(config), 16); + runner.parse_and_insert(Some(prepare_log("thrashed")), data); + + let key: Bytes = "gorets1".into(); + let metric = runner.short.get(&key).unwrap().clone(); + assert_eq!(metric.value, 1000f64); + assert_eq!(metric.mtype, MetricType::Gauge(Some(1i8))); + assert_eq!(metric.sampling, None); + + let key: Bytes = "gorets2".into(); + let metric = runner.short.get(&key).unwrap().clone(); + assert_eq!(metric.value, 1000f64); + assert_eq!(metric.mtype, MetricType::Gauge(Some(-1i8))); + assert_eq!(metric.sampling, Some(0.5f32)); } } diff --git a/src/udp.rs b/src/udp.rs index caf6323..1512efb 100644 --- a/src/udp.rs +++ b/src/udp.rs @@ -211,7 +211,7 @@ pub(crate) fn start_sync_udp( } else { ichans.next().unwrap().clone() }; - chan.try_send(Task::Parse(ahash, buf.take().freeze())) + chan.try_send(Task::Parse(ahash, buf.take())) .map_err(|_| { warn!(log, "error sending buffer(queue full?)"); DROPS.fetch_add( diff --git a/src/util.rs b/src/util.rs index 3aa7ace..d2f7011 100644 --- a/src/util.rs +++ b/src/util.rs @@ -12,17 +12,27 @@ use futures::sync::mpsc::{Sender, UnboundedSender}; use futures::sync::oneshot; use futures::{Async, Future, IntoFuture, Poll, Sink, Stream}; use resolve::resolver; -use slog::Logger; +use slog::{Logger, Drain}; use tokio::executor::current_thread::spawn; use tokio::timer::{Delay, Interval}; use metric::{Metric, MetricType}; -use task::{AggregateData, Task}; +use task::{AggregateData, Task, aggregate_task}; use {Cache, Float}; use {AGG_ERRORS, DROPS, EGRESS, INGRESS, INGRESS_METRICS, PARSE_ERRORS, PEER_ERRORS}; use {ConsensusState, CONSENSUS_STATE, IS_LEADER}; +pub fn prepare_log(root: &'static str) -> Logger { + // Set logging + let decorator = slog_term::TermDecorator::new().build(); + let drain = slog_term::FullFormat::new(decorator).build().fuse(); + let filter = slog::LevelFilter::new(drain, slog::Level::Trace).fuse(); + let drain = slog_async::Async::new(filter).build().fuse(); + let rlog = slog::Logger::root(drain, o!("program"=>"test", "test"=>root)); + return rlog; +} + pub fn try_resolve(s: &str) -> SocketAddr { s.parse().unwrap_or_else(|_| { // for name that have failed to be parsed we try to resolve it via DNS @@ -279,25 +289,25 @@ impl IntoFuture for Aggregator { }).enumerate() .map(move |(num, (name, metric))| { let buf = BytesMut::with_capacity(1024); - let task = Task::Aggregate(AggregateData { + let task_data = AggregateData { buf, name: Bytes::from(name), metric, options: options.clone(), response: tx.clone(), - }); + }; if options.fast_aggregation { spawn( chans[num % chans.len()] .clone() - .send(task) + .send(Task::Aggregate(task_data)) .map(|_| ()) .map_err(|_| { DROPS.fetch_add(1, Ordering::Relaxed); }), ); } else { - task.run(); + aggregate_task(task_data); } }).last(); Ok(()) From 3860ff9ddc978927e430b0cb8c6f6565cb372a7b Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Fri, 9 Nov 2018 10:14:03 +0300 Subject: [PATCH 20/24] FIx buffer alloc bug --- src/udp.rs | 48 +++++++++++++++++++++++++++++++++--------------- 1 file changed, 33 insertions(+), 15 deletions(-) diff --git a/src/udp.rs b/src/udp.rs index 1512efb..d9cc58f 100644 --- a/src/udp.rs +++ b/src/udp.rs @@ -8,9 +8,9 @@ use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::thread; +use config::System; use server::StatsdServer; use task::Task; -use config::System; use {DROPS, INGRESS}; use bytes::{BufMut, BytesMut}; @@ -35,7 +35,7 @@ pub(crate) fn start_sync_udp( mm_async: bool, mm_timeout: u64, flush_flags: Arc>, - ) { +) { info!(log, "multimessage enabled, starting in sync UDP mode"; "socket-is-blocking"=>!mm_async, "packets"=>mm_packets); // It is crucial for recvmmsg to have one socket per many threads @@ -132,7 +132,7 @@ pub(crate) fn start_sync_udp( let mut chunk = iovec { iov_base: recv_buffer[i * rowsize..i * rowsize + rowsize].as_mut_ptr() as *mut c_void, - iov_len: rowsize, + iov_len: rowsize, }; chunks.push(chunk); // put the result to mheaders @@ -167,7 +167,7 @@ pub(crate) fn start_sync_udp( } else { null_mut() }, - ) + ) }; if res == 0 { @@ -182,10 +182,20 @@ pub(crate) fn start_sync_udp( INGRESS.fetch_add(mlen, Ordering::Relaxed); // create address entry in messagemap - let mut entry = bufmap - .entry(addrs[i]) - .or_insert(BytesMut::with_capacity(config.network.buffer_flush_length)); - // and put it's buffer there + let mut entry = bufmap.entry(addrs[i]).or_insert( + BytesMut::with_capacity(config.network.buffer_flush_length), + ); + + // check we can fit the buffer + if entry.remaining_mut() < mlen + 1 { + entry.reserve(if mlen > config.network.buffer_flush_length { + mlen + } else { + config.network.buffer_flush_length + }); + } + + // and put the buffer into the map entry.put(&recv_buffer[i * rowsize..i * rowsize + mlen]); // reset addres to be used in next cycle @@ -201,12 +211,20 @@ pub(crate) fn start_sync_udp( .drain() .map(|(addr, mut buf)| { // in some ideal world we want all values from the same host to be parsed by the - // same thread, but this could cause load unbalancing between - // threads TODO: make it an option in config + // same thread, but this could cause load unbalancing between threads in some + // corner cases, i.e. when only few hosts are sending most + // metrics + // TODO: we can work this around by fast-scanning buffer + // for newlines. if more than 2 newlines are there, buffer + // can be split into 3 parts and the middle part can be + // cropped from the buffer and relatively safely given to other nodes for + // parsing. It would be WAY better to do this in counting + // nodes rather than networking ones, but could be harder + // than it seems because of queue reordering let mut hasher = DefaultHasher::new(); hasher.write(&addr); let ahash = hasher.finish(); - let mut chan = if config.consistent_parsing { + let mut chan = if config.metrics.consistent_parsing { chans[ahash as usize % chlen].clone() } else { ichans.next().unwrap().clone() @@ -217,7 +235,7 @@ pub(crate) fn start_sync_udp( DROPS.fetch_add( messages as usize, Ordering::Relaxed, - ); + ); }).unwrap_or(()); }).last(); } @@ -247,7 +265,7 @@ pub(crate) fn start_async_udp( async_sockets: usize, bufsize: usize, flush_flags: Arc>, - ) { +) { info!(log, "multimessage is disabled, starting in async UDP mode"); // Create a pool of listener sockets @@ -287,7 +305,7 @@ pub(crate) fn start_async_udp( let socket = socket.try_clone().expect("cloning socket"); let socket = UdpSocket::from_std(socket, &::tokio::reactor::Handle::current()) - .expect("adding socket to event loop"); + .expect("adding socket to event loop"); let server = StatsdServer::new( socket, @@ -300,7 +318,7 @@ pub(crate) fn start_async_udp( readbuf, flush_flags.clone(), i, - ); + ); runtime.spawn(server.into_future()); } From 2a01fe81a7cd2e32611f33d8b0e87ac2b7072431 Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Fri, 9 Nov 2018 10:14:14 +0300 Subject: [PATCH 21/24] Move consistent_parsing option to metrics section --- src/config.rs | 52 +++++++++++++++---------------- src/main.rs | 86 +++++++++++++++++++++++++++------------------------ src/parser.rs | 54 ++++++++++++++++---------------- src/peer.rs | 85 ++++++++++++++++++++++++++------------------------ src/server.rs | 22 ++++++------- src/task.rs | 80 +++++++++++++++++++++++------------------------ src/util.rs | 72 +++++++++++++++++++++--------------------- 7 files changed, 230 insertions(+), 221 deletions(-) diff --git a/src/config.rs b/src/config.rs index 489ca5c..cda53cc 100644 --- a/src/config.rs +++ b/src/config.rs @@ -55,9 +55,6 @@ pub struct System { /// Consensus kind to use pub consensus: ConsensusKind, - - /// Consistent parsing - pub consistent_parsing: bool, } impl Default for System { @@ -76,7 +73,6 @@ impl Default for System { start_as_leader: false, stats_prefix: "resources.monitoring.bioyino".to_string(), consensus: ConsensusKind::None, - consistent_parsing: true, } } } @@ -101,6 +97,9 @@ pub struct Metrics { /// Aggregate faster at the price or probably loosing some incoming metrics pub fast_aggregation: bool, + /// Consistent parsing + pub consistent_parsing: bool, + /// Whether we should spam parsing errors in logs pub log_parse_errors: bool, } @@ -114,6 +113,7 @@ impl Default for Metrics { update_counter_suffix: String::new(), update_counter_threshold: 200, fast_aggregation: true, + consistent_parsing: true, log_parse_errors: false, } } @@ -317,35 +317,35 @@ impl System { let app = app_from_crate!() .arg( Arg::with_name("config") - .help("configuration file path") - .long("config") - .short("c") - .required(true) - .takes_value(true) - .default_value("/etc/bioyino/bioyino.toml"), - ).arg( - Arg::with_name("verbosity") + .help("configuration file path") + .long("config") + .short("c") + .required(true) + .takes_value(true) + .default_value("/etc/bioyino/bioyino.toml"), + ).arg( + Arg::with_name("verbosity") .short("v") .help("logging level") .takes_value(true), - ).subcommand( - SubCommand::with_name("query") - .about("send a management command to running bioyino server") - .arg( - Arg::with_name("host") + ).subcommand( + SubCommand::with_name("query") + .about("send a management command to running bioyino server") + .arg( + Arg::with_name("host") .short("h") .default_value("127.0.0.1:8137"), - ).subcommand(SubCommand::with_name("status").about("get server state")) - .subcommand( - SubCommand::with_name("consensus") + ).subcommand(SubCommand::with_name("status").about("get server state")) + .subcommand( + SubCommand::with_name("consensus") .arg(Arg::with_name("action").index(1)) .arg( Arg::with_name("leader_action") - .index(2) - .default_value("unchanged"), - ), - ), - ).get_matches(); + .index(2) + .default_value("unchanged"), + ), + ), + ).get_matches(); let config = value_t!(app.value_of("config"), String).expect("config file must be string"); let mut file = File::open(&config).expect(&format!("opening config file at {}", &config)); @@ -370,7 +370,7 @@ impl System { ( system, Command::Query(MgmtCommand::ConsensusCommand(c_action, l_action), server), - ) + ) } else { // shold be unreachable unreachable!("clap bug?") diff --git a/src/main.rs b/src/main.rs index 26a49bb..37019e8 100644 --- a/src/main.rs +++ b/src/main.rs @@ -70,7 +70,7 @@ use std::time::{self, Duration, Instant, SystemTime}; use slog::{Drain, Level}; -use bytes::{Bytes}; +use bytes::Bytes; use futures::future::{empty, ok}; use futures::sync::mpsc; use futures::{Future, IntoFuture, Stream}; @@ -140,13 +140,13 @@ pub enum ConsensusKind { } lazy_static! { - pub static ref CONSENSUS_STATE: Mutex = { Mutex::new(ConsensusState::Disabled) }; + pub static ref CONSENSUS_STATE: Mutex = + { Mutex::new(ConsensusState::Disabled) }; } pub static IS_LEADER: AtomicBool = ATOMIC_BOOL_INIT; fn main() { - let (system, command) = System::load(); let mut config = system.clone(); @@ -163,40 +163,40 @@ fn main() { mm_async, mm_timeout, buffer_flush_time, - buffer_flush_length :_, + buffer_flush_length: _, greens, async_sockets, nodes, snapshot_interval, }, - raft, - consul: - Consul { - start_as: consul_start_as, - agent, - session_ttl: consul_session_ttl, - renew_time: consul_renew_time, - key_name: consul_key, - }, - metrics: - Metrics { - // max_metrics, - mut count_updates, - update_counter_prefix, - update_counter_suffix, - update_counter_threshold, - fast_aggregation, - log_parse_errors: _, - }, - carbon, - n_threads, - w_threads, - stats_interval: s_interval, - task_queue_size, - start_as_leader, - stats_prefix, - consensus, - consistent_parsing: _, + raft, + consul: + Consul { + start_as: consul_start_as, + agent, + session_ttl: consul_session_ttl, + renew_time: consul_renew_time, + key_name: consul_key, + }, + metrics: + Metrics { + // max_metrics, + mut count_updates, + update_counter_prefix, + update_counter_suffix, + update_counter_threshold, + fast_aggregation, + consistent_parsing: _, + log_parse_errors: _, + }, + carbon, + n_threads, + w_threads, + stats_interval: s_interval, + task_queue_size, + start_as_leader, + stats_prefix, + consensus, } = system; let verbosity = Level::from_str(&verbosity).expect("bad verbosity"); @@ -269,7 +269,12 @@ fn main() { .spawn(move || { let runner = TaskRunner::new(tlog, cf, 8192); let mut runtime = Runtime::new().expect("creating runtime for counting worker"); - let future = rx.fold(runner, move |mut runner, task: Task| {runner.run(task); Ok(runner)}).map(|_|()).map_err(|_|()); + let future = rx + .fold(runner, move |mut runner, task: Task| { + runner.run(task); + Ok(runner) + }).map(|_| ()) + .map_err(|_| ()); // let future = rx.for_each(|task: Task| ok(runner.run(task))); runtime.block_on(future).expect("worker thread failed"); }).expect("starting counting worker thread"); @@ -292,11 +297,11 @@ fn main() { nodes.clone(), Duration::from_millis(snapshot_interval as u64), &chans, - ).into_future() - .map_err(move |e| { - PEER_ERRORS.fetch_add(1, Ordering::Relaxed); - info!(snap_err_log, "error sending snapshot";"error"=>format!("{}", e)); - }); + ).into_future() + .map_err(move |e| { + PEER_ERRORS.fetch_add(1, Ordering::Relaxed); + info!(snap_err_log, "error sending snapshot";"error"=>format!("{}", e)); + }); runtime.spawn(snapshot); // settings safe for asap restart @@ -490,7 +495,6 @@ fn main() { } } - if buffer_flush_time > 0 { let dur = Duration::from_millis(buffer_flush_time); let flush_timer = Interval::new(Instant::now() + dur, dur); @@ -525,7 +529,7 @@ fn main() { mm_async, mm_timeout, flush_flags.clone(), - ); + ); } else { start_async_udp( log, @@ -537,7 +541,7 @@ fn main() { async_sockets, bufsize, flush_flags.clone(), - ); + ); } runtime diff --git a/src/parser.rs b/src/parser.rs index a8a7fbf..3f28a2b 100644 --- a/src/parser.rs +++ b/src/parser.rs @@ -4,11 +4,11 @@ use std::str::from_utf8; use std::str::FromStr; use combine::byte::{byte, bytes, newline}; -use combine::parser::byte::digit; use combine::combinator::{eof, skip_many}; use combine::error::UnexpectedParse; -use combine::parser::range::{take_while1, recognize}; -use combine::{optional, Parser, skip_many1}; +use combine::parser::byte::digit; +use combine::parser::range::{recognize, take_while1}; +use combine::{optional, skip_many1, Parser}; use metric::MetricType; @@ -17,23 +17,23 @@ use metric::MetricType; pub fn metric_parser<'a, F>( ) -> impl Parser, Option), Input = &'a [u8]> where -F: FromStr -+ Add -+ AddAssign -+ Sub -+ SubAssign -+ Div -+ Mul -+ Neg -+ PartialOrd -+ Into -+ From -+ Debug -+ Default -+ Clone -+ Copy -+ PartialEq -+ Sync, + F: FromStr + + Add + + AddAssign + + Sub + + SubAssign + + Div + + Mul + + Neg + + PartialOrd + + Into + + From + + Debug + + Default + + Clone + + Copy + + PartialEq + + Sync, { // This will parse metric name and separator let name = take_while1(|c: u8| c != b':' && c != b'\n').skip(byte(b':')); @@ -59,11 +59,13 @@ F: FromStr // .or(byte(b'h').map(|_| MetricType::Histrogram)) ; - let unsigned_float = skip_many1(digit()).and(optional( - (byte(b'.'), skip_many1(digit())) - )).and(optional( - (byte(b'e'), optional(byte(b'+').or(byte(b'-'))), skip_many1(digit())) - )); + let unsigned_float = skip_many1(digit()) + .and(optional((byte(b'.'), skip_many1(digit())))) + .and(optional(( + byte(b'e'), + optional(byte(b'+').or(byte(b'-'))), + skip_many1(digit()), + ))); let sampling = (bytes(b"|@"), recognize(unsigned_float)).and_then(|(_, value)| { // TODO replace from_utf8 with handmade parser removing recognize @@ -78,7 +80,7 @@ F: FromStr mtype, optional(sampling), skip_many(newline()).or(eof()), - ) + ) .and_then(|(name, sign, mut value, mtype, sampling, _)| { let mtype = if let MetricType::Gauge(_) = mtype { MetricType::Gauge(sign) diff --git a/src/peer.rs b/src/peer.rs index e7f2106..d0700af 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -116,7 +116,7 @@ fn parse_and_send( reader: cmsg::Reader, next_chan: Sender, log: Logger, - ) -> Result<(), MetricError> { +) -> Result<(), MetricError> { match reader.which().map_err(MetricError::CapnpSchema)? { cmsg::Single(reader) => { let reader = reader.map_err(MetricError::Capnp)?; @@ -185,7 +185,7 @@ impl NativeProtocolSnapshot { nodes: Vec, interval: Duration, chans: &Vec>, - ) -> Self { + ) -> Self { Self { log: log.new(o!("source"=>"peer-client")), nodes, @@ -219,18 +219,16 @@ impl IntoFuture for NativeProtocolSnapshot { let (tx, rx) = oneshot::channel(); spawn(chan.send(Task::TakeSnapshot(tx)).then(|_| Ok(()))); rx - }) - .collect::>(); + }).collect::>(); let get_metrics = join_all(metrics) .map_err(|_| { PEER_ERRORS.fetch_add(1, Ordering::Relaxed); PeerError::TaskSend - }) - .and_then(move |mut metrics| { - metrics.retain(|m| m.len() > 0); - Ok(metrics) - }); + }).and_then(move |mut metrics| { + metrics.retain(|m| m.len() > 0); + Ok(metrics) + }); // All nodes have to receive the same metrics // so we don't parallel connections and metrics fetching @@ -248,40 +246,41 @@ impl IntoFuture for NativeProtocolSnapshot { TcpStream::connect(&address) .map_err(|e| PeerError::Io(e)) .and_then(move |conn| { - let codec = ::capnp_futures::serialize::Transport::new(conn, ReaderOptions::default()); + let codec = ::capnp_futures::serialize::Transport::new( + conn, + ReaderOptions::default(), + ); let mut snapshot_message = Builder::new_default(); { - let builder = snapshot_message.init_root::<::protocol_capnp::message::Builder>(); - let flat_len = metrics - .iter() - .flat_map(|hmap| hmap.iter()) - .count(); + let builder = snapshot_message + .init_root::<::protocol_capnp::message::Builder>( + ); + let flat_len = + metrics.iter().flat_map(|hmap| hmap.iter()).count(); let mut multi_metric = builder.init_snapshot(flat_len as u32); metrics .into_iter() .flat_map(|hmap| hmap.into_iter()) .enumerate() .map(|(idx, (name, metric))| { - let mut c_metric = multi_metric.reborrow().get(idx as u32); - let name = unsafe { ::std::str::from_utf8_unchecked(&name) }; + let mut c_metric = + multi_metric.reborrow().get(idx as u32); + let name = + unsafe { ::std::str::from_utf8_unchecked(&name) }; c_metric.set_name(name); metric.fill_capnp(&mut c_metric); - }) - .last(); + }).last(); } codec.send(snapshot_message).map(|_| ()).map_err(move |e| { debug!(elog, "codec error"; "error"=>e.to_string()); PeerError::Capnp(e) }) - }) - .map_err(move |e| { - PEER_ERRORS.fetch_add(1, Ordering::Relaxed); - debug!(dlog, "error sending snapshot: {}", e) - }) - .then(|_| Ok(())) // we don't want to fail the whole timer cycle because of one send error - }) - .collect::>(); + }).map_err(move |e| { + PEER_ERRORS.fetch_add(1, Ordering::Relaxed); + debug!(dlog, "error sending snapshot: {}", e) + }).then(|_| Ok(())) // we don't want to fail the whole timer cycle because of one send error + }).collect::>(); join_all(clients).map(|_| ()) }) }); @@ -293,8 +292,8 @@ impl IntoFuture for NativeProtocolSnapshot { mod test { use std::net::SocketAddr; - use std::time::{SystemTime, UNIX_EPOCH}; use std::sync::Arc; + use std::time::{SystemTime, UNIX_EPOCH}; use bytes::Bytes; use capnp::message::Builder; @@ -303,10 +302,10 @@ mod test { use tokio::runtime::current_thread::Runtime; use tokio::timer::Delay; - use util::prepare_log; + use config::System; use metric::{Metric, MetricType}; use task::TaskRunner; - use config::System; + use util::prepare_log; use super::*; @@ -348,16 +347,20 @@ mod test { let metric = outmetric.clone(); let (mut runtime, rx, address) = prepare_runtime_with_server(log.clone()); - let future = rx.fold(runner, move |mut runner, task: Task| {runner.run(task); Ok(runner)}).and_then(move |runner| { - let single_name: Bytes = "complex.test.bioyino_single".into(); - let multi_name: Bytes = "complex.test.bioyino_multi".into(); - let shot_name: Bytes = "complex.test.bioyino_snapshot".into(); - assert_eq!(runner.get_long_entry(&shot_name), Some(&outmetric)); - assert_eq!(runner.get_short_entry(&single_name), Some(&outmetric)); - assert_eq!(runner.get_short_entry(&multi_name), Some(&outmetric)); - - Ok(()) - }).map_err(|_|panic!("error in the future")); + let future = rx + .fold(runner, move |mut runner, task: Task| { + runner.run(task); + Ok(runner) + }).and_then(move |runner| { + let single_name: Bytes = "complex.test.bioyino_single".into(); + let multi_name: Bytes = "complex.test.bioyino_multi".into(); + let shot_name: Bytes = "complex.test.bioyino_snapshot".into(); + assert_eq!(runner.get_long_entry(&shot_name), Some(&outmetric)); + assert_eq!(runner.get_short_entry(&single_name), Some(&outmetric)); + assert_eq!(runner.get_short_entry(&multi_name), Some(&outmetric)); + + Ok(()) + }).map_err(|_| panic!("error in the future")); runtime.spawn(future); let sender = TcpStream::connect(&address) @@ -400,7 +403,7 @@ mod test { .send(multi_message) .and_then(|codec| codec.send(snapshot_message)) }).map(|_| ()) - .map_err(|e| println!("codec error: {:?}", e)) + .map_err(|e| println!("codec error: {:?}", e)) }).map_err(move |e| debug!(log, "error sending snapshot: {:?}", e)); let d = Delay::new(Instant::now() + Duration::from_secs(1)); diff --git a/src/server.rs b/src/server.rs index ef1b74b..590c5c4 100644 --- a/src/server.rs +++ b/src/server.rs @@ -13,8 +13,8 @@ use futures::{Future, IntoFuture, Sink}; use tokio::executor::current_thread::spawn; use tokio::net::UdpSocket; -use task::Task; use config::System; +use task::Task; #[derive(Debug)] pub struct StatsdServer { @@ -42,7 +42,7 @@ impl StatsdServer { readbuf: BytesMut, flush_flags: Arc>, thread_idx: usize, - ) -> Self { + ) -> Self { Self { socket, chans, @@ -106,7 +106,7 @@ impl IntoFuture for StatsdServer { let mut hasher = DefaultHasher::new(); addr.hash(&mut hasher); let ahash = hasher.finish(); - let chan = if config.consistent_parsing { + let chan = if config.metrics.consistent_parsing { let chlen = chans.len(); chans[ahash as usize % chlen].clone() } else { @@ -121,10 +121,10 @@ impl IntoFuture for StatsdServer { spawn( chan.send(Task::Parse(ahash, buf)) - .map_err(|_| { - DROPS.fetch_add(1, Ordering::Relaxed); - }).map(|_| ()), - ) + .map_err(|_| { + DROPS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()), + ) }).last(); spawn( @@ -139,8 +139,8 @@ impl IntoFuture for StatsdServer { received, flush_flags, thread_idx, - ).into_future(), - ); + ).into_future(), + ); } else { spawn( StatsdServer::new( @@ -154,8 +154,8 @@ impl IntoFuture for StatsdServer { received, flush_flags, thread_idx, - ).into_future(), - ); + ).into_future(), + ); } Ok(()) }); diff --git a/src/task.rs b/src/task.rs index fbe74f4..ef050b3 100644 --- a/src/task.rs +++ b/src/task.rs @@ -1,23 +1,23 @@ -use std::collections::HashMap; use std::collections::hash_map::Entry; +use std::collections::HashMap; use std::sync::atomic::Ordering; use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; -use combine::Parser; use combine::error::UnexpectedParse; +use combine::Parser; use futures::sync::mpsc::UnboundedSender; use futures::sync::oneshot; use futures::{Future, Sink}; -use tokio::runtime::current_thread::spawn; use slog::Logger; +use tokio::runtime::current_thread::spawn; +use config::System; use metric::Metric; use parser::metric_parser; use util::AggregateOptions; -use config::System; -use { Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, PARSE_ERRORS, PEER_ERRORS }; +use {Cache, Float, AGG_ERRORS, DROPS, INGRESS_METRICS, PARSE_ERRORS, PEER_ERRORS}; #[derive(Debug)] pub struct AggregateData { @@ -57,7 +57,7 @@ fn cut_bad(log: Option, buf: &mut Bytes) -> Option { match buf.iter().position(|&c| c == 10u8) { Some(pos) if pos <= buf.len() - 1 => { if let Some(log) = log { - warn!(log, "dropping buffer: {:?}", &buf[0..pos+1]); + warn!(log, "dropping buffer: {:?}", &buf[0..pos + 1]); } buf.advance(pos + 1); Some(pos) @@ -97,29 +97,28 @@ impl TaskRunner { }; let mut buf = { let len = buf.len(); - let (_, ref mut prev_buf) = self.buffers.entry(addr) + let (_, ref mut prev_buf) = self + .buffers + .entry(addr) .and_modify(|(times, _)| { *times += 1; - }) - .or_insert((0, BytesMut::with_capacity(len))); + }).or_insert((0, BytesMut::with_capacity(len))); prev_buf.reserve(buf.len()); prev_buf.put(buf); prev_buf.clone().freeze() }; let parsed = self.parse_and_insert(log, buf); - self.buffers.entry(addr) - .and_modify(|(_, buf)| { - buf.advance(parsed); - }); + self.buffers.entry(addr).and_modify(|(_, buf)| { + buf.advance(parsed); + }); + } + Task::AddMetric(name, metric) => update_metric(&mut self.short, name, metric), + Task::AddMetrics(mut list) => { + list.drain(..) + .map(|(name, metric)| update_metric(&mut self.short, name, metric)) + .last(); } - Task::AddMetric(name, metric) => - update_metric(&mut self.short, name, metric), - Task::AddMetrics(mut list) => { - list.drain(..) - .map(|(name, metric)| update_metric(&mut self.short, name, metric)) - .last(); - } Task::AddSnapshot(mut list) => { // snapshots go to long cache to avoid being duplicated to other nodes list.drain(..) @@ -132,9 +131,11 @@ impl TaskRunner { // self.short now contains empty hashmap // join a copy of data in short cache to long cache - short.iter() - .map(|(name, metric)| update_metric(&mut self.long, name.clone(), metric.clone())) - .last(); + short + .iter() + .map(|(name, metric)| { + update_metric(&mut self.long, name.clone(), metric.clone()) + }).last(); // TODO: we could do this without additional copying if borrowck would allow us // this. Maybe NLL would help us with that in Rust 2018: @@ -190,8 +191,8 @@ impl TaskRunner { input = input.split_at(pos + 1).1; continue; } else { - return rest.len() - //break; + return rest.len(); + //break; } } @@ -202,8 +203,8 @@ impl TaskRunner { input = input.split_at(pos + 1).1; continue; } else { - return rest.len() - //break; + return rest.len(); + //break; } } }; @@ -211,8 +212,8 @@ impl TaskRunner { INGRESS_METRICS.fetch_add(1, Ordering::Relaxed); update_metric(&mut self.short, name, metric); if rest.len() == 0 { - return 0 - //break; + return 0; + //break; } } Err(UnexpectedParse::Eoi) => { @@ -239,7 +240,6 @@ impl TaskRunner { pub fn get_short_entry(&self, e: &Bytes) -> Option<&Metric> { self.short.get(e) } - } pub fn aggregate_task(data: AggregateData) { @@ -283,16 +283,16 @@ pub fn aggregate_task(data: AggregateData) { let name = buf.take().freeze(); (name, value) }).chain(upd) - .map(|data| { - spawn( - response - .clone() - .send(data) - .map_err(|_| { - AGG_ERRORS.fetch_add(1, Ordering::Relaxed); - }).map(|_| ()), + .map(|data| { + spawn( + response + .clone() + .send(data) + .map_err(|_| { + AGG_ERRORS.fetch_add(1, Ordering::Relaxed); + }).map(|_| ()), ); - }).last(); + }).last(); } #[cfg(test)] @@ -307,7 +307,7 @@ mod tests { let mut data = Bytes::new(); data.extend_from_slice( b"trash\ngorets1:+1000|g\nTRASH\ngorets2:-1000|g|@0.5\nMORETrasH\nFUUU", - ); + ); let mut config = System::default(); config.metrics.log_parse_errors = true; diff --git a/src/util.rs b/src/util.rs index d2f7011..8e1f0eb 100644 --- a/src/util.rs +++ b/src/util.rs @@ -12,12 +12,12 @@ use futures::sync::mpsc::{Sender, UnboundedSender}; use futures::sync::oneshot; use futures::{Async, Future, IntoFuture, Poll, Sink, Stream}; use resolve::resolver; -use slog::{Logger, Drain}; +use slog::{Drain, Logger}; use tokio::executor::current_thread::spawn; use tokio::timer::{Delay, Interval}; use metric::{Metric, MetricType}; -use task::{AggregateData, Task, aggregate_task}; +use task::{aggregate_task, AggregateData, Task}; use {Cache, Float}; use {AGG_ERRORS, DROPS, EGRESS, INGRESS, INGRESS_METRICS, PARSE_ERRORS, PEER_ERRORS}; @@ -227,7 +227,7 @@ impl Aggregator { chans: Vec>, tx: UnboundedSender<(Bytes, Float)>, log: Logger, - ) -> Self { + ) -> Self { Self { options, chans, @@ -287,31 +287,31 @@ impl IntoFuture for Aggregator { .inspect(|_| { EGRESS.fetch_add(1, Ordering::Relaxed); }).enumerate() - .map(move |(num, (name, metric))| { - let buf = BytesMut::with_capacity(1024); - let task_data = AggregateData { - buf, - name: Bytes::from(name), - metric, - options: options.clone(), - response: tx.clone(), - }; - if options.fast_aggregation { - spawn( - chans[num % chans.len()] - .clone() - .send(Task::Aggregate(task_data)) - .map(|_| ()) - .map_err(|_| { - DROPS.fetch_add(1, Ordering::Relaxed); - }), + .map(move |(num, (name, metric))| { + let buf = BytesMut::with_capacity(1024); + let task_data = AggregateData { + buf, + name: Bytes::from(name), + metric, + options: options.clone(), + response: tx.clone(), + }; + if options.fast_aggregation { + spawn( + chans[num % chans.len()] + .clone() + .send(Task::Aggregate(task_data)) + .map(|_| ()) + .map_err(|_| { + DROPS.fetch_add(1, Ordering::Relaxed); + }), ); - } else { - aggregate_task(task_data); - } - }).last(); + } else { + aggregate_task(task_data); + } + }).last(); Ok(()) - //}); + //}); /* * TODO: this was an expermient for multithreade aggregation in separate threadpol with rayon * it worked, but needs more work to be in prod @@ -345,7 +345,7 @@ impl IntoFuture for Aggregator { }); Ok(()) */ }); - Box::new(aggregate) + Box::new(aggregate) } else { // only get metrics from threads let not_leader = futures_unordered(metrics).for_each(|_| Ok(())); @@ -375,16 +375,16 @@ impl Default for BackoffRetryBuilder { impl BackoffRetryBuilder { pub fn spawn(self, action: F) -> BackoffRetry - where + where F: IntoFuture + Clone, - { - let inner = Either::A(action.clone().into_future()); - BackoffRetry { - action, - inner: inner, - options: self, - } + { + let inner = Either::A(action.clone().into_future()); + BackoffRetry { + action, + inner: inner, + options: self, } + } } /// TCP client that is able to reconnect with customizable settings @@ -396,7 +396,7 @@ pub struct BackoffRetry { impl Future for BackoffRetry where -F: IntoFuture + Clone, + F: IntoFuture + Clone, { type Item = F::Item; type Error = Option; From 181be3ff982c4d5baafb70202c7b26398f0250ee Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Fri, 9 Nov 2018 10:31:20 +0300 Subject: [PATCH 22/24] Fix buffer unuse counter --- src/task.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/task.rs b/src/task.rs index ef050b3..21ee7d1 100644 --- a/src/task.rs +++ b/src/task.rs @@ -101,7 +101,7 @@ impl TaskRunner { .buffers .entry(addr) .and_modify(|(times, _)| { - *times += 1; + *times = 0; }).or_insert((0, BytesMut::with_capacity(len))); prev_buf.reserve(buf.len()); prev_buf.put(buf); From d0cf729f675e8012fd8b7f8d94444ff0dc257b1e Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Fri, 9 Nov 2018 11:08:39 +0300 Subject: [PATCH 23/24] Add changelog, update readme --- CHANGELOG.md | 18 ++++++++++++++++++ README.md | 10 +++++----- 2 files changed, 23 insertions(+), 5 deletions(-) create mode 100644 CHANGELOG.md diff --git a/CHANGELOG.md b/CHANGELOG.md new file mode 100644 index 0000000..740075b --- /dev/null +++ b/CHANGELOG.md @@ -0,0 +1,18 @@ + Release 0.4.0 + +This release is notable by internal Raft implementation and schema-based node interconnection, opening a route for large, cluster-aware installations. + +User-visible changes: + * Built-in Raft implemented: multi-node configuration can now be standalone(not requiring Consul) when running on 3 or more nodes + * Snapshotting has been reworked. Serialization layer migrated to Cap'n'Proto for backwards compatibility. + * Metrics can now be received using Cap'n'Proto representation in addition to statsd format. This opens a possibility for agent-mode operation, where multiple bioyinos gather metrics in some place, closer to producer, and sending their snapshots to central place for aggregation using TCP and allowing to avoid UDP inconsistency problems. + * Commands for changing internal state now allow to change consensus and leadership state independently, making possible to manage server leadership state flexibly in critical situations. + * Commanding interface is now HTTP, which allows to interact with bioyino without having the original binary + * Carbon backend client is now prone to network problems, trying to reconnect if/when carbon server is inaccessible + * Incoming buffers are now almost-guaranteed to be delivered for aggregation after some time or amount of data received, which wasn't true for 0.3.0 and frustrated almost every early-adopter + + Developer-visible changes: + * Most of buffer and string processing is now zero-copy using the excellent `bytes` crate + * Buffer processing is now made per incoming host, giving better support for hosts sending many metrics to a single UDP socket + * Lots of bufs fixed including parsing and some corner case processing + * Library dependencies are updated to lastest versions of everything including `hyper`, `tokio`, `futures`, `combine`, etc. diff --git a/README.md b/README.md index a5737de..1e8131b 100644 --- a/README.md +++ b/README.md @@ -2,8 +2,7 @@ The StatsD server written in Rust # Description # -A fully statsd-protocol compliant server with carbon backend -This server was written as a drop-in replacement for now unsupported [github's brubeck](https://github.com/github/brubeck/issues). +Bioyino is a distributed statsd-protocol server with carbon backend. # Features # @@ -11,16 +10,18 @@ This server was written as a drop-in replacement for now unsupported [github's b * fault tolerant: metrics are replicated to all nodes in the cluster * clustering: all nodes gather and replicate metrics, but only leader sends metrics to backend * precise: 64-bit floats, full metric set is stored in memory (for metric types that require post-processing), no approximation algorithms involved +* standalone: can work without external services * safety and security: written in memory-safe language -* networking is separate from counting to avoid dropping UDP packets as much as possible +* networking tries to do it's best to avoid dropping UDP packets as much as possible * networking is asynchronous * small memory footprint and low CPU consumption # Status # -Currently works, being tested on production-grade metric stream (~1,5M metrics per second) +Currently works in production at Avito, processing production-grade metric stream (~4M metrics per second on 3 nodes) # Installing # Do the usual Rust-program build-install cycle +Please note, that building is always tested on latest stable version of Rust. ``` $ git clone @@ -38,4 +39,3 @@ You can help project by doing the following: * create issues to request new features * add new features, like new metric types * test the server on your environment and creating new issues if/when bugs found - From bf5f70670bb3921db6942d1d385a6acd24cafe5d Mon Sep 17 00:00:00 2001 From: Sergey Noskov Date: Fri, 9 Nov 2018 11:09:46 +0300 Subject: [PATCH 24/24] Fix heading in changelog --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 740075b..3b9b060 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,4 @@ - Release 0.4.0 +# Release 0.4.0 # This release is notable by internal Raft implementation and schema-based node interconnection, opening a route for large, cluster-aware installations.