diff --git a/CHANGELOG.md b/CHANGELOG.md index 8e5b634..9829553 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,8 +1,40 @@ -# Release 0.7.1 # +# Release 0.8.0 # +**WARNING!** This release has **lots** of incompatible changes. Please make sure to read the section below before doing +important changes in critical infrastructures. -* fixed a bug when all snapshots were going to first thread, making this thread more busy than others bottlenecking aggregation +## Incompatible changes ## +* Consul support has been completely removed. Ensure to remove the consul section from your configuration files and use internal Raft. +* Threading model received a big rework and should lead to lower memory consumption and lower latencies, especially on higher metric cardinalities. See docs/threading.md for more details. +* Due to changes in threading: + * the following options are removed: `aggregation.mode`, `aggregation.threads` + * the following option has been added: `p-threads` + * the `c-threads` option is now `a-threads` +* statsd sampling rate is now completely supported +* The peer protocol and it's capnp schema is now considered only for internal use. It received a new version and is better structured for this purpose and internal +metrics representation. Due to this changes: + * the new option `network.peer-protocol` has been added to specify an exact version + * version 2 **is the default value in 0.8.0**, please consider setting `network.peer-protocol = "1"` explicitly in client configs before upgrading + * version 1 will be removed in 0.9.0, all users are recommended to migrate to using v2 since it's release +* diff-counter metric type has been deprecated to being unituitive and therefore avoided of being unused by anyone +* bioynio's own internal metric - `egress` is now called `egress-carbon` which better points to it's real meaning + +See config.toml for further configuration instructions + +## Major changes ## +* `.rate` aggregate has been added to show number of incoming values per second +* new internal own metrics added - `egress-peer` - showing number of metrics sent to peers per second + +## Internal changes ## +* the internal structure of metrics has changed to make a type system more helping +* many libraries have their versions updated to latest major versions + +## Minor changes +* bioyino can now be compiled with 32-bit floats instead of 64-bit. This may bring some storage economy at the price of +lower precision +* two new internal metrics has been added: + * `slow-q-len` - number of pending tasks in "slow" threadpool queue + * `ingress-metrics-peer` - number of metrics received via TCP - # Release 0.7.0 # ## Incompatible changes ## diff --git a/Cargo.toml b/Cargo.toml index 1a6c8b4..8747628 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "bioyino" -version = "0.7.2" +version = "0.8.0" authors = ["Sergey Noskov aka Albibek "] description = "StatsD-compatible, high-performance, fault-tolerant metric aggregator" edition = "2018" @@ -18,27 +18,30 @@ thiserror="^1.0" libc="^0.2" num_cpus="^1.8" -futures="^0.1" -tokio="^0.1" +futures1 = { package = "futures", version = "^0.1" } +tokio1 = { package = "tokio", version = "^0.1" } tokio-io="^0.1" #tokio-codec="^0.1" # new wave -tokio2 = { package = "tokio", version = "^0.2", features = ["net", "time", "sync", "macros", "io-util", "io-driver", "rt-core", "rt-util", "rt-threaded"] } -futures3 = { package = "futures", version = "^0.3", features = ["compat"] } -hyper13 = { package = "hyper", version = "^0.13" } -trust-dns-resolver = { version = "^0.19" } +tokio = { version = "^1.5", features = ["net", "time", "sync", "macros", "rt", "rt-multi-thread", "io-util"] } +tokio-stream = { version = "^0.1", features = ["sync" ]} +tokio-util = { version = "^0.6", features = ["codec", "compat"] } +futures = { version = "^0.3", features = ["compat"] } +hyper = { version = "^0.14", features = ["client", "server", "http1", "http2", "tcp"] } +trust-dns-resolver = { version = "^0.20" } url = "^2.1" -tokio-util = { version = "^0.3", features = ["codec", "compat"] } -bytes = { version = "^0.5", features = [ "serde" ] } -ring-channel = {version = "^0.8", features = ["futures_api"]} -capnp = "^0.13" -capnp-futures = "^0.13" +bytes = { version = "^1.0", features = [ "serde" ] } +async-channel = "^1.6" +ring-channel = { version = "^0.9", features = ["futures_api"] } +capnp = "^0.14" +capnp-futures = "^0.14" +crossbeam-channel = "^0.5" resolve="^0.2" -socket2 = { version = "^0.3", features = ["reuseport"] } +array-init="^2.0" +socket2 = { version = "^0.4", features = ["all"] } combine="^4.0" -hyper="^0.12" mime="^0.3" serde="^1.0" serde_derive="^1.0" @@ -48,16 +51,19 @@ slog-term="^2.4" slog-async="^2.3" slog-scope="^4.0" slog-stdlog="^4.0" -slog-syslog="^0.12" +slog-syslog="^0.13" log="^0.4" toml="^0.5" -ftoa = "^0.1" +dtoa = "^0.4" raft-tokio = { git = "https://github.com/Albibek/raft-tokio" } -rand = "^0.7" -rayon = "^1.0" -bioyino-metric = "^0.3" +rand = "^0.8" +bioyino-metric = "^0.5" [build-dependencies] -capnpc = "^0.12" -vergen = "^3" +capnpc = "^0.14" +vergen = "^5" + +[features] +default = [] +f32 = [] diff --git a/build.rs b/build.rs index 9a9ab7a..82d0d91 100644 --- a/build.rs +++ b/build.rs @@ -1,5 +1,11 @@ -use vergen::{generate_cargo_keys, ConstantsFlags}; +use vergen::{vergen, Config, ShaKind}; fn main() { - generate_cargo_keys(ConstantsFlags::all()).expect("Unable to generate cargo keys!"); + let mut config = Config::default(); + *config.build_mut().timestamp_mut() = true; + *config.git_mut().commit_timestamp_mut() = true; + *config.git_mut().sha_kind_mut() = ShaKind::Short; + + // Generate the instructions + vergen(config).expect("Unable to generate cargo keys!"); } diff --git a/config.toml b/config.toml index f00490e..645ab20 100644 --- a/config.toml +++ b/config.toml @@ -16,10 +16,18 @@ verbosity-console = "warn" # Number of network worker threads in any mode, use 0(not recommended) to use all CPU cores n-threads = 4 -# Number of aggregating and counting threads, use 0(not recommended) to use all CPU cores +# Number of parsing "fast" threads, use 0(not recommended) to use all CPU cores +p-threads = 4 + +# Number of "slow" worker threads, use 0(not recommended) to use all CPU cores w-threads = 4 -# Queue size for single counting thread before task is dropped +# Number of asynchronous threads +# use 0(not recommended) to use all CPU cores +# setting this value to lower than 4 may lead to random lockups and timer delays +a-threads = 4 + +# Queue size for fast and slow threads task-queue-size = 1024 # If server should become leader from it's very start @@ -32,7 +40,7 @@ stats-interval = 10000 # Prefix for sending own stats stats-prefix = "resources.monitoring.bioyino" -# What consensus to use: "consul", "internal" or "none" +# What consensus to use: "internal" or "none" consensus = "none" [carbon] @@ -60,12 +68,17 @@ connect-delay-max = 10000 #note, that 0 means 1 try send-retries = 30 +# The resulting aggregated set can be split into chunks to be sent to i.e. a cluster +# of receiving nodes in parallel in `chunks` parallel TCP connections. +# This may speedup a delivery and make a postprocessing in parallel +chunks = 1 + # Network settings [network] -# Address:port to listen for metrics at +# address and UDP port to listen for statsd metrics at listen = "127.0.0.1:8125" -# Address and port for replication server to listen on +# Address and port for peer protocol TCP server to listen on peer-listen = "127.0.0.1:8136" # Address for peer client to bind to @@ -118,7 +131,7 @@ async-sockets = 4 # List of nodes to replicate metrics to nodes = [] -# Interval to send snapshots to nodes, ms +# Interval(milliseconds) to take snapshots for aggregation and remote replication snapshot-interval = 1000 [metrics] @@ -127,7 +140,7 @@ snapshot-interval = 1000 # 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 all buffers being dropped due to parsing errors. Can be very spammyif you have many incorrect data coming in. # log-parse-errors = false # Size of buffer that parser considers invalid. Used to avoid DoS attacks on parser. @@ -137,33 +150,30 @@ snapshot-interval = 1000 # Size of tags part of a metric (after semicolon character, not including the leading semicolon itself) # max-tags-len = 9000 -# Since tagged metric becomes a totally different metric in many systems the timeseries for such metrics +# In some systems tagged metric becomes a totally different metric. This means the timeseries for such metrics # can be broken. To avoid this for the time the metrics is adopted, this option allows to create a copy of # such tagged metric, but without tags -# Please note that this is effectively the double amount of memory and processing times +# Please note that this is effectively doubles the amount of memory and processing times # create-untagged-copy = false [aggregation] # round timestamp to interval of aggregation # possible values: -# "up" - round to uppor bound of interval +# "up" - round to upper bound of interval # "down" - round to lower bound # "no" - use exact timestamp, i.e. no rounding # round-timestamp = "no" -# the threading mode of aggregation: -# single - in a single thread -# common - in the general thread pool along with parsing ang aggregating metrics -# separate - in a separate thread pool -mode = "single" - -# updates aggregate is usually a debug value than some real one -# also this is the only one used for every type of metric, not only ms -# so it's reasonable to avoid receiving a doubled amount of all metrics -# This option allows to receive only metrics updated too often +# `updates` aggregate is usually a debug value than some real one. +# Usually one want it to see which metrics come too fast taking too much resources. +# Since this is the only one used for every type of metric, not only timers, +# it is reasonable to avoid receiving a doubled amount of all metrics. +# +# This option allows to receive only metrics updated too often. +# In this example is is more than 200 per interval. update-count-threshold = 200 -# a list of aggregates gathered per type of metric +# A list of aggregates gathered per type of metric. # Setting this value will define the exact way, so, i.e. ["mean"] will gather ONLY ONE aggregate # # To add new percentiles an integer value with "percentile-" prefix should be used. @@ -171,7 +181,7 @@ update-count-threshold = 200 # For example, "percentile-9999" will become 0.9999th percentile # # this is the full list of default values for timer -#aggregates.timer = [ "count", "last", "min", "max", "sum", "median", "mean", "updates", "percentile-75", "percentile-95", "percentile-98", "percentile-99", "percentile-999" ] +#aggregates.timer = [ "count", "rate", "last", "min", "max", "sum", "median", "mean", "updates", "percentile-75", "percentile-95", "percentile-98", "percentile-99", "percentile-999" ] # an example for gauge with "updates" aggregate turned off #aggregates.gauge = [ "value" ] @@ -181,6 +191,7 @@ update-count-threshold = 200 # Keys of map in this section are metric types: gauge, counter, diff-counter, timer, set # All sections have same set of parameters. # More examples can be seen in docs/aggregation.md file. + [naming.default] # These setting will be applied for all unspecified naming parts. # # for example if naming.set does not exist or naming.set.tag-values is not specified, @@ -246,20 +257,3 @@ nodes = {} # allow binding raft outgoing connnections to specific IP # default: not specified, so no bind happens # client-bind = "127.0.0.1:8138" - -[consul] -# 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 -agent = "127.0.0.1:8500" - -# TTL of consul session, ms (Consul cannot set it to less than 10s) -session-ttl = 11000 - -# How often to renew Consul session, ms -renew-time = 1000 - -# Key name to lock in Consul -key-name = "service/bioyino/lock" diff --git a/doc/aggregation.md b/doc/aggregation.md index cbc33ee..54ce8d9 100644 --- a/doc/aggregation.md +++ b/doc/aggregation.md @@ -4,11 +4,10 @@ Each metric type has different list of aggregates which can be counted: | Metric type name | statsd name | Aggregates | | --- | --- | --- | -| counter | c | value, updates | -| diff-counter | C | value, updates | -| timer | ms | last, min, max, sum, median, mean, updates, percentile-\* | -| gauge | g | value, updates | -| set | s | count, updates | +| counter | c | value, updates, rate | +| timer | ms | last, min, max, sum, median, mean, updates, rate, percentile-\* | +| gauge | g | value, updates, rate | +| set | s | count, updates, rate | `value` is the aggregate of the resulting value of that type. @@ -19,6 +18,8 @@ By default the percentiles for timer are: 75th, 95th, 98th, 99th and 999th. `updates` is a special aggregate, showing the number of metrics with this name that came in during the aggregation period. It is counted always, even if disabled by configuration and can be additionally filtered by `aggregation.update-count-threshold` parameter. +`rate` aggregate takes sampling rate in account, and tries to restore the original value rate on the metric source rather that "system" rate of receiving metric on the server + By default, bioyino counts all the aggregates. This behaviour can be changed by `aggregation.aggregates` option. If a type is not specified, the default value(meaning all aggregates) will be used. @@ -93,31 +94,3 @@ prefix-overrides = {"updates" = "global.namespace.debug-updates.gauges" } destination = "tag" tag = "agg" ``` - -# Aggregation modes -Depending of aggregation heaviness, there may be different ways to perform it internally. - -The most notable parameter here is the size of and array for a single metric. There may be a lot of i.e. `ms`-typed -metrics, but when only few metrics come during the aggregation period, counting stats for all of them is fast and one -thread will most probably be enough. - -## Single threaded -In this mode the whole array of metrics is counted in a single thread. Usually this is enough for a low-sized batches -i.e. when only a few metrics with the same name are received during aggregation period. Another use case is when -aggregation time is not important and it is ok to wait some time leaving all other cores for processing. - -## Common pool multithreaded -Can be enabled by setting `aggregation-mode` to "common" in `metrics` section. - -The aggregation is distributed between same worker threads, that do parsing and initial metric processing. -Only one thread is started to join the results received from these workers. - -The biggest disadvantage of this approach is that aggregation of some big metric packs can block worker threads, so for -some time they could not process incoming data and therefore UDP drops may increase. - -## Separate pool multithreaded -Can be enabled by setting `aggregation-mode` to "separate" in `metrics` section. - -This mode runs a totally separate thread pool which will try to aggregate all the metrics as fast as possible. -To set the size of the pool `aggregaton-threads` option can be used. It only works in "separate" mode. If set to 0, -the number of threads will be taken automatically by number of cpu cores. diff --git a/doc/threading.md b/doc/threading.md new file mode 100644 index 0000000..2984072 --- /dev/null +++ b/doc/threading.md @@ -0,0 +1,60 @@ +The new threading model introduced since 0.8.0 has some basic goals +1. Avoid dropping UDP packets as much as possible +2. Distribute blocking counting work between sync threads +3. Distribute asynchronous work, like timers or TCP-based networking using a single async runtime + +# Thread pools +The most important thing for processing UDP traffic is to avoid blocking any thread +that does UDP processing. Because in such a case especially under heavy ingress traffic the thread's queue +may be filled almost instantly. This will lead to filling OS queue for UDP socket and OS will start +dropping UDP packets. + +To avoid blocking situations, UDP processing is moved to 2 separate thread pools: + +* UDP networking thread pool. + In sync UDP mode(i.e. when multimessage is set to true) contains blocking synchronous threads responsible for listening UDP sockets and periodically flushing the + collected raw data to parsing threads. + In async UDP mode this is a separate async threadpool dedicated only to reading a number of UDP sockets asynchronously along with one main thread to rule them all. + +* "fast" UDP parsing thread pool. + These threads are also blocking and are responsible for parsing raw data and accumulating such data inrelatively small + local batches. These batches(we call them snapshots) are flushed relatively often(around few seconds) and sent out of the threads for being + aggregated into a big global batch and/or be sent to remote nodes. The flushing is done fast and has a higher priority than parsing. + + To maintain consistent parsing and precise snapshot taking, each thread in this pool has it's own channel and may be reached individually. + + +All other tasks, not sensitive to blocking as UDP, are distributed between 2 other pools, which should be familiar +to anyone who develops multithreading applications these days: + +* "Slow" general purpose thread pool. + As it was said, the "slow" threads are responsible for making all other tasks, that could block for a relatively long + time or require a heavy calculations to be done. The examples of tasks good for this pool are accumulation of metrics + in some global hashmap and parallel aggregate calculation from this pool when time comes. + + Their "slow" threads' main difference from the "fast" ones is the single task queue, which they process alltogether. + So, for example, if one of the threads blocks on lock or a heavy calculation like aggregation of a big timer metric, + other threads are still getting other tasks from the queue, alowing it to empty faster and doing work in parallel. + +* Async thread pool. + This one is basically a standard async runtime(based on `tokio` library) able to process M:N scheduling for N green threads. + This makes it perfect for processing timer-based and I/O-bound processing of everything our daemon needs. All TCP clients, servers, + periodic tasks are starting here, sometimes offloading their heavy work to slow synchronous pool. + +There also some additional threads, existing because of implementation details, which are not important here. + +# Configuration +A number of threads for each thread pool can be set in config by using the parameters below. +Each parameter's default is 4, and may be set to 0 to be detected automatically from number of CPU cores (not recommended). + +Every user has it's own different use case and hardware for their load, which means there is no best out of the box configuration +for number of threads in each pool. Still, obviously, for UDP-heavy traffic n-threads and p-threads should prevail. But for +agent-heavy infrastructures and high cadrinality metrics it's more valuable to increase the a-threads and w-threads numbers leaving UDP low. + +We recommend experimenting on these parameters watching each thread's load in `top` or similar utilities and tuning them accordingly. + +* `n-threads` for networking threads +* `p-threads` for "fast" parsing threads +* `w-threads` for sync "slow" threads +* `a-threads` for async threads. (please note, that despite the async threads' low load, it is not receommended to set their amount to a value + lower than 4 to avoid strange locks or delays in their activity) diff --git a/rustfmt.toml b/rustfmt.toml index 65241ae..4090030 100644 --- a/rustfmt.toml +++ b/rustfmt.toml @@ -1,2 +1,3 @@ max_width=160 ideal_width=80 +edition="2018" diff --git a/src/aggregate.rs b/src/aggregate.rs index fa765e4..2dfd9bc 100644 --- a/src/aggregate.rs +++ b/src/aggregate.rs @@ -1,71 +1,40 @@ use std::collections::HashMap; use std::sync::Arc; -use futures3::channel::mpsc::{self, Sender, UnboundedSender}; -use futures3::{SinkExt, StreamExt, TryFutureExt}; -use tokio2::spawn; +use async_channel::Sender as AsyncSender; -use rayon::{iter::IntoParallelIterator, iter::ParallelIterator, ThreadPoolBuilder}; -use serde_derive::{Deserialize, Serialize}; -use slog::{info, warn, Logger}; +use slog::{warn, Logger}; use bioyino_metric::{ aggregate::{Aggregate, AggregateCalculator}, metric::MetricTypeName, name::{MetricName, NamingOptions}, - Metric, + FromF64, }; +use crate::cache::RotatedCacheShard; use crate::config::{all_aggregates, Aggregation, ConfigError, Naming, RoundTimestamp}; -use crate::task::Task; use crate::{s, Float}; -#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)] -#[serde(rename_all = "kebab-case", deny_unknown_fields)] -pub enum AggregationMode { - Single, - #[serde(alias = "common-pool", alias = "common_pool")] - Common, - #[serde(alias = "separate-pool", alias = "separate_pool")] - Separate, -} - #[derive(Debug, Clone)] pub struct AggregationOptions { pub round_timestamp: RoundTimestamp, - pub mode: AggregationMode, - pub multi_threads: usize, pub update_count_threshold: Float, pub aggregates: HashMap>>, pub namings: HashMap<(MetricTypeName, Aggregate), NamingOptions>, } impl AggregationOptions { - pub(crate) fn from_config(config: Aggregation, naming: HashMap, log: Logger) -> Result, ConfigError> { + pub(crate) fn from_config(config: Aggregation, interval: Float, naming: HashMap, log: Logger) -> Result, ConfigError> { let Aggregation { round_timestamp, - mode, - threads, update_count_threshold, aggregates, } = config; - let multi_threads = match threads { - Some(value) if mode == AggregationMode::Separate => value, - Some(_) => { - info!(log, "aggregation.threads parameter only works in \"separate\" mode and will be ignored"); - 0 - } - None if mode == AggregationMode::Separate => 0, - _ => 0, - }; - let mut opts = Self { - // round_timestamp, - mode, - multi_threads, - update_count_threshold: Float::from(update_count_threshold), + update_count_threshold: Float::from_f64(update_count_threshold as f64), aggregates: HashMap::new(), namings: HashMap::new(), }; @@ -78,7 +47,7 @@ impl AggregationOptions { // First task: deal with aggregates to be counted // consider 2 cases: // 1. aggregates is not specified at all, then the default value of all_aggregates() has - // been used already + // been used already // 2. aggregates is defined partially per type, then we take only replacements specified // for type and take others from defaults wich is in all_aggregates() @@ -127,257 +96,114 @@ impl AggregationOptions { // aggregate we use an empty value as a 'signal' noptions.tag = b""[..].into(); } - opts.namings.insert((ty.clone(), agg.clone()), noptions); + if let Aggregate::Rate(None) = agg { + opts.namings.insert((ty.clone(), Aggregate::Rate(Some(interval))), noptions); + } else { + opts.namings.insert((ty.clone(), agg.clone()), noptions); + } } } - Ok(Arc::new(opts)) - } -} - -pub struct Aggregator { - is_leader: bool, - options: Arc, - chans: Vec>, - // a channel where we receive rotated metrics from tasks - tx: UnboundedSender<(MetricName, MetricTypeName, Aggregate, Float)>, - log: Logger, -} - -impl Aggregator { - pub fn new( - is_leader: bool, - options: Arc, - chans: Vec>, - tx: UnboundedSender<(MetricName, MetricTypeName, Aggregate, Float)>, - log: Logger, - ) -> Self { - Self { - is_leader, - options, - chans, - tx, - log, - } - } - - pub async fn run(self) { - let Self { - is_leader, - options, - chans, - tx, - log, - } = self; - let (task_tx, mut task_rx) = mpsc::unbounded(); - - let response_chan = if is_leader { - Some(task_tx) - } else { - info!(log, "not leader, clearing metrics"); - drop(task_tx); - None - }; - - let ext_log = log.clone(); - - // regardless of leader state send rotate tasks with or without response channel - // we don't need to await send, because we are waiting on task_rx eventually - let mut handles = Vec::new(); - for chan in &chans { - let mut chan = chan.clone(); - let rchan = response_chan.clone(); - let handle = spawn(async move { chan.send(Task::Rotate(rchan)).map_err(|_| s!(queue_errors)).await }); - handles.push(handle); - } - drop(response_chan); - // wait for senders to do their job - futures3::future::join_all(handles).await; - - // when we are not leader the aggregator job is done here: send_tasks will delete metrics - if !is_leader { - return; - } - // from now we consider us being a leader - - let mut cache: HashMap>> = HashMap::new(); - while let Some(metrics) = task_rx.next().await { - // #[allow(clippy::map_entry)] // clippy offers us the entry API here, but it doesn't work without additional cloning - for (name, metric) in metrics { - let entry = cache.entry(name).or_default(); - entry.push(metric); + // Now we can set the correct rate value + for aggs in opts.aggregates.values_mut() { + for agg in aggs { + if let Aggregate::Rate(ref mut r) = agg { + *r = Some(interval) + } } } - info!(log, "leader aggregating metrics"; "amount"=>format!("{}", cache.len())); - - match options.mode { - AggregationMode::Single => { - cache - .into_iter() - .map(move |(name, metrics)| { - let task_data = AggregationData { - name, - metrics, - options: options.clone(), - response: tx.clone(), - }; - aggregate_task(task_data); - }) - .last(); - } - AggregationMode::Common => { - cache - .into_iter() - .enumerate() - .map(move |(num, (name, metrics))| { - let task_data = AggregationData { - name, - metrics, - options: options.clone(), - response: tx.clone(), - }; - let mut chan = chans[num % chans.len()].clone(); - spawn(async move { chan.send(Task::Aggregate(task_data)).await }); - }) - .last(); - } - AggregationMode::Separate => { - let pool = ThreadPoolBuilder::new() - .thread_name(|i| format!("bioyino_agg{}", i)) - .num_threads(options.multi_threads) - .build() - .unwrap(); - pool.install(|| { - cache.into_par_iter().for_each(move |(name, metrics)| { - let task_data = AggregationData { - name, - metrics, - options: options.clone(), - response: tx.clone(), - }; - aggregate_task(task_data); - }); - }); - } - }; - - info!(ext_log, "done aggregating"); + Ok(Arc::new(opts)) } } +pub type Aggregated = (MetricName, MetricTypeName, Aggregate, Float); + #[derive(Debug)] pub struct AggregationData { - pub name: MetricName, - pub metrics: Vec>, + pub metrics: RotatedCacheShard, pub options: Arc, - pub response: UnboundedSender<(MetricName, MetricTypeName, Aggregate, Float)>, + pub response: AsyncSender>, } pub fn aggregate_task(data: AggregationData) { - let AggregationData { - name, - mut metrics, - options, - response, - } = data; - - // accumulate vector of metrics into a single metric first - let first = if let Some(metric) = metrics.pop() { - metric - } else { - // empty metric case is not possible actually - s!(agg_errors); - return; - }; - - let mut metric = metrics.into_iter().fold(first, |mut acc, next| { - acc.accumulate(next).unwrap_or_else(|_| { + let AggregationData { metrics, options, response } = data; + + let mut result = Vec::new(); + for (name, metric) in metrics.into_iter() { + // remove mutex + let mut metric = metric.into_inner().unwrap(); + + let typename = MetricTypeName::from_metric(&metric); + // find all needed aggregates by type name + let aggregates = if let Some(agg) = options.aggregates.get(&typename) { + agg + } else { s!(agg_errors); - }); - acc - }); - - let mode = options.mode; - let typename = MetricTypeName::from_metric(&metric); - let aggregates = if let Some(agg) = options.aggregates.get(&typename) { - agg - } else { - s!(agg_errors); - return; - }; - - // take all required aggregates - let calculator = AggregateCalculator::new(&mut metric, aggregates); - calculator - // count all of them that are countable (filtering None) - .filter_map(|result| result) - // set corresponding name - .filter_map(|(idx, value)| { - let aggregate = &aggregates[idx]; - match aggregate { - Aggregate::UpdateCount => { - if value < options.update_count_threshold { - // skip aggregates below update counter threshold - None - } else { - Some((name.clone(), typename, *aggregate, value)) + return; + }; + + // take all required aggregates + let calculator = AggregateCalculator::new(&mut metric, aggregates); + + calculator + // count all of them that are countable, i.e. filtering None + .filter_map(|result| result) + // set corresponding name + .filter_map(|(idx, value)| { + let aggregate = &aggregates[idx]; + match aggregate { + Aggregate::UpdateCount => { + if value < options.update_count_threshold { + // skip aggregates below update counter threshold + None + } else { + Some((name.clone(), typename, *aggregate, value)) + } } + _ => Some((name.clone(), typename, *aggregate, value)), } - _ => Some((name.clone(), typename, *aggregate, value)), - } - }) - .map(|data| { - let mut response = response.clone(); - let respond = async move { response.send(data).await }; - - match mode { - AggregationMode::Separate => { - // In the separate mode there is no runtime, so we just run future - // synchronously - futures3::executor::block_on(respond).expect("responding thread: error sending aggregated metrics back"); - } - _ => { - spawn(respond); - } - } - }) - .last(); + }) + .map(|data| result.push(data)) + .last(); + } + + futures::executor::block_on(response.send(result)).unwrap_or(()); } #[cfg(test)] mod tests { use super::*; + use std::convert::TryFrom; use std::sync::atomic::Ordering; use std::time::Duration; use crate::util::prepare_log; - use futures3::channel::mpsc; - use tokio2::runtime::Builder; - use tokio2::time::delay_for; + use futures::channel::oneshot; + use tokio::runtime::Builder; + use tokio::time::sleep; - use bioyino_metric::{name::MetricName, Metric, MetricType}; + use bioyino_metric::aggregate::Aggregate; + use bioyino_metric::{name::MetricName, Metric, MetricTypeName, MetricValue}; use crate::config; + use crate::slow_task::{start_slow_threads, SlowTask}; #[test] - fn parallel_aggregation_ms_rayon() { - let log = prepare_log("test_parallel_aggregation_ms"); - let mut chans = Vec::new(); - let (tx, mut rx) = mpsc::channel(5); - chans.push(tx); + fn aggregation() { + let log = prepare_log("test_parallel_aggregation"); - let mut runtime = Builder::new() + let mut config = config::System::default(); + let slow_chan = start_slow_threads(log.clone(), &config).expect("starting slow threads"); + let runtime = Builder::new_multi_thread() .thread_name("bio_agg_test") - .basic_scheduler() .enable_all() .build() .expect("creating runtime for test"); - let mut config = config::Aggregation::default(); - + let config = &mut config.aggregation; let timer = MetricTypeName::Timer; // add 0.8th percentile to timer aggregates config @@ -389,38 +215,39 @@ mod tests { .push(Aggregate::Percentile(0.8, 80)); //"percentile-80".into()); - /*config.postfix_replacements.insert("percentile-80".into(), "percentile80".into()); - config.postfix_replacements.insert("min".into(), "lower".into()); - */ + //*config.postfix_replacements.insert("percentile-80".into(), "percentile80".into()); + //config.postfix_replacements.insert("min".into(), "lower".into()); // TODO: check tag replacements //config.tag_replacements.insert("percentile-80".into(), "p80".into()); config.update_count_threshold = 1; - config.mode = AggregationMode::Separate; - config.threads = Some(2); let naming = config::default_namings(); //;.get(&timer).unwrap().clone(); - let options = AggregationOptions::from_config(config, naming, log.clone()).unwrap(); - - let (backend_tx, backend_rx) = mpsc::unbounded(); - - let aggregator = Aggregator::new(true, options, chans, backend_tx, log.clone()); + let agg_opts = AggregationOptions::from_config(config.clone(), 30., naming, log.clone()).unwrap(); let counter = std::sync::atomic::AtomicUsize::new(0); + + // Create some cache data, send it to be joined into long cache let mut cache = HashMap::new(); - for i in 0..10 { - let mut metric = Metric::new(0f64, MetricType::Timer(Vec::new()), None, None).unwrap(); - for j in 1..100 { - let new_metric = Metric::new(j.into(), MetricType::Timer(Vec::new()), None, None).unwrap(); + for i in 0..10u16 { // u16 can be converted to f32 + let mut metric = Metric::new(MetricValue::Timer(vec![0.]), None, 1.); + for j in 1..100u16 { + let new_metric = Metric::new(MetricValue::Timer(vec![j.into()]), None, 1.); metric.accumulate(new_metric).unwrap(); } let counter = counter.fetch_add(1, Ordering::Relaxed); cache.insert(MetricName::from_raw_parts(format!("some.test.metric.{}.{}", i, counter).into(), None), metric); } + slow_chan.send(SlowTask::Join(Arc::new(cache.clone()))).unwrap(); + + // right after cache send a rotation task + let (tx, rx) = oneshot::channel(); + slow_chan.send(SlowTask::Rotate(Some(tx))).unwrap(); + // the result of timer aggregations we want is each key mapped to name - let required_aggregates: Vec<(MetricName, Aggregate)> = cache + let required_aggregates: Vec<(MetricName, Aggregate)> = cache .keys() .map(|key| { config::all_aggregates() @@ -428,56 +255,85 @@ mod tests { .unwrap() .clone() .into_iter() - .map(|agg| Aggregate::::try_from(agg).unwrap()) + .map(|agg| Aggregate::::try_from(agg).unwrap()) + .map(|agg| if agg == Aggregate::Rate(None) { Aggregate::Rate(Some(30.)) } else { agg }) .chain(Some(Aggregate::Percentile(0.8, 80))) .map(move |agg| (key.clone(), agg)) }) - .flatten() + .flatten() .collect(); - let sent_cache = cache.clone(); - let rotate = async move { - while let Some(task) = rx.next().await { - if let Task::Rotate(Some(mut response)) = task { - let sent_cache = sent_cache.clone(); - // Emulate rotation in task - spawn(async move { response.send(sent_cache).await }); - } - } - }; + let required_len = required_aggregates.len(); - runtime.spawn(rotate); - runtime.spawn(aggregator.run()); + // When things happen inside threads, panics are catched by runtime. + // So test did not fail correctly event on asserts inside + // To avoid this, we make a fail counter which we check afterwards + + use std::sync::Mutex; + let fails = Arc::new(Mutex::new(0usize)); + let inner_fails = fails.clone(); + let receiver = async move { + let rotated = rx.await.unwrap(); + + let (agg_tx, mut agg_rx) = async_channel::unbounded(); + + // then send each shard to be aggregated separately + for shard in rotated { + let agg_data = AggregationData { + metrics: shard, + options: agg_opts.clone(), + response: agg_tx.clone(), + }; + slow_chan.send(SlowTask::Aggregate(agg_data)).unwrap(); + } + + use futures::StreamExt; + // wait for answers nd collect data for further sending + let mut aggregated = Vec::new(); + while let Some(blob) = agg_rx.next().await { + // we want to send data in chunks but if shard has been sent, + // there is no point of holding the whole shitload of metrics + // all together, so we wrap blobs in an Arc, so it is dropped + // when all chunks of it are sent + aggregated.push(blob); + } - let required_len = required_aggregates.len(); - let receive = async move { - let result = backend_rx.collect::>().await; // ensure aggregated data has ONLY required aggregates and no other shit - for (n, _) in cache { - // each metric should have all aggregates - for (rname, ragg) in &required_aggregates { - assert!( - result.iter().position(|(name, _, ag, _)| (name, ag) == (&rname, &ragg)).is_some(), - "could not find {:?}", - ragg - ); - //dbg!(result); + //for (n, _) in aggregated.iter().flatten() { + // each metric should have all aggregates + for (rname, ragg) in &required_aggregates { + if !aggregated + .iter() + .flatten() + .position(|(name, _, ag, _)| { + if let (&Aggregate::Rate(_), &Aggregate::Rate(_)) = (ag, &ragg) { + // compare rate aggregate without internal value + &name == &rname + } else { + (name, ag) == (&rname, &ragg) + } + }) + .is_some() + { + let mut fails = inner_fails.lock().unwrap(); + *fails += 1; + println!("could not find {:?}", ragg); } // length should match the length of aggregates - - assert_eq!( - result.len(), - required_len, - "found other than required aggregates for {}", - String::from_utf8_lossy(&n.name), - ); + let result_len = aggregated.iter().fold(0, |acc, elem| { acc + elem.len() } ); + if result_len != required_len { + let mut fails = inner_fails.lock().unwrap(); + *fails += 1; + dbg!("found other than required aggregates: {} {}",result_len, required_len ); + } } }; + runtime.spawn(receiver); - runtime.spawn(receive); - - let test_delay = async { delay_for(Duration::from_secs(2)).await }; + let test_delay = async { sleep(Duration::from_secs(2)).await }; runtime.block_on(test_delay); + drop(runtime); + assert_eq!(Arc::try_unwrap(fails).unwrap().into_inner().unwrap(), 0); + } } -} diff --git a/src/async_udp.rs b/src/async_udp.rs new file mode 100644 index 0000000..b111001 --- /dev/null +++ b/src/async_udp.rs @@ -0,0 +1,205 @@ +use std::{collections::hash_map::DefaultHasher, sync::atomic::AtomicBool}; +use std::collections::HashMap; +use std::hash::{Hash, Hasher}; +use std::sync::atomic::Ordering; +use std::sync::Arc; +use std::io; + +use bytes::{BufMut, BytesMut}; +use crossbeam_channel::Sender; +use futures::future::{pending, select}; +use tokio::{net::UdpSocket, sync::Notify}; +use tokio::runtime::Builder; +use socket2::{Domain, Protocol, Socket, Type}; + +use slog::{ error, o, warn, Logger}; + +use crate::config::System; +use crate::s; +use crate::stats::STATS; +use crate::fast_task::FastTask; +use crate::errors::GeneralError; + +pub(crate) fn start_async_udp( + log: Logger, chans: &[Sender], config: Arc +) -> Arc { + + + let flush_notify_ret = Arc::new(Notify::new()); + + let chans = chans.to_vec(); + + let flush_notify = flush_notify_ret.clone(); + // to provide a multithreading listerning of multiple sockets, we start a separate + // tokio runtime with a separate threadpool of `n_threads` + std::thread::Builder::new() + .name("bioyino_udp".into()) + .spawn(move || { + let runtime = Builder::new_multi_thread() + .thread_name("bioyino_audp") + .worker_threads(config.n_threads) + .enable_all() + .build() + .expect("creating runtime for async UDP threads"); + + runtime.spawn(async move { + // Create a pool of listener sockets + for _ in 0..config.network.async_sockets { + let socket = Socket::new(Domain::IPV4, Type::DGRAM, Some(Protocol::UDP)).expect("creating UDP socket"); + socket.set_reuse_address(true).expect("reusing address"); + socket.set_reuse_port(true).expect("reusing port"); + socket.set_nonblocking(true).expect("setting nonblocking"); + socket.bind(&config.network.listen.into()).expect("binding"); + + let socket = UdpSocket::from_std(socket.into()).expect("adding socket to event loop"); + let socket = Arc::new(socket); + + // for every single socket we want a number of green threads (specified by `greens`) listening for it + for _ in 0..config.network.greens { + + let config = config.clone(); + let log = log.clone(); + let flush_notify = flush_notify.clone(); + let chans = chans.clone(); + let socket = socket.clone(); + let worker = AsyncUdpWorker::new( + log, + chans, + config, + socket, + flush_notify, + ); + + tokio::spawn(worker.run()); + } + } + }); + + runtime.block_on(pending::<()>()) + }) + .expect("starting thread for async UDP threadpool"); + + flush_notify_ret +} + +struct AsyncUdpWorker { + log: Logger, + chans: Vec>, + config: Arc, + socket: Arc, + flush: Arc, +} + +impl AsyncUdpWorker { + + fn new( + log: Logger, + chans: Vec>, + config: Arc, + socket: Arc, + flush: Arc, + ) -> Self { + + Self { + log, + chans, + config, + socket, + flush, + } + } + + async fn run( + self, + ) -> Result<(), GeneralError> { + + let Self { + log, + mut chans, + config, + socket, + flush, + } = self; + let mut bufmap = HashMap::new(); + let bufsize = config.network.bufsize; + let mut readbuf = Vec::with_capacity(bufsize); + readbuf.resize(bufsize, 0); + let log = log.new(o!("source"=>"async_udp_worker")); + + let do_flush = AtomicBool::new(false); + let mut next_chan = config.p_threads; + let mut recv_counter = 0usize; + + loop { + + let f1 = async { + socket.readable().await + }; + + let f2 = async { + flush.notified().await; + do_flush.swap(true, Ordering::Relaxed); + }; + + futures::pin_mut!(f1); + futures::pin_mut!(f2); + + select(f1, f2).await; + + loop { + // read everything from socket while it's readable + let (size, addr) = match socket.try_recv_from(&mut readbuf) { + Ok((0, _)) => { + // size = 0 means EOF + warn!(log, "exiting on EOF"); + return Ok(()) + } + Ok((size, addr)) => (size, addr), + Err(e) if e.kind() == io::ErrorKind::WouldBlock => { + //continue + break + //return Ok(()) + } + Err(e) => { + error!(log, "error reading UDP socket {:?}", e); + return Err(GeneralError::Io(e)) + } + }; + + // we only get here in case of success + STATS.ingress.fetch_add(size, Ordering::Relaxed); + + let buf = bufmap + .entry(addr) + .or_insert_with(|| BytesMut::with_capacity(config.network.buffer_flush_length)); + recv_counter += size; + // check we can fit the buffer + if buf.remaining_mut() < config.network.bufsize { + buf.reserve(size + 1) + } + buf.put_slice(&readbuf[..size]); + + let flush = do_flush.swap(false, Ordering::Relaxed); + if recv_counter >= config.network.buffer_flush_length || flush { + for (addr, buf) in bufmap.drain() { + let mut hasher = DefaultHasher::new(); + addr.hash(&mut hasher); + let ahash = hasher.finish(); + let chan = if config.metrics.consistent_parsing { + let chlen = chans.len(); + &mut chans[ahash as usize % chlen] + } else { + if next_chan >= chans.len() { + next_chan = 0; + } + let chan = &mut chans[next_chan]; + next_chan += 1; + chan + }; + chan.send(FastTask::Parse(ahash, buf)).unwrap_or_else(|_| s!(drops)); + } + } + } + } + } +} diff --git a/src/cache.rs b/src/cache.rs new file mode 100644 index 0000000..b52fcb7 --- /dev/null +++ b/src/cache.rs @@ -0,0 +1,70 @@ +use std::collections::hash_map::DefaultHasher; +use std::collections::HashMap; +use std::hash::Hasher; +use std::ops::DerefMut; +use std::sync::{Arc, Mutex, RwLock}; + +use bioyino_metric::{name::MetricName, Metric}; + +use crate::errors::GeneralError; +use crate::Float; + +const SHARDS: usize = 64; + +pub type RotatedCacheShard = HashMap>>; +pub type RotatedCache = Vec; + +#[derive(Clone)] +pub struct SharedCache { + shards: Arc<[RwLock>>>; SHARDS]>, +} + +impl SharedCache { + pub fn new() -> Self { + Self { + shards: Arc::new(array_init::array_init(|_| RwLock::new(HashMap::new()))), + } + } + + pub fn accumulate(&self, name: MetricName, new: Metric) -> Result<(), GeneralError> { + let mut hasher = DefaultHasher::new(); + hasher.write(name.name_with_tags()); + let index = hasher.finish() as usize % SHARDS; + let read = self.shards[index].read().unwrap(); + match read.get(&name) { + Some(metric) => { + let mut metric = metric.lock().unwrap(); + metric.accumulate(new)?; + } + None => { + drop(read); + let mut write = self.shards[index].write().unwrap(); + write.insert(name, Mutex::new(new)); + } + } + Ok(()) + } + + pub fn rotate(&self, collect: bool) -> RotatedCache { + // avoid allocation if it is not required (Vec::new does not allocate) + let mut rotated = if collect { Vec::with_capacity(SHARDS) } else { Vec::new() }; + for i in 0..SHARDS { + let mut write = self.shards[i].write().unwrap(); + let mut m = HashMap::with_capacity(write.len() / 2); + std::mem::swap(write.deref_mut(), &mut m); + if collect { + rotated.push(m); + } + } + rotated + } + + #[cfg(test)] + pub fn get(&self, name: &MetricName) -> Option> { + let mut hasher = DefaultHasher::new(); + hasher.write(name.name_with_tags()); + let index = hasher.finish() as usize % SHARDS; + let read = self.shards[index].read().unwrap(); + read.get(&name).map(|m| m.lock().unwrap().clone()) + } +} diff --git a/src/carbon.rs b/src/carbon.rs index d563379..582c200 100644 --- a/src/carbon.rs +++ b/src/carbon.rs @@ -4,27 +4,26 @@ use std::sync::atomic::Ordering; use std::sync::Arc; use std::time::{self, Duration, SystemTime}; -use bytes::{buf::BufMutExt, Bytes, BytesMut}; -use futures3::channel::mpsc::{self, Sender}; -use futures3::sink::SinkExt; -use futures3::stream::StreamExt; -use futures3::TryFutureExt; +use bytes::{BufMut, Bytes, BytesMut}; +use crossbeam_channel::Sender; +use futures::channel::oneshot; +use futures::sink::SinkExt; +use futures::stream::StreamExt; use log::warn; use slog::{debug, error, info, o, Logger}; -use tokio2::net::TcpStream; -use tokio2::runtime::Builder; -use tokio2::spawn; -use tokio2::time::{interval_at, Instant}; +use tokio::net::TcpStream; +use tokio::spawn; +use tokio::time::{interval_at, Instant}; use tokio_util::codec::{Decoder, Encoder}; -use bioyino_metric::{aggregate::Aggregate, metric::MetricTypeName, name::MetricName}; +use bioyino_metric::{aggregate::Aggregate, metric::MetricTypeName, name::MetricName, FromF64}; -use crate::aggregate::{AggregationOptions, Aggregator}; -use crate::config::{Aggregation, Carbon, ConfigError, Naming, RoundTimestamp}; +use crate::aggregate::{Aggregated, AggregationData, AggregationOptions}; +use crate::config::{Aggregation, Carbon, Naming, RoundTimestamp}; use crate::errors::GeneralError; -use crate::task::Task; +use crate::slow_task::SlowTask; use crate::util::{bound_stream, resolve_with_port, retry_with_backoff, Backoff}; use crate::{s, Float, IS_LEADER}; @@ -33,117 +32,199 @@ pub async fn carbon_timer( mut options: Carbon, aggregation: Aggregation, naming: HashMap, - chans: Vec>, + chan: Sender, ) -> Result<(), GeneralError> { - let chans = chans.clone(); - let dur = Duration::from_millis(options.interval); let mut carbon_timer = interval_at(Instant::now() + dur, dur); if options.chunks == 0 { options.chunks = 1 } - let agg_opts = AggregationOptions::from_config(aggregation, naming, log.clone())?; - let log = log.new(o!("thread"=>"carbon")); + + let interval = Float::from_f64(options.interval as f64 / 1000f64); + let agg_opts = AggregationOptions::from_config(aggregation, interval, naming, log.clone())?; + let log = log.new(o!("task"=>"carbon")); loop { carbon_timer.tick().await; - let worker = CarbonWorker::new(log.clone(), agg_opts.clone(), options.clone(), chans.clone())?; - std::thread::Builder::new() - .name("bioyino_carbon".into()) - .spawn(move || worker.run()) - .map_err(|e| ConfigError::Io("spawning carbon thread".into(), e))?; + let is_leader = IS_LEADER.load(Ordering::SeqCst); + if is_leader { + info!(log, "leader ready to aggregate metrics"); + + // create rotation task, send it and wait for an answer + let (tx, rx) = oneshot::channel(); + chan.send(SlowTask::Rotate(Some(tx))).unwrap_or_else(|_| { + error!(log, "error sending rotation task"); + }); + let rotated = if let Ok(r) = rx.await { + r + } else { + error!(log, "error receiving metric cache, worker thread may have panicked"); + continue; + }; + + let (agg_tx, mut agg_rx) = async_channel::unbounded(); + + // then send each shard to be aggregated separately + for shard in rotated { + let agg_data = AggregationData { + metrics: shard, + options: agg_opts.clone(), + response: agg_tx.clone(), + }; + chan.send(SlowTask::Aggregate(agg_data)).unwrap_or_else(|_| { + error!(log, "error sending aggregation task"); + }); + } + drop(agg_tx); + + // wait for answers nd collect data for further sending + let mut aggregated = Vec::new(); + while let Some(blob) = agg_rx.next().await { + // we want to send data in chunks but if shard has been sent, + // there is no point of holding the whole shitload of metrics + // all together, so we wrap blobs in an Arc, so it is dropped + // when all chunks of it are sent + aggregated.push(blob); + } + + info!(log, "done aggregating"); + + let sender = CarbonSender::new(log.clone(), aggregated, agg_opts.clone(), options.clone())?; + spawn(sender.run()); + } else { + info!(log, "not leader, clearing metrics"); + chan.send(SlowTask::Rotate(None)).unwrap_or_else(|_| { + error!(log, "error sending rotation task"); + }); + } + } +} + +fn rechunk(mut input: Vec>, chunks: usize) -> Vec>> { + if input.len() == 0 { + return Vec::new(); + } + + if chunks == 1 { + let mut res = Vec::new(); + res.push(input); + return res; + } + + let mut len = 0; + for shard in &input { + len += shard.len(); + } + + let chunk_size = if len > chunks { len / chunks } else { 1 }; + + let mut recipient = Vec::with_capacity(chunks); + let mut len_required = chunk_size; + + // create first vector of chunks, we will push vectors here + recipient.push(Vec::new()); + + // create first donor + let mut donor = input.pop().unwrap(); + loop { + let last = recipient.len() - 1; + if donor.len() >= len_required { + // donor has more len than required + let new = donor.split_off(donor.len() - len_required); // remove that amount from donor + recipient[last].push(new); // give it as a part of current chunk + len_required = chunk_size; // reset the required len + recipient.push(Vec::new()); // recipient is satisfied + } else { + // donor has not enough len + // give what is left to recipient + len_required -= donor.len(); + recipient[last].push(donor); + // get a new donor or end + if let Some(new) = input.pop() { + donor = new + } else { + break; + } + } + } + + if recipient.len() > 1 && recipient[recipient.len() - 1].is_empty() { + recipient.pop().unwrap(); } + recipient } -pub struct CarbonWorker { +pub struct CarbonSender { ts: u64, log: Logger, + metrics: Vec>, agg_opts: Arc, backend_opts: Carbon, - chans: Vec>, } -impl CarbonWorker { - pub fn new(log: Logger, agg_opts: Arc, backend_opts: Carbon, chans: Vec>) -> Result { +impl CarbonSender { + pub fn new(log: Logger, metrics: Vec>, agg_opts: Arc, backend_opts: Carbon) -> Result { let ts = SystemTime::now().duration_since(time::UNIX_EPOCH).map_err(GeneralError::Time)?.as_secs(); + + let log = log.new(o!("ts"=>ts.clone())); Ok(Self { ts, log, + metrics, agg_opts, backend_opts, - chans, }) } - pub fn run(self) { + pub async fn run(self) { let Self { ts, log, + metrics, agg_opts, backend_opts, - chans, } = self; - debug!(log, "carbon thread running"); - let mut runtime = Builder::new() - .basic_scheduler() - .enable_all() - .build() - .expect("creating runtime for carbon backend thread"); + let total_len = metrics.iter().fold(0, |acc, elem| acc + elem.len()); + if total_len == 0 { + info!(log, "metric set is empty, not sending"); + return; + } - let is_leader = IS_LEADER.load(Ordering::SeqCst); + // at this point we have metrics as Vec + // but we want to send them in other chunk sizes without iterating + // over those, probably very big, vectors, so we change the chunks + // using maybe a few allocations but avoiding lots of iterations - let options = agg_opts.clone(); + let chunks = rechunk(metrics, backend_opts.chunks); - if is_leader { - info!(log, "leader ready to aggregate metrics"); - let (backend_tx, backend_rx) = mpsc::unbounded(); - let aggregator = Aggregator::new(is_leader, options.clone(), chans, backend_tx, log.clone()); - - runtime.spawn(aggregator.run()); - - let carbon_sender = async { - let metrics = backend_rx.collect::>().await; - let carbon_log = log.clone(); - let carbon = backend_opts.clone(); - let chunk_size = if metrics.len() > carbon.chunks { metrics.len() / carbon.chunks } else { 1 }; - let mut senders = Vec::new(); - for (nth, chunk) in metrics.chunks(chunk_size).enumerate() { - let retry_log = carbon_log.clone(); - let elog = carbon_log.clone(); - let options = CarbonClientOptions { - addr: backend_opts.address.clone(), - bind: backend_opts.bind_address, - interval: carbon.interval, - options: options.clone(), - }; - - let backoff = Backoff { - delay: backend_opts.connect_delay, - delay_mul: backend_opts.connect_delay_multiplier, - delay_max: backend_opts.connect_delay_max, - retries: backend_opts.send_retries, - }; - let options = options.clone(); - let chunk = Arc::new(chunk.to_vec()); - let retrier = retry_with_backoff(backoff, move || { - let client = CarbonBackend::new(options.clone(), ts, chunk.clone(), retry_log.clone()); - client.run() - }); - let handle = spawn(async move { - retrier.await.unwrap_or_else(move |e| { - error!(elog.clone(), "failed to send chunk to graphite"; "chunk" => format!("{}", nth), "error"=>format!("{:?}",e)); - }) - }); - senders.push(handle); - } - futures3::future::join_all(senders).await + for (nth, chunk) in chunks.into_iter().enumerate() { + let options = CarbonClientOptions { + addr: backend_opts.address.clone(), + bind: backend_opts.bind_address, + interval: backend_opts.interval, + agg: agg_opts.clone(), }; - runtime.block_on(carbon_sender); - } else { - info!(log, "not leader, removing metrics"); - let (backend_tx, _) = mpsc::unbounded(); - let aggregator = Aggregator::new(is_leader, options, chans, backend_tx, log.clone()); - runtime.block_on(aggregator.run()) + + let backoff = Backoff { + delay: backend_opts.connect_delay, + delay_mul: backend_opts.connect_delay_multiplier, + delay_max: backend_opts.connect_delay_max, + retries: backend_opts.send_retries, + }; + + let retry_log = log.clone(); + let chunk = Arc::new(chunk); + let retrier = retry_with_backoff(backoff, move || { + let client = CarbonBackend::new(options.clone(), ts, chunk.clone(), retry_log.clone()); + client.run() + }); + + let elog = log.clone(); + spawn(async move { + retrier.await.unwrap_or_else(move |e| { + error!(elog, "failed to send chunk to graphite"; "chunk" => format!("{}", nth), "error"=>format!("{:?}",e)); + }) + }); } } } @@ -153,29 +234,29 @@ pub struct CarbonClientOptions { pub addr: String, pub bind: Option, pub interval: u64, // the timer is external, this is only used to calculate timestamp rounding - pub options: Arc, + pub agg: Arc, } #[derive(Clone)] pub struct CarbonBackend { options: CarbonClientOptions, ts: Bytes, - metrics: Arc, Float)>>, + metrics: Arc>>, log: Logger, } impl CarbonBackend { - pub(crate) fn new(options: CarbonClientOptions, ts: u64, metrics: Arc, Float)>>, log: Logger) -> Self { + pub(crate) fn new(options: CarbonClientOptions, ts: u64, metrics: Arc>>, log: Logger) -> Self { // we have interval in milliseconds // but we need a timestamp to be rounded to seconds let interval = options.interval / 1000; - let ts = match options.options.round_timestamp { + let ts = match options.agg.round_timestamp { RoundTimestamp::Down => ts - (ts % interval), RoundTimestamp::No => ts, RoundTimestamp::Up => ts - (ts % interval) + interval, }; let ts = ts.to_string(); - let log = log.new(o!("module"=>"carbon backend", "ts"=>ts.clone())); + let log = log.new(o!("module"=>"carbon backend")); Self { options, @@ -191,18 +272,25 @@ impl CarbonBackend { let conn = match options.bind { Some(bind_addr) => match bound_stream(&bind_addr) { - Ok(std_stream) => TcpStream::connect_std(std_stream, &addr).map_err(|e| GeneralError::Io(e)).await?, + Ok(std_stream) => TcpStream::from_std(std_stream).map_err(|e| GeneralError::Io(e))?, Err(e) => { return Err(GeneralError::Io(e)); } }, - None => TcpStream::connect(&options.addr).await?, + None => TcpStream::connect(&addr).await?, }; - info!(log, "sending metrics chunk"; "metrics"=>format!("{}", metrics.len())); - let mut writer = CarbonCodec::new(ts.clone(), options.options.clone()).framed(conn); + let chunk_len = metrics.iter().fold(0, |acc, elem| acc + elem.len()); - for m in metrics.iter().cloned() { + if chunk_len == 0 { + debug!(log, "empty chunk, skipping"); + return Ok(()); + } + + info!(log, "sending metrics chunk"; "metrics"=>format!("{}", chunk_len)); + let mut writer = CarbonCodec::new(ts.clone(), options.agg.clone()).framed(conn); + + for m in metrics.iter().flatten() { writer.send(m).await? } @@ -223,13 +311,13 @@ impl CarbonCodec { } // item: Metric name, type name, aggregate and counted value -impl Encoder<(MetricName, MetricTypeName, Aggregate, Float)> for CarbonCodec { +impl<'a> Encoder<&'a (MetricName, MetricTypeName, Aggregate, Float)> for CarbonCodec { type Error = GeneralError; - fn encode(&mut self, item: (MetricName, MetricTypeName, Aggregate, Float), buf: &mut BytesMut) -> Result<(), Self::Error> { + fn encode(&mut self, item: &'a (MetricName, MetricTypeName, Aggregate, Float), buf: &mut BytesMut) -> Result<(), Self::Error> { let (name, typename, aggregate, value) = item; let options = &self.options; - if name.put_with_options(buf, typename, aggregate, &options.namings).is_err() { + if name.put_with_options(buf, typename.clone(), aggregate.clone(), &options.namings).is_err() { warn!("could not serialize '{:?}' with {:?}", &name.name[..], aggregate); s!(agg_errors); return Ok(()); @@ -238,7 +326,7 @@ impl Encoder<(MetricName, MetricTypeName, Aggregate, Float)> for CarbonCo buf.extend_from_slice(&b" "[..]); // write somehow doesn't extend buffer size giving "cannot fill sholw buffer" error buf.reserve(64); - if let Err(e) = ftoa::write(&mut buf.writer(), value) { + if let Err(e) = dtoa::write(&mut buf.writer(), *value) { warn!("buffer write error {:?}", e); s!(agg_errors); return Ok(()); @@ -246,7 +334,7 @@ impl Encoder<(MetricName, MetricTypeName, Aggregate, Float)> for CarbonCo buf.extend_from_slice(&b" "[..]); buf.extend_from_slice(&self.ts[..]); buf.extend_from_slice(&b"\n"[..]); - s!(egress); + s!(egress_carbon); Ok(()) } } @@ -265,10 +353,10 @@ mod test { use super::*; - use tokio2::net::TcpListener; - use tokio2::runtime::Builder as RBuilder; - use tokio2::stream::StreamExt; - use tokio2::time::delay_for; + use futures::TryFutureExt; + use tokio::net::TcpListener; + use tokio::runtime::Builder as RBuilder; + use tokio::time::sleep; use tokio_util::codec::{Decoder, LinesCodec}; use crate::config::{self, Aggregation}; @@ -281,9 +369,8 @@ mod test { let mut intermediate = Vec::with_capacity(128); intermediate.resize(128, 0u8); - let mut runtime = RBuilder::new() + let runtime = RBuilder::new_current_thread() .thread_name("bio_carbon_test") - .basic_scheduler() .enable_all() .build() .expect("creating runtime for carbon test"); @@ -301,33 +388,69 @@ mod test { agg_opts.round_timestamp = RoundTimestamp::Up; let naming = config::default_namings(); - let agg_opts = AggregationOptions::from_config(agg_opts, naming, log.clone()).unwrap(); + let agg_opts = AggregationOptions::from_config(agg_opts, 30., naming, log.clone()).unwrap(); let options = CarbonClientOptions { addr: "127.0.0.1:2003".parse().unwrap(), bind: None, interval: 30000, - options: agg_opts, + agg: agg_opts, }; //pub(crate) fn new(options: CarbonClientOptions, ts: u64, metrics: Arc, Float)>>, log: Logger) -> Self - let backend = CarbonBackend::new(options, ts, Arc::new(vec![(name, MetricTypeName::Gauge, Aggregate::Value, 42f64)]), log.clone()); + let backend = CarbonBackend::new( + options, + ts, + Arc::new(vec![vec![(name, MetricTypeName::Gauge, Aggregate::Value, 42.)]]), + log.clone(), + ); let server = async { - let mut listener = TcpListener::bind(&"127.0.0.1:2003".parse::<::std::net::SocketAddr>().unwrap()).await.unwrap(); + let listener = TcpListener::bind(&"127.0.0.1:2003".parse::<::std::net::SocketAddr>().unwrap()).await.unwrap(); // we need only one accept here yet let (conn, _) = listener.accept().await.expect("server getting next connection"); let mut codec = LinesCodec::new().framed(conn); while let Some(line) = codec.next().await { let line = line.unwrap(); // with "up" rounding the timestamp have to be rounded to 30th second which is at 1574745750 - assert_eq!(&line, "complex.test.bioyino_tagged;tag1=value1;tag2=val2 42 1574745750"); + assert_eq!(&line, "complex.test.bioyino_tagged;tag1=value1;tag2=val2 42.0 1574745750"); } }; runtime.spawn(server); runtime.spawn(backend.run().map_err(|_| panic!("codec error"))); - let test_delay = async { delay_for(Duration::from_secs(2)).await }; + let test_delay = async { sleep(Duration::from_secs(2)).await }; runtime.block_on(test_delay); } + + #[test] + fn test_rechunk() { + let sizes = vec![1, 2, 6, 13, 22, 22, 10, 1, 1, 1, 1, 1, 1, 25, 25, 120, 121, 122, 1, 1, 1]; + let mut vecs = Vec::new(); + let mut j = 0; + for size in sizes { + let mut v = Vec::new(); + for i in 0..size { + v.push(j * 100 + i); + } + vecs.push(v); + j += 1; + } + + for chunks in 1..25 { + let res = rechunk(vecs.clone(), chunks); + let mut lens = Vec::new(); + for res in res.into_iter() { + let chunk_len = res.into_iter().fold(0, |acc, elem| acc + elem.len()); + lens.push(chunk_len); + } + //dbg!(&lens); + if lens.len() > 1 { + let last = lens.len() - 2; + for i in 0..last { + assert_eq!(lens[i], lens[i + 1]); + } + } + } + } } diff --git a/src/config.rs b/src/config.rs index 4166bbf..489e9de 100644 --- a/src/config.rs +++ b/src/config.rs @@ -9,15 +9,21 @@ use bytes::Bytes; use raft_tokio::RaftOptions; use thiserror::Error; -use crate::aggregate::AggregationMode; use crate::management::{ConsensusAction, LeaderAction, MgmtCommand}; -use crate::{ConsensusKind, ConsensusState, Float}; +use crate::Float; use bioyino_metric::{ aggregate::Aggregate, - metric::MetricTypeName, + metric::{MetricTypeName, ProtocolVersion}, name::{AggregationDestination, NamingOptions}, }; +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case", deny_unknown_fields)] +pub enum ConsensusKind { + None, + Internal, +} + #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] pub struct System { @@ -36,9 +42,6 @@ pub struct System { /// Internal Raft settings pub raft: Raft, - /// Consul settings - pub consul: Consul, - /// Metric settings pub metrics: Metrics, @@ -54,11 +57,14 @@ pub struct System { /// Number of networking threads, use 0 for number of CPUs pub n_threads: usize, - /// Number of aggregating(worker) threads, set to 0 to use all CPU cores + /// Number of parsing threads, set to 0 to use all CPU cores + pub p_threads: usize, + + /// Number of worker threads, set to 0 to use all CPU cores pub w_threads: usize, - /// Number of core threads, doing other work, set to 0 to use all CPU cores - pub c_threads: usize, + /// Number of async core threads, doing async work, set to 0 to use all CPU cores + pub a_threads: usize, /// queue size for single counting thread before packet is dropped pub task_queue_size: usize, @@ -85,14 +91,14 @@ impl Default for System { daemon: false, network: Network::default(), raft: Raft::default(), - consul: Consul::default(), metrics: Metrics::default(), aggregation: Aggregation::default(), naming: default_namings(), carbon: Carbon::default(), n_threads: 4, + p_threads: 4, w_threads: 4, - c_threads: 4, + a_threads: 4, stats_interval: 10000, task_queue_size: 2048, start_as_leader: false, @@ -171,12 +177,6 @@ pub struct Aggregation { /// Timestamp rounding pub round_timestamp: RoundTimestamp, - /// Choose the way of aggregation - pub mode: AggregationMode, - - /// Number of threads when aggregating in "multi" mode - pub threads: Option, - /// Minimal update count to be reported pub update_count_threshold: u32, @@ -199,16 +199,18 @@ impl Default for Aggregation { fn default() -> Self { Self { round_timestamp: RoundTimestamp::No, - mode: AggregationMode::Single, - threads: None, update_count_threshold: 200, aggregates: all_aggregates().into_iter().map(|(k, v)| (k, Some(v))).collect(), } } } +// NOTE: while working with configuration values, the rate aggregate is intentionally set +// to None, because we only want aggregation interval at the very late stage (when we are ready to +// aggregate something) and in some future it may differ depending on backend type or other +// settings pub(crate) fn all_aggregates() -> HashMap>> { - let mut map = bioyino_metric::aggregate::possible_aggregates(); + let mut map = bioyino_metric::aggregate::possible_aggregates(None, None); let timers = map.get_mut(&MetricTypeName::Timer).expect("only BUG in possible_aggregates can panic here"); timers.push(Aggregate::Percentile(0.75, 75)); timers.push(Aggregate::Percentile(0.95, 95)); @@ -370,6 +372,9 @@ pub struct Network { /// Snapshot client bind address pub peer_client_bind: Option, + /// Version of Cap'n Proto protocol to use for encoding peer messages + pub peer_protocol: ProtocolVersion, + /// Address and port for management server to listen on pub mgmt_listen: SocketAddr, @@ -417,6 +422,7 @@ impl Default for Network { listen: "127.0.0.1:8125".parse().unwrap(), peer_listen: "127.0.0.1:8136".parse().unwrap(), peer_client_bind: None, + peer_protocol: ProtocolVersion::V2, mgmt_listen: "127.0.0.1:8137".parse().unwrap(), bufsize: 1500, multimessage: false, @@ -434,37 +440,6 @@ impl Default for Network { } } -#[derive(Debug, Clone, Serialize, Deserialize)] -#[serde(rename_all = "kebab-case", default, deny_unknown_fields)] -pub struct Consul { - /// Start in disabled leader finding mode - pub start_as: ConsensusState, - - /// Consul agent address - pub agent: SocketAddr, - - /// TTL of consul session, ms (consul cannot set it to less than 10s) - pub session_ttl: usize, - - /// How often to renew consul session, ms - pub renew_time: usize, - - /// Name of ke to be locked in consul - pub key_name: String, -} - -impl Default for Consul { - fn default() -> Self { - Self { - start_as: ConsensusState::Disabled, - agent: "127.0.0.1:8500".parse().unwrap(), - session_ttl: 11000, - renew_time: 1000, - key_name: "service/bioyino/lock".to_string(), - } - } -} - #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", default, deny_unknown_fields)] pub struct Raft { @@ -547,7 +522,13 @@ impl System { pub fn load() -> Result<(Self, Command), ConfigError> { // This is a first copy of args - with the "config" option let app = app_from_crate!() - .long_version(concat!(crate_version!(), " ", env!("VERGEN_COMMIT_DATE"), " ", env!("VERGEN_SHA_SHORT"))) + .long_version(concat!( + crate_version!(), + " ", + env!("VERGEN_GIT_COMMIT_TIMESTAMP"), + " ", + env!("VERGEN_GIT_SEMVER") + )) .arg( Arg::with_name("config") .help("configuration file path") @@ -626,6 +607,7 @@ mod tests { use crate::aggregate::AggregationOptions; use crate::util::prepare_log; + use bioyino_metric::FromF64; #[test] fn parsing_full_config() { @@ -638,7 +620,8 @@ mod tests { config.prepare().expect("preparing config"); let log = prepare_log("parse_full_config test"); - let aopts = AggregationOptions::from_config(config.aggregation, config.naming, log).expect("checking aggregate options"); + let interval: Float = Float::from_f64(config.carbon.interval as f64 / 1000.); + let aopts = AggregationOptions::from_config(config.aggregation, interval, config.naming, log).expect("checking aggregate options"); for (ty, _) in all_aggregates() { // all_aggregates contain all types and all possible aggregate pairs // so we must check aggregation_options has all of them after conversion diff --git a/src/consul.rs b/src/consul.rs deleted file mode 100644 index e3fdcd7..0000000 --- a/src/consul.rs +++ /dev/null @@ -1,371 +0,0 @@ -use std::net::SocketAddr; -use std::time::{Duration, Instant}; - -use futures::future::{err, loop_fn, ok, Either, Future, IntoFuture, Loop}; -use futures::Stream; -use hyper; -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::{debug, o, warn, Logger}; -use tokio::timer::{self, Delay, Interval}; - -use serde_derive::Deserialize; -use thiserror::Error; - -use crate::util::switch_leader; -use crate::{ConsensusState, CONSENSUS_STATE}; - -#[derive(Error, Debug)] -pub enum ConsulError { - #[error("session create error: {}", _0)] - Session(String), - - #[error("server responded with bad status code '{}': {}", _0, _1)] - HttpStatus(hyper::StatusCode, String), - - #[error("agent connection timed out")] - ConnectionTimeout, - - #[error("Http error: {}", _0)] - Http(#[from] hyper::Error), - - #[error("Parsing response: {}", _0)] - Parsing(#[from] serde_json::Error), - - #[error("I/O error {}", _0)] - Io(#[from] ::std::io::Error), - - #[error("{}", _0)] - Renew(String), - - #[error("creating timer: {}", _0)] - Timer(#[from] timer::Error), -} - -#[derive(Deserialize)] -struct ConsulSessionResponse { - #[serde(rename = "ID")] - id: String, -} - -pub struct ConsulConsensus { - log: Logger, - agent: SocketAddr, - key: String, - session_ttl: Duration, - renew_time: Duration, - error_pause: Duration, -} - -impl ConsulConsensus { - pub fn new(log: &Logger, agent: SocketAddr, key: String) -> Self { - Self { - log: log.new(o!("source"=>"consensus")), - agent, - key, - session_ttl: Duration::from_secs(7), - renew_time: Duration::from_secs(1), - error_pause: Duration::from_secs(1), - } - } - - pub fn set_key(&mut self, key: String) { - self.key = key; - } - - pub fn set_session_ttl(&mut self, ttl: Duration) { - self.session_ttl = ttl; - } - - pub fn set_renew_time(&mut self, ttl: Duration) { - self.renew_time = ttl; - } - - pub fn set_error_pause(&mut self, pause: Duration) { - self.error_pause = pause; - } -} - -impl IntoFuture for ConsulConsensus { - type Item = (); - type Error = ConsulError; - type Future = Box + Send>; - - fn into_future(self) -> Self::Future { - let Self { - log, - agent, - key, - session_ttl, - renew_time, - error_pause, - } = self; - - let renew_loop = loop_fn((), move |()| { - let key = key.clone(); - let log = log.clone(); - let session = ConsulSession { - log: log.new(o!("source"=>"consul-session")), - agent, - ttl: session_ttl, - }; - - 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); - */ - // 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 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 = _> + Send> - //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)))) - } - }); - - // the returned future will work until renew error - let renew = - // make connection, then start to renew - loop_session - .and_then(move |sid| { - let timer = Interval::new(Instant::now()+renew_time, renew_time); - - timer.map_err(ConsulError::Timer).for_each(move |_| { - let log = renewlog.clone(); - 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, - sid: sid.clone(), - ttl: session_ttl, - }.into_future() - .map_err(move |e| { - warn!(log, "session renew error"; "error"=> format!("{}",e)); - e - }); - - let log = renewlog.clone(); - let acquire = ConsulAcquire { - log: log.new(o!("source"=>"consul-acquire")), - agent, - sid: sid.clone(), - key: key.clone(), - }.into_future() - .map_err(move |e| { - warn!(log, "session acquire error"; "error"=>format!("{:?}", e)); - e - }); - - Either::A(renew.join(acquire).map(|_|())) - } else { - Either::B(ok(()))// as Box>) - } - }) - }); - - // restart the whole loop as soon as ANY future exits with any result - // (is is supposed to exit only with error) - renew.then(move |_| Delay::new(Instant::now() + error_pause).then(move |_| Ok(Loop::Continue(())))) - }); - Box::new(renew_loop) - } -} - -#[derive(Clone)] -pub struct ConsulSession { - log: Logger, - agent: SocketAddr, - ttl: Duration, -} - -impl IntoFuture for ConsulSession { - type Item = Option; - type Error = ConsulError; - type Future = Box + Send>; - - fn into_future(self) -> Self::Future { - let Self { log, agent, ttl } = self; - // create HTTP client for consul agent leader - 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.body_mut() = hyper::Body::from(b); - // Override sending request as multipart - 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).map_err(ConsulError::Http).and_then(move |resp| { - let status = resp.status(); - if status == StatusCode::OK { - let body = resp.into_body().concat2().map_err(ConsulError::Http).and_then(move |body| { - let resp: ConsulSessionResponse = - //try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); - from_slice(&body).map_err(ConsulError::Parsing)?; - debug!(log, "new session"; "id"=>resp.id.to_string()); - Ok(Some(resp.id)) - }); - Box::new(body) as Box, Error = ConsulError> + Send> - } else { - let body = resp.into_body().concat2().map_err(ConsulError::Http); - // TODO make this into option - let sleep = Delay::new(Instant::now() + Duration::from_millis(1000)).map_err(ConsulError::Timer); - let future = sleep.join(body).then(move |res| match res { - Ok((_, body)) => Err::, _>(ConsulError::HttpStatus(status, format!("{:?}", String::from_utf8(body.to_vec())))), - Err(e) => Err(e), - }); - Box::new(future) - } - }); - let timeout = Delay::new(Instant::now() + ttl); - let future = timeout - .map_err(ConsulError::Timer) - .map(|_| None) - .select(c_session) - .map(|res| res.0) - .map_err(|e| e.0); - Box::new(future) - } -} - -pub struct ConsulRenew { - agent: SocketAddr, - sid: String, - ttl: Duration, -} - -impl IntoFuture for ConsulRenew { - type Item = (); - type Error = ConsulError; - type Future = Box + Send>; - - fn into_future(self) -> Self::Future { - 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() * 1_000_000_000u64 + ttl.subsec_nanos() as u64; - let b = format!("{{\"TTL\": \"{}ns\"}}", ttl_ns); - let bodylen = b.len() as u64; - - *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(); - 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() != StatusCode::OK { - let status = resp.status(); - let body = resp.into_body().concat2().map_err(ConsulError::Http).and_then(move |body| { - let msg = format!("renew error: {:?} {:?}", status, String::from_utf8(body.to_vec())); - Err(ConsulError::Renew(msg)) - }); - Box::new(body) as Box + Send> - } else { - Box::new(ok(())) - } - } - }); - Box::new(future) - } -} - -pub struct ConsulAcquire { - log: Logger, - agent: SocketAddr, - sid: String, - key: String, -} - -impl IntoFuture for ConsulAcquire { - type Item = (); - type Error = ConsulError; - type Future = Box + Send>; - - fn into_future(self) -> Self::Future { - let Self { log, agent, sid, key } = self; - - 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(); - let acquire = client.request(req).map_err(ConsulError::Http).and_then(move |resp| { - resp.into_body().concat2().map_err(ConsulError::Http).and_then(move |body| { - let acquired: bool = - //try!(from_slice(&body).map_err(|e| ConsulError::Parsing(e))); - from_slice(&body).map_err(ConsulError::Parsing)?; - - 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(()) - }) - }); - Box::new(acquire) - } -} diff --git a/src/errors.rs b/src/errors.rs index 7793557..7598943 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -7,7 +7,7 @@ pub enum GeneralError { Io(#[from] ::std::io::Error), #[error("Error when creating timer: {}", _0)] - Timer(#[from] ::tokio::timer::Error), + Timer(#[from] ::tokio1::timer::Error), #[error("getting system time")] Time(#[from] ::std::time::SystemTimeError), @@ -24,6 +24,9 @@ pub enum GeneralError { #[error("unknown consensus state")] UnknownState, + #[error("metric error")] + Metric(#[from] bioyino_metric::MetricError), + #[error("configuration error: {}", _0)] Configuration(#[from] crate::config::ConfigError), diff --git a/src/fast_task.rs b/src/fast_task.rs new file mode 100644 index 0000000..e412d04 --- /dev/null +++ b/src/fast_task.rs @@ -0,0 +1,298 @@ +use std::collections::hash_map::Entry; +use std::collections::HashMap; +use std::sync::Arc; + +use bytes::{BufMut, BytesMut}; +use crossbeam_channel::{Select, Sender, TryRecvError}; +use futures::channel::oneshot; +use log::warn as logw; +use slog::{error, info, warn, Logger}; + +use bioyino_metric::parser::{MetricParser, MetricParsingError, ParseErrorHandler}; +use bioyino_metric::{name::MetricName, Metric, MetricTypeName, StatsdMetric}; + +use crate::config::System; + +use crate::{s, Cache, Float}; + +// Fast task should not block for a long time because it is responsible for +// parsing metrics coming from UDP ans storing them parsed in a short-living local cache +// which should be given away to be used in snapshots or global long-living cache +#[derive(Debug)] +pub enum FastTask { + Parse(u64, BytesMut), + Accumulate(MetricName, StatsdMetric), + TakeSnapshot(oneshot::Sender), +} + +pub fn start_fast_threads(log: Logger, config: Arc) -> Result<(Vec>, Vec>), std::io::Error> { + info!(log, "starting parsing threads"); + let threads = config.p_threads; + let mut chans = Vec::with_capacity(threads); + let mut prio_chans = Vec::with_capacity(threads); + for i in 0..threads { + let (tx, rx) = crossbeam_channel::bounded(config.task_queue_size); + let (prio_tx, prio_rx) = crossbeam_channel::bounded(config.task_queue_size); + chans.push(tx); + prio_chans.push(prio_tx); + let tlog = log.clone(); + let elog = log.clone(); + let cf = config.clone(); + std::thread::Builder::new().name(format!("bioyino_fast{}", i)).spawn(move || { + let mut runner = FastTaskRunner::new(tlog, cf, 8192); + let mut select = Select::new(); + let prio_op = select.recv(&prio_rx); + let lower_op = select.recv(&rx); + loop { + let op = select.ready(); + + if op == prio_op { + // on higher priority operation try to get all + // data from the priority channel + while let Ok(task) = prio_rx.try_recv() { + runner.run(task) + } + } else if op == lower_op { + match rx.try_recv() { + Ok(task) => { + runner.run(task); + } + Err(TryRecvError::Empty) => {} + Err(TryRecvError::Disconnected) => { + error!(elog, "UDP thread closed connection"); + return; + } + } + } + } + })?; + } + + Ok((chans, prio_chans)) +} + +fn update_metric(cache: &mut Cache, name: MetricName, metric: StatsdMetric) { + let ename = name.clone(); + let em = MetricTypeName::from_statsd_metric(&metric); + if em == MetricTypeName::CustomHistogram { + s!(agg_errors); + logw!("skipped histogram, histograms are not supported"); + return; + } + + match cache.entry(name) { + Entry::Occupied(ref mut entry) => { + entry.get_mut().accumulate_statsd(metric).unwrap_or_else(|_| { + let mtype = MetricTypeName::from_metric(&entry.get()); + logw!( + "could not accumulate {:?}: type '{}' into type '{}'", + String::from_utf8_lossy(&ename.name[..]), + em.to_string(), + mtype.to_string(), + ); + s!(agg_errors); + }); + } + + Entry::Vacant(entry) => match Metric::from_statsd(&metric, 1, None) { + Ok(metric) => { + entry.insert(metric); + } + Err(e) => { + s!(agg_errors); + logw!( + "could not create new metric from statsd metric at {:?}: {}", + String::from_utf8_lossy(&ename.name[..]), + e + ) + } + }, + } +} + +#[derive(Debug)] +pub struct FastTaskRunner { + short: HashMap>, + buffers: HashMap, + names_arena: BytesMut, + config: Arc, + log: Logger, +} + +impl FastTaskRunner { + pub fn new(log: Logger, config: Arc, cap: usize) -> Self { + Self { + short: HashMap::with_capacity(cap), + buffers: HashMap::with_capacity(cap), + names_arena: BytesMut::new(), + config, + log, + } + } + + pub fn run(&mut self, task: FastTask) { + match task { + FastTask::Parse(addr, buf) => { + let log = if self.config.metrics.log_parse_errors { Some(self.log.clone()) } else { None }; + let buf = { + let len = buf.len(); + let (_, ref mut prev_buf) = self + .buffers + .entry(addr) + .and_modify(|(times, _)| { + *times = 0; + }) + .or_insert((0, BytesMut::with_capacity(len))); + prev_buf.reserve(buf.len()); + prev_buf.put(buf); + prev_buf + }; + + let parser = MetricParser::new( + buf, + self.config.metrics.max_unparsed_buffer, + self.config.metrics.max_tags_len, + TaskParseErrorHandler(log), + ); + + for (mut name, metric) in parser { + s!(ingress_metrics); + if name.has_tags() && self.config.metrics.create_untagged_copy { + self.names_arena.extend_from_slice(name.name_without_tags()); + let untagged = MetricName::new_untagged(self.names_arena.split()); + update_metric(&mut self.short, untagged, metric.clone()); + } + + update_metric(&mut self.short, name, metric); + } + } + FastTask::Accumulate(name, metric) => update_metric(&mut self.short, name, metric), + FastTask::TakeSnapshot(channel) => { + // we need our cache to be sent for processing + // in place of it we need a new cache with most probably the same size + // BUT if we use exactly the same size, it may grow infinitely in long term + // so we halve the size so it could be reduced if ingress flow amounts + // become lower + + let mut rotated = HashMap::with_capacity(self.short.len() / 2); + std::mem::swap(&mut self.short, &mut rotated); + + channel.send(rotated).unwrap_or_else(|_| { + s!(queue_errors); + info!(self.log, "task could not send snapshot, receiving thread may be dead"); + }); + + let interval = self.config.carbon.interval; + self.buffers.retain(|_, (ref mut times, _)| { + *times += 1; + *times < interval as usize * 5 + }); + } + } + } +} + +struct TaskParseErrorHandler(Option); + +impl ParseErrorHandler for TaskParseErrorHandler { + fn handle(&self, input: &[u8], _pos: usize, e: MetricParsingError) { + s!(parse_errors); + if let Some(ref log) = self.0 { + if let Ok(string) = std::str::from_utf8(input) { + // TODO better error formatting instead of Debug + warn!(log, "parsing error"; "buffer"=> format!("{:?}", string), "position"=>format!("{}", e.position.translate_position(input)), "error"=>format!("{:?}", e)); + } else { + warn!(log, "parsing error (bad unicode)"; "buffer"=> format!("{:?}", input), "position"=>format!("{}",e.position.translate_position(input) ), "error"=>format!("{:?}", e)); + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use bioyino_metric::{name::TagFormat, MetricValue}; + + use crate::util::{new_test_graphite_name as new_name, prepare_log}; + + #[test] + fn accumulate_tagged_metrics() { + let mut data = BytesMut::new(); + + // ensure metrics with same tags (going probably in different orders) go to same aggregate + data.extend_from_slice(b"gorets;t2=v2;t1=v1:1000|c"); + data.extend_from_slice(b"\ngorets;t1=v1;t2=v2:1000|c"); + + // ensure metrics with same name but different tags go to different aggregates + data.extend_from_slice(b"\ngorets;t1=v1;t2=v3:1000|c"); + + let mut config = System::default(); + config.metrics.log_parse_errors = true; + let mut runner = FastTaskRunner::new(prepare_log("aggregate_tagged"), Arc::new(config), 16); + runner.run(FastTask::Parse(2, data)); + + let mut intermediate = Vec::new(); + intermediate.resize(9000, 0u8); + let mode = TagFormat::Graphite; + + // must be aggregated into two as sum + let key = MetricName::new("gorets;t1=v1;t2=v2".into(), mode, &mut intermediate).unwrap(); + assert!(runner.short.contains_key(&key), "could not find {:?}", key); + let metric = runner.short.get(&key).unwrap().clone(); + assert_eq!(metric.value(), &MetricValue::Counter(2000.)); + + // must be aggregated into separate + let key = MetricName::new("gorets;t1=v1;t2=v3".into(), mode, &mut intermediate).unwrap(); + assert!(runner.short.contains_key(&key), "could not find {:?}", key); + let metric = runner.short.get(&key).unwrap().clone(); + assert_eq!(metric.value(), &MetricValue::Counter(1000.)); + } + + #[test] + fn create_untagged_copy() { + let mut data = BytesMut::new(); + data.extend_from_slice(b"tagged.metric;t2=v2;t1=v1:1000|c"); + + let mut config = System::default(); + config.metrics.create_untagged_copy = true; + let mut runner = FastTaskRunner::new(prepare_log("aggregate_with_copy"), Arc::new(config), 16); + // "send" metric two times + runner.run(FastTask::Parse(2, data.clone())); + runner.run(FastTask::Parse(2, data)); + + assert_eq!(runner.short.len(), 2, "additional metrics apepar from nowhere"); + // must be aggregated into two as sum + let key = new_name("tagged.metric;t1=v1;t2=v2"); + assert!(runner.short.contains_key(&key), "could not find {:?}", key); + let metric = runner.short.get(&key).unwrap().clone(); + assert_eq!(metric.value(), &MetricValue::Counter(2000.)); + assert_eq!(metric.updates(), 2.); + + // ensure "independent" untagged version of tagged metric also exists with same values + let key = new_name("tagged.metric"); + assert!(runner.short.contains_key(&key), "could not find {:?}", key); + let metric = runner.short.get(&key).unwrap().clone(); + assert_eq!(metric.value(), &MetricValue::Counter(2000.)); + assert_eq!(metric.updates(), 2.); + } + + #[test] + fn parse_trashed_metric_buf() { + let mut data = BytesMut::new(); + data.extend_from_slice(b"trash\ngorets1:+1000|g\nTRASH\ngorets2;tag3=shit;t2=fuuck:-1000|g|@0.5\nMORE;tra=sh;|TrasH\nFUUU"); + + let mut config = System::default(); + config.metrics.log_parse_errors = true; + let mut runner = FastTaskRunner::new(prepare_log("parse_trashed"), Arc::new(config), 16); + runner.run(FastTask::Parse(2, data)); + + let key = new_name("gorets1"); + let metric = runner.short.get(&key).unwrap().clone(); + assert_eq!(metric.value(), &MetricValue::Gauge(1000.)); + + // expect tags to be sorted after parsing + let key = new_name("gorets2;t2=fuuck;tag3=shit"); + let metric = runner.short.get(&key).unwrap().clone(); + assert_eq!(metric.value(), &MetricValue::Gauge(-1000.)); + } +} diff --git a/src/main.rs b/src/main.rs index 79a4e72..c2b44ae 100644 --- a/src/main.rs +++ b/src/main.rs @@ -1,98 +1,68 @@ // General -//pub mod bigint; pub mod aggregate; pub mod carbon; pub mod config; -pub mod consul; pub mod errors; pub mod management; pub mod peer; pub mod raft; -pub mod server; +pub mod async_udp; +pub mod sync_udp; pub mod stats; -pub mod task; -pub mod udp; +pub mod fast_task; +pub mod slow_task; pub mod util; +pub mod cache; use std::collections::HashMap; use std::io; -use std::str::FromStr; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::{Arc, Mutex}; use std::time::{Duration, Instant}; use std::{panic, process, thread}; -use slog::{info, o}; +use slog::{info, o, debug}; -use futures::future::empty; -use futures::{Future as Future1, IntoFuture}; -use futures3::channel::mpsc; -use futures3::stream::StreamExt; +use futures::future::{pending, TryFutureExt}; -use futures3::future::{pending, TryFutureExt}; - -use serde_derive::{Deserialize, Serialize}; use slog::{error, warn}; -use tokio2::runtime::Builder; -use tokio2::time::interval_at; +use tokio::runtime::Builder; +use tokio::time::interval_at; -use tokio::runtime::current_thread::Runtime; -use tokio::timer::Delay; +use tokio1::runtime::current_thread::Runtime; +use tokio1::timer::Delay; use bioyino_metric::metric::Metric; use bioyino_metric::name::MetricName; use once_cell::sync::Lazy; use crate::carbon::carbon_timer; -use crate::config::{Command, Consul, Network, System}; -use crate::consul::ConsulConsensus; +use crate::config::{Command, Network, System}; use crate::errors::GeneralError; use crate::management::MgmtClient; use crate::peer::{NativeProtocolServer, NativeProtocolSnapshot}; use crate::raft::start_internal_raft; pub use crate::stats::OwnStats; -use crate::task::TaskRunner; -use crate::udp::{start_async_udp, start_sync_udp}; +use crate::fast_task::start_fast_threads; +use crate::slow_task::start_slow_threads; +use crate::sync_udp::start_sync_udp; +use crate::async_udp::start_async_udp; use crate::util::{retry_with_backoff, setup_logging, try_resolve, Backoff}; +use crate::management::ConsensusState; +use crate::config::ConsensusKind; // 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 +#[cfg(feature = "f32")] +pub type Float = f32; + +#[cfg(not(feature = "f32"))] pub type Float = f64; // a type to store pre-aggregated data pub type Cache = HashMap>; -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] -#[serde(rename_all = "kebab-case", deny_unknown_fields)] -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), - } - } -} - -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] -#[serde(rename_all = "kebab-case", deny_unknown_fields)] -pub enum ConsensusKind { - None, - Consul, - Internal, -} - pub static CONSENSUS_STATE: Lazy> = Lazy::new(|| Mutex::new(ConsensusState::Disabled)); pub static IS_LEADER: AtomicBool = AtomicBool::new(false); @@ -109,7 +79,6 @@ fn main() { Network { listen, peer_listen, - peer_client_bind, mgmt_listen, bufsize, multimessage, @@ -117,34 +86,20 @@ fn main() { mm_async, mm_timeout, buffer_flush_time, - buffer_flush_length: _, - greens, - async_sockets, - nodes, - snapshot_interval, - max_snapshots, - }, - raft, - consul: - Consul { - start_as: consul_start_as, - agent, - session_ttl: consul_session_ttl, - renew_time: consul_renew_time, - key_name: consul_key, + .. }, - metrics: _, - aggregation, - naming, - carbon, - n_threads, - w_threads, - c_threads, - stats_interval: s_interval, - task_queue_size, - start_as_leader, - stats_prefix, - consensus, + raft, + metrics: _, + aggregation, + naming, + carbon, + n_threads, + a_threads, + stats_interval: s_interval, + start_as_leader, + stats_prefix, + consensus, + .. } = system; if daemon && verbosity_syslog.is_off() { @@ -167,10 +122,9 @@ fn main() { slog_stdlog::init().unwrap(); - let mut runtime = Builder::new() - .thread_name("bioyino_main") - .threaded_scheduler() - .core_threads(c_threads) + let runtime = Builder::new_multi_thread() + .thread_name("bioyino_async") + .worker_threads(a_threads) .enable_all() .build() .expect("creating runtime for main thread"); @@ -202,38 +156,22 @@ fn main() { process::exit(42); })); - // Start counting threads - info!(log, "starting counting threads"); - let mut chans = Vec::with_capacity(w_threads); - for i in 0..w_threads { - let (tx, mut 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)) - .spawn(move || { - let mut runner = TaskRunner::new(tlog, cf, 8192); - let mut runtime = Builder::new().basic_scheduler().enable_all().build().expect("creating runtime for test"); - - let tasks = async { - while let Some(task) = rx.next().await { - runner.run(task); - } - }; + let elog = log.clone(); + let (fast_chans, fast_prio_chans) = start_fast_threads(log.clone(), config.clone()) + .map_err(move |e| error!(elog, "starting parsing worker threads"; "error" => format!("{}", e))) + .expect("starting parsing worker thread"); - runtime.block_on(tasks); - }) - .map_err(|e| error!(log, "worker thread dead: {:?}", e)) - .expect("starting counting worker thread"); - } + let elog = log.clone(); + let slow_chan = start_slow_threads(log.clone(), &config) + .map_err(move |e| error!(elog, "starting counting worker threads"; "error" => format!("{}", e))) + .expect("starting counting worker threads"); let own_stat_log = log.clone(); let stats_prefix = stats_prefix.trim_end_matches('.').to_string(); // Spawn future gatering bioyino own stats info!(own_stat_log, "starting own stats counter"); - let own_stats = OwnStats::new(s_interval, stats_prefix, chans.clone(), own_stat_log); - runtime.spawn(async { own_stats.run().await }); + let own_stats = OwnStats::new(s_interval, stats_prefix, slow_chan.clone(), fast_prio_chans.clone(), own_stat_log); + runtime.spawn(own_stats.run()); let compat_log = rlog.new(o!("thread" => "compat")); let snap_err_log = compat_log.clone(); @@ -247,6 +185,8 @@ fn main() { thread::Builder::new() .name("bioyino_compat".into()) .spawn(move || { + use futures1::future::empty; + use futures1::Future as Future1; let mut runtime = Runtime::new().expect("creating runtime for counting worker"); // Init leader state before starting backend @@ -283,31 +223,8 @@ fn main() { info!(flog, "consensus thread stopped"); }) - .expect("starting counting worker thread"); - } - ConsensusKind::Consul => { - warn!( - compat_log, - "CONSUL CONSENSUS IS DEPRECATED AND WILL BE REMOVED IN VERSION 0.8, CONSIDER USING BUILT IN RAFT" - ); - - if start_as_leader { - warn!( - compat_log, - "Starting as leader with enabled consensus. More that one leader is possible before consensus settle up." - ); + .expect("starting counting worker thread"); } - { - let mut con_state = CONSENSUS_STATE.lock().unwrap(); - info!(compat_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(|_| ())); - } ConsensusKind::None => { if !start_as_leader { // starting as non-leader in this mode can be useful for agent mode @@ -322,7 +239,7 @@ fn main() { runtime.block_on(empty::<(), ()>()).expect("compat thread failed"); }) - .expect("starting compat thread"); + .expect("starting compat thread"); // settings safe for asap restart info!(log, "starting snapshot receiver"); @@ -333,11 +250,11 @@ fn main() { retries: ::std::usize::MAX, }; - let server_chans = chans.clone(); + let server_chan = slow_chan.clone(); let server_log = log.clone(); let peer_server = retry_with_backoff(peer_server_bo, move || { let server_log = server_log.clone(); - let peer_server = NativeProtocolServer::new(server_log.clone(), peer_listen, server_chans.clone()); + let peer_server = NativeProtocolServer::new(server_log.clone(), peer_listen, server_chan.clone()); peer_server.run().inspect_err(move |e| { info!(server_log, "error running snapshot server"; "error"=>format!("{}", e)); }) @@ -347,11 +264,9 @@ fn main() { info!(log, "starting snapshot sender"); let snapshot = NativeProtocolSnapshot::new( &log, - nodes, - peer_client_bind, - Duration::from_millis(snapshot_interval as u64), - &chans, - max_snapshots, + config.clone(), + &fast_prio_chans, + slow_chan.clone(), ); runtime.spawn(snapshot.run().map_err(move |e| { @@ -361,66 +276,61 @@ fn main() { info!(log, "starting management server"); let m_serv_log = rlog.clone(); - let m_server = async move { hyper13::Server::bind(&mgmt_listen).serve(management::MgmtServer(m_serv_log, mgmt_listen)).await }; + let m_server = async move { hyper::Server::bind(&mgmt_listen).serve(management::MgmtServer(m_serv_log, mgmt_listen)).await }; runtime.spawn(m_server); info!(log, "starting carbon backend"); let carbon_log = log.clone(); - let carbon_t = carbon_timer(log.clone(), carbon, aggregation, naming, chans.clone()) + let carbon_t = carbon_timer(log.clone(), carbon, aggregation, naming, slow_chan.clone()) .map_err(move |e| error!(carbon_log, "running carbon thread"; "error" => format!("{}", e))); runtime.spawn(carbon_t); - // For each out sync thread we create the buffer flush timer, that sets the atomic value to 1 - // every interval - 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)); - } - } - - if buffer_flush_time > 0 { - let flags = flush_flags.clone(); - let flush_timer = async move { - let dur = Duration::from_millis(buffer_flush_time); - let mut timer = interval_at(tokio2::time::Instant::now() + dur, dur); - - loop { - timer.tick().await; - info!(rlog, "buffer flush requested"); - flags.iter().map(|flag| flag.swap(true, Ordering::SeqCst)).last(); - } - }; - runtime.spawn(flush_timer); - } if multimessage { - start_sync_udp( + let flush_flags = start_sync_udp( log, listen, - &chans, + &fast_chans, config.clone(), n_threads, bufsize, mm_packets, mm_async, mm_timeout, - flush_flags.clone(), ); + // spawn a flushing timer if required + if buffer_flush_time > 0 { + let flush_timer = async move { + let dur = Duration::from_millis(buffer_flush_time); + let mut timer = interval_at(tokio::time::Instant::now() + dur, dur); + + loop { + timer.tick().await; + debug!(rlog, "buffer flush requested"); + flush_flags.iter().map(|flag| flag.swap(true, Ordering::SeqCst)).last(); + } + }; + runtime.spawn(flush_timer); + } } else { - start_async_udp( - log, - listen, - &chans, - config.clone(), - n_threads, - greens, - async_sockets, - bufsize, - flush_flags.clone(), - ); - } + info!(log, "multimessage is disabled, starting in async UDP mode"); + let flush_sender = start_async_udp(log, &fast_chans, config.clone()); + + if buffer_flush_time > 0 { + let flush_timer = async move { + let dur = Duration::from_millis(buffer_flush_time); + let mut timer = interval_at(tokio::time::Instant::now() + dur, dur); + + loop { + timer.tick().await; + debug!(rlog, "buffer flush requested"); + flush_sender.notify_waiters(); + } + }; + runtime.spawn(flush_timer); + } + }; runtime.block_on(pending::<()>()); } diff --git a/src/management.rs b/src/management.rs index 72e2394..518d8ba 100644 --- a/src/management.rs +++ b/src/management.rs @@ -1,23 +1,21 @@ -use std::collections::HashMap; -use std::iter::FromIterator; use std::net::SocketAddr; use std::pin::Pin; use std::str::FromStr; use std::sync::atomic::Ordering; use std::task::{Context, Poll}; -use bytes::{buf::BufMutExt, BytesMut}; -use futures3::future::{ok, Future}; +use futures::future::{ok, Future}; use slog::{info, o, warn, Logger}; -use hyper13::service::Service; -use hyper13::{self, body::to_bytes, http, Body, Client, Method, Request, Response, StatusCode}; +use hyper::service::Service; +use hyper::{self, body::to_bytes, http, Body, client::Client, Method, Request, Response, StatusCode}; use serde_derive::{Deserialize, Serialize}; use thiserror::Error; use crate::stats::STATS_SNAP; -use crate::{ConsensusState, Float, CONSENSUS_STATE, IS_LEADER}; +use crate::GeneralError; +use crate::{CONSENSUS_STATE, IS_LEADER}; #[derive(Error, Debug)] pub enum MgmtError { @@ -25,7 +23,7 @@ pub enum MgmtError { Io(#[from] ::std::io::Error), #[error("Http error: {}", _0)] - Http(#[from] hyper13::error::Error), + Http(#[from] hyper::Error), #[error("Http error: {}", _0)] HttpProto(#[from] http::Error), @@ -43,6 +41,27 @@ pub enum MgmtError { Response, } +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case", deny_unknown_fields)] +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), + } + } +} + // Top level list of available commands #[derive(Clone, Debug, Serialize, Deserialize)] #[serde(rename_all = "kebab-case", deny_unknown_fields)] @@ -131,6 +150,8 @@ impl MgmtService { log, } } + + } impl Service> for MgmtService { @@ -173,7 +194,6 @@ impl Service> for MgmtService { Box::pin(ok(response)) } (&Method::GET, "/stats", qry) => { - let mut buf = BytesMut::new(); let mut json = false; if let Some(qry) = qry { for (k, v) in qry { @@ -183,45 +203,16 @@ impl Service> for MgmtService { } } } - if !json { - let snapshot = STATS_SNAP.lock().unwrap(); - let ts = snapshot.ts.to_string(); - for (name, value) in &snapshot.data { - buf.extend_from_slice(&name[..]); - buf.extend_from_slice(&b" "[..]); - // write somehow doesn't extend buffer size giving "cannot fill buffer" error - buf.reserve(64); - let mut writer = buf.writer(); - ftoa::write(&mut writer, *value).unwrap_or(()); // TODO: think if we should not ignore float error - buf = writer.into_inner(); - buf.extend_from_slice(&b" "[..]); - buf.extend_from_slice(ts.as_bytes()); - buf.extend_from_slice(&b"\n"[..]); - } - } else { - let snapshot = STATS_SNAP.lock().unwrap(); - #[derive(Serialize)] - struct JsonSnap { - ts: u128, - metrics: HashMap, - } - - let snap = JsonSnap { - ts: snapshot.ts, - metrics: HashMap::from_iter( - snapshot - .data - .iter() - .map(|(name, value)| (String::from_utf8_lossy(&name[..]).to_string(), *value)), - ), + Box::pin(async move { + let snapshot = { + let snapshot = STATS_SNAP.read().await; + snapshot.clone() }; - let mut writer = buf.writer(); - serde_json::to_writer_pretty(&mut writer, &snap).unwrap_or(()); - buf = writer.into_inner(); - } - *response.body_mut() = Body::from(buf.freeze()); - Box::pin(ok(response)) + let buf = snapshot.render(json); + *response.body_mut() = Body::from(buf); + Ok(response) + }) } (&Method::GET, _, _) => { *response.status_mut() = StatusCode::NOT_FOUND; @@ -296,7 +287,7 @@ pub struct MgmtServer(pub Logger, pub SocketAddr); impl Service for MgmtServer { type Response = MgmtService; type Error = std::io::Error; - type Future = futures3::future::Ready>; + type Future = futures::future::Ready>; fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll> { Ok(()).into() @@ -304,7 +295,7 @@ impl Service for MgmtServer { fn call(&mut self, _: T) -> Self::Future { ok(MgmtService::new( - self.0.new(o!("source"=>"management-server", "server"=>format!("{}", self.1.clone()))), + self.0.new(o!("source"=>"management-server", "server"=>format!("{}", self.1.clone()))), )) } } @@ -391,8 +382,8 @@ mod test { use {slog, slog_async, slog_term}; use slog::{Drain, Logger}; - use tokio2::runtime::{Builder, Runtime}; - use tokio2::time::delay_for; + use tokio::runtime::{Builder, Runtime}; + use tokio::time::sleep; use super::*; fn prepare_log() -> Logger { @@ -409,10 +400,10 @@ mod test { let rlog = prepare_log(); let mgmt_listen: ::std::net::SocketAddr = "127.0.0.1:8137".parse().unwrap(); - let runtime = Builder::new().basic_scheduler().enable_all().build().expect("creating runtime for main thread"); + let runtime = Builder::new_current_thread().enable_all().build().expect("creating runtime for testing management server"); let m_serv_log = rlog.clone(); - let m_server = async move { hyper13::Server::bind(&mgmt_listen).serve(MgmtServer(m_serv_log, mgmt_listen)).await }; + let m_server = async move { hyper::Server::bind(&mgmt_listen).serve(MgmtServer(m_serv_log, mgmt_listen)).await }; runtime.spawn(m_server); @@ -421,12 +412,12 @@ mod test { #[test] fn management_command() { - let (mut runtime, log, address) = prepare_runtime_with_server(); + let (runtime, log, address) = prepare_runtime_with_server(); let check = async move { // let server some time to settle // then test the commands - let d = delay_for(Duration::from_secs(1)); + let d = sleep(Duration::from_secs(1)); d.await; // status @@ -451,7 +442,7 @@ mod test { }; runtime.spawn(check); - let test_delay = async { delay_for(Duration::from_secs(3)).await }; + let test_delay = async { sleep(Duration::from_secs(3)).await }; runtime.block_on(test_delay); } } diff --git a/src/peer.rs b/src/peer.rs index 3fbffb6..84be7c5 100644 --- a/src/peer.rs +++ b/src/peer.rs @@ -1,35 +1,41 @@ use std::collections::HashMap; use std::net::SocketAddr; use std::num::NonZeroUsize; -use std::sync::Arc; +use std::sync::{atomic::Ordering, Arc}; use std::time::Duration; use capnp::message::{Builder, ReaderOptions}; use slog::{debug, error, info, o, warn, Logger}; use thiserror::Error; -use futures3::channel::mpsc::Sender; -use futures3::channel::oneshot; -use futures3::SinkExt; +use crossbeam_channel::Sender; +use futures::channel::oneshot; -use futures3::future::join_all; -use futures3::future::TryFutureExt; +use futures::future::{join_all, TryFutureExt}; use ring_channel::{ring_channel, RingReceiver, RingSender}; -use tokio2::net::{TcpListener, TcpStream}; -use tokio2::spawn; -use tokio2::stream::StreamExt; -use tokio2::time::{interval_at, Instant}; -use tokio_util::compat::Tokio02AsyncReadCompatExt; +use tokio::{ + io::AsyncWriteExt, + net::{TcpListener, TcpStream}, + spawn, + time::{interval_at, Instant}, +}; + +use tokio_stream::StreamExt; +use tokio_util::compat::TokioAsyncReadCompatExt; use bioyino_metric::protocol_capnp::{message as cmsg, message::Builder as CBuilder}; -use bioyino_metric::Metric; +use bioyino_metric::protocol_v2_capnp::{message as cmsgv2, message::Builder as CBuilderV2}; +use bioyino_metric::{metric::ProtocolVersion, Metric}; -use crate::task::Task; +use crate::config::System; +use crate::fast_task::FastTask; +use crate::slow_task::SlowTask; +use crate::stats::STATS; use crate::util::{bound_stream, resolve_with_port, Backoff}; -use crate::{s, Cache, Float}; +use crate::{s, Cache, ConsensusKind, Float, IS_LEADER}; const CAPNP_READER_OPTIONS: ReaderOptions = ReaderOptions { - traversal_limit_in_words: 8 * 1024 * 1024 * 1024, + traversal_limit_in_words: Some(8 * 1024 * 1024 * 1024), nesting_limit: 16, }; @@ -39,7 +45,7 @@ pub enum PeerError { Io(#[from] ::std::io::Error), #[error("Error when creating timer: {}", _0)] - Timer(#[from] ::tokio::timer::Error), + Timer(#[from] ::tokio1::timer::Error), #[error("error sending task to worker thread")] TaskSend, @@ -79,65 +85,58 @@ pub enum PeerError { pub struct NativeProtocolServer { log: Logger, listen: SocketAddr, - chans: Vec>, + chan: Sender, } impl NativeProtocolServer { - pub fn new(log: Logger, listen: SocketAddr, chans: Vec>) -> Self { + pub fn new(log: Logger, listen: SocketAddr, chan: Sender) -> Self { Self { log: log.new(o!("source"=>"canproto-peer-server", "ip"=>format!("{}", listen.clone()))), listen, - chans, + chan, } } pub(crate) async fn run(self) -> Result<(), PeerError> { - let Self { log, listen, chans } = self; - let mut listener = TcpListener::bind(&listen).await?; - let mut incoming = listener.incoming(); - - let chlen = chans.len(); - let mut next_chan = chlen - 1; - while let Some(conn) = incoming.next().await { - let conn = conn?; - let peer_addr = conn.peer_addr().map(|addr| addr.to_string()).unwrap_or_else(|_| "[UNCONNECTED]".into()); + let Self { log, listen, chan } = self; + let listener = TcpListener::bind(&listen).await?; + + loop { + let (conn, peer_addr) = listener.accept().await?; let mut conn = conn.compat(); let log = log.new(o!("remote"=>peer_addr)); // debug!(log, "got new connection"); let elog = log.clone(); - let mut chans = chans.clone(); - next_chan = if next_chan >= (chlen - 1) { 0 } else { next_chan + 1 }; - + let chan = chan.clone(); let receiver = async move { let elog = log.clone(); //let transport = capnp_futures::ReadStream::new(&mut conn, CAPNP_READER_OPTIONS); - //while let Some(reader) = futures3::stream::TryStreamExt::try_next(&mut transport).await? + //while let Some(reader) = futures::stream::TryStreamExt::try_next(&mut transport).await? loop { - let reader = if let Some(reader) = capnp_futures::serialize::read_message(&mut conn, CAPNP_READER_OPTIONS).await? { + let reader = if let Ok(reader) = capnp_futures::serialize::read_message(&mut conn, CAPNP_READER_OPTIONS).await { reader } else { break; }; - //while let Some(reader) = futures3::stream::TryStreamExt::try_next(&mut transport).await? - let task = { - let elog = elog.clone(); - // debug!(log, "received peer message"); - parse_and_send(log.clone(), reader).map_err(move |e| { - warn!(elog, "bad incoming message"; "error" => e.to_string()); - e - })? - }; - - next_chan = if next_chan >= (chlen - 1) { 0 } else { next_chan + 1 }; - let chan = &mut chans[next_chan]; - chan.send(task) - .map_err(|_| { - s!(queue_errors); - PeerError::TaskSend - }) - .await?; + tokio::task::block_in_place(|| -> Result<(), PeerError> { + let task = { + let elog = elog.clone(); + //debug!(log, "received peer message"); + parse_capnp(log.clone(), reader).map_err(move |e| { + warn!(elog, "bad incoming message"; "error" => e.to_string()); + e + })? + }; + if let Some(task) = task { + chan.send(task).map_err(|_| { + s!(queue_errors); + PeerError::TaskSend + })?; + } + Ok(()) + })?; } Ok::<(), PeerError>(()) } @@ -148,77 +147,100 @@ impl NativeProtocolServer { spawn(receiver); } - Ok(()) } } -//fn parse_and_send(reader: cmsg::Reader<'_>) -> Result { -fn parse_and_send(log: Logger, reader: capnp::message::Reader) -> Result { +fn parse_capnp(log: Logger, reader: capnp::message::Reader) -> Result, PeerError> { + if let Ok(reader) = reader.get_root::() { + // somehow this does not return an error for v1 + if reader.get_version() == ProtocolVersion::V2.id() { + return match reader.which()? { + cmsgv2::Noop(()) => Ok(None), + cmsgv2::Snapshot(reader) => { + let reader = reader?; + let mut metrics = Vec::new(); + reader + .iter() + .map(|reader| Metric::::from_capnp(reader).map(|(name, metric)| metrics.push((name, metric)))) + .last(); + + debug!(log, "received snapshot v2"; "metrics"=>format!("{}", metrics.len())); + STATS.ingress_metrics_peer.fetch_add(metrics.len(), Ordering::Relaxed); + Ok(Some(SlowTask::AddSnapshot(metrics))) + } + }; + } + } + let reader = reader.get_root::()?; match reader.which()? { cmsg::Single(reader) => { let reader = reader?; - let (name, metric) = Metric::::from_capnp(reader)?; - // debug!(log, "received single-metric message"); - Ok(Task::AddMetric(name, metric)) + let (name, metric) = Metric::::from_capnp_v1(reader)?; + STATS.ingress_metrics_peer.fetch_add(1, Ordering::Relaxed); + Ok(Some(SlowTask::AddMetric(name, metric))) } cmsg::Multi(reader) => { let reader = reader?; let mut metrics = Vec::new(); reader .iter() - .map(|reader| Metric::::from_capnp(reader).map(|(name, metric)| metrics.push((name, metric)))) + .map(|reader| Metric::::from_capnp_v1(reader).map(|(name, metric)| metrics.push((name, metric)))) .last(); - // debug!(log, "received multi-metric message"; "amount"=>format!("{}", metrics.len())); - Ok(Task::AddMetrics(metrics)) + STATS.ingress_metrics_peer.fetch_add(metrics.len(), Ordering::Relaxed); + Ok(Some(SlowTask::AddMetrics(metrics))) } cmsg::Snapshot(reader) => { let reader = reader?; let mut metrics = Vec::new(); reader .iter() - .map(|reader| Metric::::from_capnp(reader).map(|(name, metric)| metrics.push((name, metric)))) + .map(|reader| Metric::::from_capnp_v1(reader).map(|(name, metric)| metrics.push((name, metric)))) .last(); - debug!(log, "received snapshot"; "metrics"=>format!("{}", metrics.len())); - Ok(Task::AddSnapshot(metrics)) + debug!(log, "received snapshot v1"; "metrics"=>format!("{}", metrics.len())); + STATS.ingress_metrics_peer.fetch_add(metrics.len(), Ordering::Relaxed); + Ok(Some(SlowTask::AddSnapshot(metrics))) } } } pub struct NativeProtocolSnapshot { log: Logger, - nodes: Vec, - client_bind: Option, + config: Arc, interval: Duration, - chans: Vec>, + fast_chans: Vec>, + slow_chan: Sender, snapshots: NonZeroUsize, } impl NativeProtocolSnapshot { - pub fn new(log: &Logger, nodes: Vec, client_bind: Option, interval: Duration, chans: &[Sender], mut snapshots: usize) -> Self { - if snapshots == 0 { - warn!(log, "snapshots cannot be 0, value is set to 1"); - snapshots = 1 - } + pub fn new(log: &Logger, config: Arc, fast_chans: &[Sender], slow_chan: Sender) -> Self { + let snapshots = if config.network.max_snapshots == 0 { + warn!(log, "max_snapshots set to 0 in config, this is incorrect, real value is set to 1"); + 1 + } else { + config.network.max_snapshots + }; + let interval = Duration::from_millis(config.network.snapshot_interval as u64); Self { log: log.new(o!("source"=>"peer-client")), - nodes, - client_bind, - interval, - chans: chans.to_owned(), + config, + fast_chans: fast_chans.to_owned(), + slow_chan, snapshots: NonZeroUsize::new(snapshots).unwrap(), + interval, } } pub(crate) async fn run(self) -> Result<(), PeerError> { let Self { log, - nodes, - client_bind, - interval, - mut chans, + config, + fast_chans, + slow_chan, snapshots, + interval, } = self; // Snapshots come every `interval`. When one of the nodes goes down it is possible to leak // all the memory if interval is less than backoff period because in this case snapshots will @@ -229,21 +251,21 @@ impl NativeProtocolSnapshot { // Since all metrics is lost after some time on remote node, it is only reasonable to store // few latest intervals. This is the reason of using a ring buffer instead of a channel. - let mut node_chans = nodes + let mut node_chans = config + .network + .nodes .iter() .map(|address| { - let (tx, rx): (RingSender>>, RingReceiver>>) = ring_channel(snapshots); + let (tx, rx): (RingSender>>, RingReceiver>>) = ring_channel(snapshots); let log = log.clone(); let options = SnapshotClientOptions { address: address.clone(), - bind: client_bind, + bind: config.network.peer_client_bind, + proto_version: config.network.peer_protocol.clone(), }; - spawn(async move { - let client = SnapshotSender::new(rx, options.clone(), log.clone()); - client.run().await - }); - //spawn(snapshot_sender); + let client = SnapshotSender::new(rx, options.clone(), log.clone()); + spawn(client.run()); tx }) .collect::>(); @@ -253,23 +275,49 @@ impl NativeProtocolSnapshot { timer.tick().await; // send snapshot requests to workers let mut outs = Vec::new(); - for chan in chans.iter_mut() { - let (tx, rx) = oneshot::channel(); - chan.send(Task::TakeSnapshot(tx)).await.unwrap_or(()); - outs.push(rx.unwrap_or_else(|_| { - s!(queue_errors); - HashMap::new() - })); - } + debug!(log, "taking snapshot from UDP"); + tokio::task::block_in_place(|| { + for chan in &fast_chans { + let (tx, rx) = oneshot::channel(); + chan.send(FastTask::TakeSnapshot(tx)).unwrap_or(()); + outs.push(rx.unwrap_or_else(|_| { + s!(queue_errors); + HashMap::new() + })); + } + }); // and start waiting for them - let mut all_metrics = join_all(outs).await; - all_metrics.retain(|m| !m.is_empty()); - let all_metrics = Arc::new(all_metrics); + let mut caches = join_all(outs).await; + caches.retain(|m| !m.is_empty()); + + // snapshots are relatively big and we don't want to copy them, + // so we wrap them in Arc first + + let caches = caches.into_iter().map(Arc::new).collect::>(); + + // Send snapshots to aggregation if required + let is_leader = IS_LEADER.load(Ordering::SeqCst); + + // there is special case used in agents: when we are not leader and there is + // no consensus, that cannot make us leader, there is no point of aggregating + // long cache at all because it will never be sent anywhere + if !is_leader && config.consensus == ConsensusKind::None { + debug!(log, "skipped aggregating UDP snapshot (not leader)"); + } else { + tokio::task::block_in_place(|| { + for cache in &caches { + slow_chan.send(SlowTask::Join(cache.clone())).unwrap_or_else(|_| { + s!(queue_errors); + info!(log, "task could not send snapshot, receiving thread may be dead"); + }); + } + }); + } // after that clone snapshots to all nodes' queues for ch in &mut node_chans { // sender has sync send method which conflicts with one from Sink - futures3::SinkExt::send(ch, all_metrics.clone()) + futures::SinkExt::send(ch, caches.clone()) .await .map_err(|_| { s!(queue_errors); @@ -277,6 +325,7 @@ impl NativeProtocolSnapshot { }) .unwrap_or(()); } + debug!(log, "UDP snapshot done"); } } } @@ -285,19 +334,20 @@ impl NativeProtocolSnapshot { pub struct SnapshotClientOptions { address: String, bind: Option, + proto_version: ProtocolVersion, } #[derive(Clone)] pub struct SnapshotSender { //metrics: Arc>, options: SnapshotClientOptions, - rx: RingReceiver>>, + rx: RingReceiver>>, log: Logger, } impl SnapshotSender { - pub fn new(rx: RingReceiver>>, options: SnapshotClientOptions, log: Logger) -> Self { - let log = log.new(o!("remote"=>options.address.clone())); + pub fn new(rx: RingReceiver>>, options: SnapshotClientOptions, log: Logger) -> Self { + let log = log.new(o!("task"=>"snapshot sender", "remote"=>options.address.clone())); Self { rx, options, log } } @@ -316,33 +366,63 @@ impl SnapshotSender { debug!(log, "skipped empty snapshot"); continue; } - let snapshot_message = { - let mut snapshot_message = Builder::new_default(); - let builder = snapshot_message.init_root::(); - let mut multi_metric = builder.init_snapshot(mlen as u32); - metrics - .iter() - .flat_map(|hmap| hmap.iter()) - .enumerate() - .map(|(idx, (name, metric))| { - let mut c_metric = multi_metric.reborrow().get(idx as u32); - // parsing stage has a guarantee that name is a valid unicode - // metrics that come over capnproto also has Text type in schema, - // so capnproto decoder will ensure unicode here - let name = unsafe { ::std::str::from_utf8_unchecked(name.name_with_tags()) }; - c_metric.set_name(&name); - metric.fill_capnp(&mut c_metric); - }) - .last(); + let snapshot_message = match options.proto_version { + ProtocolVersion::V1 => { + let mut snapshot_message = Builder::new_default(); + let builder = snapshot_message.init_root::(); + let mut multi_metric = builder.init_snapshot(mlen as u32); + metrics + .iter() + .flat_map(|hmap| hmap.iter()) + .enumerate() + .map(|(idx, (name, metric))| { + let mut c_metric = multi_metric.reborrow().get(idx as u32); + // parsing stage has a guarantee that name is a valid unicode + // metrics that come over capnproto also has Text type in schema, + // so capnproto decoder will ensure unicode here + let name = unsafe { ::std::str::from_utf8_unchecked(name.name_with_tags()) }; + c_metric.set_name(&name); + metric.fill_capnp_v1(&mut c_metric); + }) + .last(); - // this is just an approximate capacity to avoid first small allocations - let mut buf = Vec::with_capacity(mlen * 8); - if let Err(e) = capnp::serialize::write_message(&mut buf, &snapshot_message) { - error!(log, "encoding packed message"; "error" => format!("{}", e)); - return; - }; - buf - // capnp::serialize::write_message_to_words(&snapshot_message) + // this is just an approximate capacity to avoid first small allocations + let mut buf = Vec::with_capacity(mlen * 8); + if let Err(e) = capnp::serialize::write_message(&mut buf, &snapshot_message) { + error!(log, "encoding packed message"; "error" => format!("{}", e)); + return; + }; + buf + } + ProtocolVersion::V2 => { + let mut snapshot_message = Builder::new_default(); + let mut builder = snapshot_message.init_root::(); + builder.set_version(options.proto_version.id()); + let mut multi_metric = builder.init_snapshot(mlen as u32); + metrics + .iter() + .flat_map(|hmap| hmap.iter()) + .enumerate() + .map(|(idx, (name, metric))| { + let mut c_metric = multi_metric.reborrow().get(idx as u32); + // parsing stage has a guarantee that name is a valid unicode + // metrics that come over capnproto also has Text type in schema, + // so capnproto decoder will ensure unicode here + // this we can pass true as unicode_checked flag, meaning it will + // not be rechecked + metric.fill_capnp_name(&mut c_metric, &name, true); + metric.fill_capnp(&mut c_metric); + }) + .last(); + + // this is just an approximate capacity to avoid first small allocations + let mut buf = Vec::with_capacity(mlen * 8); + if let Err(e) = capnp::serialize::write_message(&mut buf, &snapshot_message) { + error!(log, "encoding packed message"; "error" => format!("{}", e)); + return; + }; + buf + } }; let mut backoff = Backoff { @@ -354,29 +434,26 @@ impl SnapshotSender { loop { let connect_and_send = async { - //debug!(log, "resolving"); let addr = resolve_with_port(&options.address, 8136).await?; - - //debug!(log, "connecting"); let mut conn = match options.bind { Some(bind_addr) => { let std_stream = bound_stream(&bind_addr)?; - TcpStream::connect_std(std_stream, &addr).await? + TcpStream::from_std(std_stream)? } None => TcpStream::connect(&addr).await?, }; - use tokio2::io::AsyncWriteExt; //let mut conn = conn.compat_write(); - info!(log, "writing snapshot"; "bytes" => format!("{}", snapshot_message.len()), "metrics" => format!("{}", mlen)); + STATS.egress_peer.fetch_add(mlen, Ordering::Relaxed); + debug!(log, "writing snapshot"; "bytes" => format!("{}", snapshot_message.len()), "metrics" => format!("{}", mlen)); //let write = write_message(&mut conn, &snapshot_message).map_err(|e| { let write = conn.write_all(&snapshot_message).map_err(|e| { warn!(log, "error encoding snapshot"; "error"=>e.to_string()); PeerError::Io(e) }); - tokio2::time::timeout(std::time::Duration::from_millis(30000), write) + tokio::time::timeout(std::time::Duration::from_millis(30000), write) .await .map_err(|_| PeerError::SnapshotWriteTimeout)??; debug!(log, "flushing"); @@ -405,106 +482,108 @@ impl SnapshotSender { mod test { use std::net::SocketAddr; - use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; - //use bytes::BytesMut; use capnp::message::Builder; use capnp_futures::serialize::write_message; - use futures3::channel::mpsc::{self, Receiver}; + + use crossbeam_channel::Receiver; + use futures::{FutureExt, TryFutureExt}; use slog::{debug, Logger}; - use tokio2::runtime::{Builder as RBuilder, Runtime}; - use tokio2::time::delay_for; - use tokio_util::compat::Tokio02AsyncWriteCompatExt; + use tokio::runtime::{Builder as RBuilder, Runtime}; + use tokio::time::sleep; + use tokio_util::compat::TokioAsyncWriteCompatExt; use bioyino_metric::name::{MetricName, TagFormat}; - use bioyino_metric::{Metric, MetricType}; + use bioyino_metric::{Metric, MetricValue}; + use crate::cache::SharedCache; use crate::config::System; - use crate::task::TaskRunner; + use crate::slow_task::SlowTaskRunner; use crate::util::prepare_log; use super::*; - fn prepare_runtime_with_server(log: Logger) -> (Runtime, Receiver, SocketAddr) { - let mut chans = Vec::new(); - let (tx, rx) = mpsc::channel(5); - chans.push(tx); + fn prepare_runtime_with_server(log: Logger, address: SocketAddr) -> (Runtime, Sender, Receiver, tokio::task::JoinHandle<()>) { + let (tx, rx) = crossbeam_channel::bounded(5); - let address: ::std::net::SocketAddr = "127.0.0.1:8136".parse().unwrap(); - let runtime = RBuilder::new() + let runtime = RBuilder::new_multi_thread() .thread_name("bio_peer_test") - .basic_scheduler() .enable_all() .build() .expect("creating runtime for test"); let peer_listen = address.clone(); let server_log = log.clone(); - let peer_server = NativeProtocolServer::new(server_log.clone(), peer_listen, chans.clone()); + let peer_server = NativeProtocolServer::new(server_log.clone(), peer_listen, tx.clone()); let peer_server = peer_server.run().inspect_err(move |e| { debug!(server_log, "error running snapshot server"; "error"=>format!("{}", e)); panic!("shot server"); }); - runtime.spawn(peer_server); + let handle = runtime.spawn(peer_server.map_err(|_| ()).map(|r| r.unwrap())); + + (runtime, tx, rx, handle) + } + + // unline fast_task pool, this one returns a cache for further analysis + fn create_test_slow_threads(log: Logger, threads: usize, rx: Receiver) -> (SharedCache, Vec>) { + let cache = SharedCache::new(); + + let mut handles = Vec::new(); + for _ in 0..threads { + let rx = rx.clone(); + let mut runner = SlowTaskRunner::new(log.clone(), cache.clone()); + let handle = std::thread::spawn(move || { + while let Ok(task) = rx.recv() { + runner.run(task) + } + }); - (runtime, rx, address) + handles.push(handle); + } + (cache, handles) } #[test] fn test_peer_protocol_capnp() { let log = prepare_log("test_peer_capnp"); + let address: ::std::net::SocketAddr = "127.0.0.1:8136".parse().unwrap(); let mut config = System::default(); config.metrics.log_parse_errors = true; - let mut 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 (runtime, tx, rx, server_handle) = prepare_runtime_with_server(log.clone(), address.clone()); + let (cache, slow_handles) = create_test_slow_threads(log.clone(), 3, rx); - let metric = outmetric.clone(); - let (mut runtime, mut rx, address) = prepare_runtime_with_server(log.clone()); - - let receiver = async move { - while let Some(task) = rx.next().await { - runner.run(task) - } + let outmetric = Metric::new(MetricValue::Gauge(42.), ts.into(), 1.); - let mut interm = Vec::with_capacity(128); - interm.resize(128, 0u8); - let m = TagFormat::Graphite; - - let single_name = MetricName::new("complex.test.bioyino_single".into(), m, &mut interm).unwrap(); - let multi_name = MetricName::new("complex.test.bioyino_multi".into(), m, &mut interm).unwrap(); - let shot_name = MetricName::new("complex.test.bioyino_snapshot".into(), m, &mut interm).unwrap(); - let tagged_name = MetricName::new("complex.test.bioyino_tagged;tag2=val2;tag1=value1".into(), m, &mut interm).unwrap(); - 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)); - assert_eq!(runner.get_short_entry(&tagged_name), Some(&outmetric)); - }; - runtime.spawn(receiver); + let metric = outmetric.clone(); + // Create a peer client and send some metrics to the server let sender = async move { let conn = TcpStream::connect(&address).await.expect("connecting tcp client"); + // Version 1 messages let mut single_message = Builder::new_default(); { let builder = single_message.init_root::(); let mut c_metric = builder.init_single(); c_metric.set_name("complex.test.bioyino_single"); - metric.fill_capnp(&mut c_metric); + metric.fill_capnp_v1(&mut c_metric); } let mut tagged_message = Builder::new_default(); { let builder = tagged_message.init_root::(); let mut c_metric = builder.init_single(); - c_metric.set_name("complex.test.bioyino_tagged;tag2=val2;tag1=value1"); - metric.fill_capnp(&mut c_metric); + // since we do this by hands, tags must be sorted + // in real world bio does not insert this from string, only from MetricName + c_metric.set_name("complex.test.bioyino_tagged;tag1=value1;tag2=val2"); + metric.fill_capnp_v1(&mut c_metric); } let mut multi_message = Builder::new_default(); @@ -513,7 +592,7 @@ mod test { 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); + metric.fill_capnp_v1(&mut new_metric); } let mut snapshot_message = Builder::new_default(); @@ -521,23 +600,97 @@ mod test { let builder = snapshot_message.init_root::(); let multi_metric = builder.init_snapshot(1); let mut new_metric = multi_metric.get(0); - new_metric.set_name("complex.test.bioyino_snapshot"); + new_metric.set_name("complex.test.bioyino_snapshot_v1"); + metric.fill_capnp_v1(&mut new_metric); + } + + // Version 2 should work along with version 1 + let mut snapshot_message_v2 = Builder::new_default(); + { + let mut interm = Vec::with_capacity(256); + interm.resize(256, 0); + let mut builder = snapshot_message_v2.init_root::(); + builder.set_version(ProtocolVersion::V2.id()); + let mut multi_metric = builder.init_snapshot(2); + let mut new_metric = multi_metric.reborrow().get(0); + let metric_name = MetricName::new("complex.test.bioyino_snapshot_v2".into(), TagFormat::Graphite, &mut interm).unwrap(); + metric.fill_capnp_name(&mut new_metric, &metric_name, true); + metric.fill_capnp(&mut new_metric); + + let mut new_metric = multi_metric.reborrow().get(1); + + let tagged_name = MetricName::new( + "complex.test.bioyino_snapshot_tagged_v2;tag1=value1;tag2=0".into(), + TagFormat::Graphite, + &mut interm, + ) + .unwrap(); + metric.fill_capnp_name(&mut new_metric, &tagged_name, true); metric.fill_capnp(&mut new_metric); } let mut conn = conn.compat_write(); write_message(&mut conn, single_message).await.unwrap(); + write_message(&mut conn, tagged_message).await.unwrap(); write_message(&mut conn, multi_message).await.unwrap(); write_message(&mut conn, snapshot_message).await.unwrap(); + write_message(&mut conn, snapshot_message_v2).await.unwrap(); + conn.into_inner().flush().await.unwrap(); }; - let delayed = async { - delay_for(Duration::from_secs(1)).await; - sender.await + let delayed = async move { + // let things settle + sleep(Duration::from_secs(1)).await; + // send the data + sender.await; + // give server time to process + sleep(Duration::from_secs(1)).await; + // stop the server + server_handle.abort(); + // let the slow threads stop + drop(tx); + // wait for them to stop + for handle in slow_handles { + handle.join().unwrap(); + } }; runtime.spawn(delayed); - let test_delay = async { delay_for(Duration::from_secs(2)).await }; + // analyze cache for correct contents + let mut interm = Vec::with_capacity(128); + interm.resize(128, 0u8); + let m = TagFormat::Graphite; + + let single_name = MetricName::new("complex.test.bioyino_single".into(), m, &mut interm).unwrap(); + let multi_name = MetricName::new("complex.test.bioyino_multi".into(), m, &mut interm).unwrap(); + let shot_name = MetricName::new("complex.test.bioyino_snapshot_v1".into(), m, &mut interm).unwrap(); + let shot_name_v2 = MetricName::new("complex.test.bioyino_snapshot_v2".into(), m, &mut interm).unwrap(); + let tagged_name = MetricName::new("complex.test.bioyino_tagged;tag2=val2;tag1=value1".into(), m, &mut interm).unwrap(); + let tagged_name_v2 = MetricName::new("complex.test.bioyino_snapshot_tagged_v2;tag1=value1;tag2=0".into(), m, &mut interm).unwrap(); + + let test_delay = async { sleep(Duration::from_secs(2)).await }; runtime.block_on(test_delay); + + let rotated = cache.rotate(true); + let cache = rotated + .into_iter() + //.map(|c| c.into_iter().map(|(k, _)|k).collect::>()) + //.filter(|v|v.len() > 0) + .flatten() + .map(|(k, v)| (k, v.into_inner().unwrap())) + .collect::>>(); + // show keys + // dbg!(&cache); + + assert_eq!(cache.get(&single_name), Some(&outmetric)); + assert_eq!(cache.get(&multi_name), Some(&outmetric)); + assert_eq!(cache.get(&shot_name), Some(&outmetric)); + assert_eq!(cache.get(&shot_name_v2), Some(&outmetric)); + assert_eq!(cache.get(&tagged_name), Some(&outmetric)); + assert_eq!(cache.get(&tagged_name), Some(&outmetric)); + assert_eq!(cache.get(&tagged_name_v2), Some(&outmetric)); + // check ourselves: in case the test is changed in future + // the resulting metric name has to be checked here too + assert_eq!(cache.len(), 6); } } diff --git a/src/raft.rs b/src/raft.rs index 9579979..01a6dc5 100644 --- a/src/raft.rs +++ b/src/raft.rs @@ -7,9 +7,9 @@ use rand::random; use slog::{warn, Logger}; -use futures::future::lazy; +use futures1::future::lazy; use socket2::Socket; -use tokio::runtime::current_thread::spawn; +use tokio1::runtime::current_thread::spawn; use raft_tokio::raft_consensus::persistent_log::mem::MemLog; use raft_tokio::raft_consensus::state::ConsensusState; @@ -90,7 +90,7 @@ pub(crate) fn start_internal_raft(options: Raft, logger: Logger) { if let Some(listen) = client_bind { let socket = unsafe { Socket::from_raw_fd(std_socket.as_raw_fd()) }; socket.bind(&listen.into())?; - *std_socket = socket.into_tcp_stream(); // ensure the ownership is passed back from builder + *std_socket = socket.into(); // ensure the ownership is passed back from builder } Ok(()) }; diff --git a/src/server.rs b/src/server.rs deleted file mode 100644 index efaef07..0000000 --- a/src/server.rs +++ /dev/null @@ -1,82 +0,0 @@ -use std::collections::hash_map::DefaultHasher; -use std::collections::HashMap; -use std::hash::{Hash, Hasher}; -use std::sync::atomic::{AtomicBool, Ordering}; -use std::sync::Arc; - -use bytes::{BufMut, BytesMut}; -use futures3::channel::mpsc; -use futures3::SinkExt; -use tokio2::net::UdpSocket; - -use slog::{error, o, warn, Logger}; - -use crate::config::System; -use crate::s; -use crate::stats::STATS; -use crate::task::Task; - -pub async fn async_statsd_server( - log: Logger, - socket: std::net::UdpSocket, - mut chans: Vec>, - config: Arc, - bufsize: usize, - thread_idx: usize, - flush_flags: Arc>, -) { - let mut bufmap = HashMap::new(); - let mut readbuf = Vec::with_capacity(bufsize); - readbuf.resize(bufsize, 0); - let mut recv_counter = 0; - let mut next = thread_idx; - let log = log.new(o!("source"=>"async_udp_server", "tid"=>thread_idx)); - let mut socket = UdpSocket::from_std(socket).expect("adding socket to event loop"); - loop { - let (size, addr) = match socket.recv_from(&mut readbuf).await { - Ok((0, _)) => { - // size = 0 means EOF - warn!(log, "exiting on EOF"); - break; - } - Ok((size, addr)) => (size, addr), - Err(e) => { - error!(log, "error receiving UDP packet {:?}", e); - break; - } - }; - // we only get here in case of success - STATS.ingress.fetch_add(size, Ordering::Relaxed); - - let buf = bufmap - .entry(addr) - .or_insert_with(|| BytesMut::with_capacity(config.network.buffer_flush_length)); - recv_counter += size; - // check we can fit the buffer - if buf.remaining_mut() < bufsize { - buf.reserve(size + 1) - } - buf.put_slice(&readbuf[..size]); - - let flush = flush_flags.get(thread_idx).unwrap().swap(false, Ordering::SeqCst); - if recv_counter >= config.network.buffer_flush_length || flush { - for (addr, buf) in bufmap.drain() { - let mut hasher = DefaultHasher::new(); - addr.hash(&mut hasher); - let ahash = hasher.finish(); - let chan = if config.metrics.consistent_parsing { - let chlen = chans.len(); - &mut chans[ahash as usize % chlen] - } else { - if next >= chans.len() { - next = 0; - } - let chan = &mut chans[next]; - next += 1; - chan - }; - chan.send(Task::Parse(ahash, buf)).await.unwrap_or_else(|_| s!(drops)); - } - } - } -} diff --git a/src/slow_task.rs b/src/slow_task.rs new file mode 100644 index 0000000..73447e5 --- /dev/null +++ b/src/slow_task.rs @@ -0,0 +1,110 @@ +use std::sync::Arc; + +use crossbeam_channel::Sender; +use futures::channel::oneshot; +use log::warn as logw; +use slog::{error, info, Logger}; + +use bioyino_metric::{name::MetricName, Metric, MetricTypeName}; + +use crate::aggregate::{aggregate_task, AggregationData}; +use crate::cache::{RotatedCache, SharedCache}; +use crate::config::System; + +use crate::{s, Cache, Float}; + +#[derive(Debug)] +pub enum SlowTask { + AddMetric(MetricName, Metric), + AddMetrics(Vec<(MetricName, Metric)>), + Join(Arc), + AddSnapshot(Vec<(MetricName, Metric)>), + Rotate(Option>), + Aggregate(AggregationData), +} + +pub fn start_slow_threads(log: Logger, config: &System) -> Result, std::io::Error> { + info!(log, "starting counting threads"); + let threads = config.w_threads; + let (tx, rx) = crossbeam_channel::bounded(config.task_queue_size); + + let cache = SharedCache::new(); + + for i in 0..threads { + let tlog = log.clone(); + let rx = rx.clone(); + let cache = cache.clone(); + let elog = log.clone(); + std::thread::Builder::new().name(format!("bioyino_cnt{}", i)).spawn(move || { + let mut runner = SlowTaskRunner::new(tlog, cache); + loop { + match rx.recv() { + Ok(task) => { + runner.run(task); + } + Err(_) => { + error!(elog, "all sending threads have closed connections"); + return; + } + } + } + })?; + } + + Ok(tx) +} + +pub struct SlowTaskRunner { + cache: SharedCache, + log: Logger, +} + +impl SlowTaskRunner { + pub fn new(log: Logger, cache: SharedCache) -> Self { + Self { cache, log } + } + + fn update_metric(&self, name: MetricName, metric: Metric) { + let ename = name.clone(); + let em = MetricTypeName::from_metric(&metric); + self.cache.accumulate(name, metric).unwrap_or_else(|_| { + logw!( + "could not accumulate in long cache at {:?} new type '{}'", + String::from_utf8_lossy(&ename.name[..]), + em.to_string(), + ); + s!(agg_errors); + }); + } + + pub fn run(&mut self, task: SlowTask) { + match task { + SlowTask::AddMetric(name, metric) => self.update_metric(name, metric), + SlowTask::AddMetrics(mut list) => { + list.drain(..).map(|(name, metric)| self.update_metric(name, metric)).last(); + } + SlowTask::Join(cache) => { + cache.iter().map(|(name, metric)| self.update_metric(name.clone(), metric.clone())).last(); + } + SlowTask::AddSnapshot(mut list) => { + list.drain(..).map(|(name, metric)| self.update_metric(name, metric)).last(); + } + SlowTask::Rotate(channel) => { + let rotated = self.cache.rotate(channel.is_some()); + if let Some(c) = channel { + let log = self.log.clone(); + c.send(rotated).unwrap_or_else(|_| { + s!(queue_errors); + info!(log, "task could not send rotated metric, receiving thread may be dead"); + }); + } + } + SlowTask::Aggregate(data) => aggregate_task(data), + } + } +} + +#[cfg(test)] +mod tests { + //use super::*; +} diff --git a/src/stats.rs b/src/stats.rs index 04b3ba6..417c4f9 100644 --- a/src/stats.rs +++ b/src/stats.rs @@ -1,23 +1,29 @@ use std::sync::atomic::{AtomicUsize, Ordering}; -use std::sync::Mutex; +use std::collections::HashMap; +use std::iter::FromIterator; use std::time::{self, Duration, SystemTime}; -use bytes::{Bytes, BytesMut}; -use futures3::channel::mpsc::Sender; -use futures3::sink::SinkExt; +use serde::Serialize; +use bytes::{Bytes, BytesMut, BufMut}; use once_cell::sync::Lazy; use slog::{info, o, Logger}; +use crossbeam_channel::Sender; -use bioyino_metric::{name::MetricName, Metric, MetricType}; +use tokio::sync::RwLock; + +use bioyino_metric::{name::MetricName, StatsdMetric, StatsdType, FromF64}; use crate::errors::GeneralError; -use crate::task::Task; +use crate::fast_task::FastTask; +use crate::slow_task::SlowTask; use crate::Float; pub struct Stats { - pub egress: AtomicUsize, + pub egress_carbon: AtomicUsize, + pub egress_peer: AtomicUsize, pub ingress: AtomicUsize, pub ingress_metrics: AtomicUsize, + pub ingress_metrics_peer: AtomicUsize, pub drops: AtomicUsize, pub parse_errors: AtomicUsize, pub agg_errors: AtomicUsize, @@ -26,9 +32,11 @@ pub struct Stats { } pub static STATS: Stats = Stats { - egress: AtomicUsize::new(0), + egress_carbon: AtomicUsize::new(0), + egress_peer: AtomicUsize::new(0), ingress: AtomicUsize::new(0), ingress_metrics: AtomicUsize::new(0), + ingress_metrics_peer: AtomicUsize::new(0), drops: AtomicUsize::new(0), parse_errors: AtomicUsize::new(0), agg_errors: AtomicUsize::new(0), @@ -36,7 +44,7 @@ pub static STATS: Stats = Stats { queue_errors: AtomicUsize::new(0), }; -pub static STATS_SNAP: Lazy> = Lazy::new(|| Mutex::new(OwnSnapshot::default())); +pub static STATS_SNAP: Lazy> = Lazy::new(|| RwLock::new(OwnSnapshot::default())); #[macro_export] macro_rules! s { @@ -45,6 +53,7 @@ macro_rules! s { }}; } +#[derive(Clone)] pub struct OwnSnapshot { pub ts: u128, pub data: Vec<(Bytes, Float)>, @@ -58,8 +67,53 @@ impl Default for OwnSnapshot { .map_err(GeneralError::Time) .expect("getting system time") .as_millis(), - data: Vec::new(), + data: Vec::new(), + } + } +} + + +impl OwnSnapshot { + pub(crate) fn render(&self, json: bool) -> Bytes { + let mut buf = BytesMut::new(); + + if !json { + let ts = self.ts.to_string(); + for (name, value) in &self.data { + buf.extend_from_slice(&name[..]); + buf.extend_from_slice(&b" "[..]); + // write somehow doesn't extend buffer size giving "cannot fill buffer" error + buf.reserve(64); + let mut writer = buf.writer(); + dtoa::write(&mut writer, *value).map(|_|()).unwrap_or(()); // TODO: think if we should not ignore float error + buf = writer.into_inner(); + buf.extend_from_slice(&b" "[..]); + buf.extend_from_slice(ts.as_bytes()); + buf.extend_from_slice(&b"\n"[..]); + } + + } else { + + #[derive(Serialize)] + struct JsonSnap { + ts: u128, + metrics: HashMap, + } + + let snap = JsonSnap { + ts: self.ts, + metrics: HashMap::from_iter( + self + .data + .iter() + .map(|(name, value)| (String::from_utf8_lossy(&name[..]).to_string(), *value)), + ), + }; + let mut writer = buf.writer(); + serde_json::to_writer_pretty(&mut writer, &snap).unwrap_or(()); + buf = writer.into_inner(); } + buf.freeze() } } @@ -67,93 +121,116 @@ impl Default for OwnSnapshot { pub struct OwnStats { interval: u64, prefix: String, - chans: Vec>, next_chan: usize, + slow_chan: Sender, + fast_chans: Vec>, log: Logger, } impl OwnStats { - pub fn new(interval: u64, prefix: String, chans: Vec>, log: Logger) -> Self { + pub fn new(interval: u64, prefix: String, slow_chan: Sender, fast_chans: Vec>, log: Logger) -> Self { let log = log.new(o!("source"=>"stats")); Self { interval, prefix, - chans, - next_chan: 0, + next_chan: fast_chans.len(), + slow_chan, + fast_chans, log, } } - fn format_metric_carbon(&self, buf: &mut BytesMut, suffix: &[u8], value: Float) -> (MetricName, Metric) { + fn format_metric_carbon(&self, buf: &mut BytesMut, suffix: &[u8]) -> MetricName { buf.extend_from_slice(self.prefix.as_bytes()); buf.extend_from_slice(&b"."[..]); buf.extend_from_slice(suffix); let name = MetricName::new_untagged(buf.split()); - let metric = Metric::new(value, MetricType::Counter, None, None).unwrap(); - (name, metric) + name } - async fn count(&mut self) { + fn next_chan(&mut self) -> &Sender { + self.next_chan = if self.next_chan >= (self.fast_chans.len() - 1) { 0 } else { self.next_chan + 1 }; + &self.fast_chans[self.next_chan] + } + + fn count(&mut self) -> OwnSnapshot { let mut buf = BytesMut::with_capacity((self.prefix.len() + 10) * 7); // 10 is suffix len, 7 is number of metrics let mut snapshot = OwnSnapshot::default(); let s_interval = if self.interval > 0 { self.interval as f64 / 1000f64 } else { 1f64 }; - - // we will start from 1st worker, not 0th, but who cares, we rotate them every second - self.next_chan += 1; - // this also covers situation when the number of workers = 1 - if self.next_chan >= self.chans.len() { - self.next_chan = 0; - } + let s_interval = Float::from_f64(s_interval); macro_rules! add_metric { ($value:ident, $suffix:expr) => { let $value = STATS.$value.swap(0, Ordering::Relaxed) as Float; if self.interval > 0 { snapshot.data.push((Bytes::copy_from_slice(($suffix).as_bytes()), $value / s_interval)); - let (name, metric) = self.format_metric_carbon(&mut buf, $suffix.as_bytes(), $value); - self.chans[self.next_chan] - .clone() - .send(Task::AddMetric(name, metric)) - .await + let metric = StatsdMetric::new($value, StatsdType::Counter, None).unwrap(); + let name = self.format_metric_carbon(&mut buf, $suffix.as_bytes()); + let chan = self.next_chan(); + chan + .send(FastTask::Accumulate(name, metric)) .map_err(|_| s!(queue_errors)) - .unwrap_or(()); + .unwrap_or(()) } }; - }; + } - add_metric!(egress, "egress"); + add_metric!(egress_carbon, "egress-carbon"); + add_metric!(egress_peer, "egress-peer"); add_metric!(ingress, "ingress"); add_metric!(ingress_metrics, "ingress-metric"); + add_metric!(ingress_metrics_peer, "ingress-metric-peer"); add_metric!(drops, "drop"); add_metric!(agg_errors, "agg-error"); add_metric!(parse_errors, "parse-error"); add_metric!(queue_errors, "queue-error"); add_metric!(peer_errors, "peer-error"); - { - let mut prev = STATS_SNAP.lock().unwrap(); - *prev = snapshot; - } + + // queue len has other type, so macro does not fit here + let chlen = Float::from_f64(self.slow_chan.len() as f64); + let qlen = StatsdMetric::new(chlen, StatsdType::Gauge(None), None).unwrap(); + snapshot.data.push((Bytes::copy_from_slice(("slow-q-len").as_bytes()), chlen)); + let name = self.format_metric_carbon(&mut buf, "slow-w-len".as_bytes()); + + let chan = self.next_chan(); + chan + .send(FastTask::Accumulate(name, qlen)) + .map_err(|_| s!(queue_errors)) + .unwrap_or(()); + + if self.interval > 0 { info!(self.log, "stats"; - "egress" => format!("{:2}", egress / s_interval), + "egress-c" => format!("{:2}", egress_carbon / s_interval), + "egress-p" => format!("{:2}", egress_peer / s_interval), "ingress" => format!("{:2}", ingress / s_interval), "ingress-m" => format!("{:2}", ingress_metrics / s_interval), + "ingress-m-p" => format!("{:2}", ingress_metrics_peer / s_interval), "drops" => format!("{:2}", drops / s_interval), "a-err" => format!("{:2}", agg_errors / s_interval), "p-err" => format!("{:2}", parse_errors / s_interval), "pe-err" => format!("{:2}", peer_errors / s_interval), "qu-err" => format!("{:2}", queue_errors / s_interval), + "qu-len" => format!("{:2}", chlen), ); - } + }; + snapshot } pub async fn run(mut self) { - let now = tokio2::time::Instant::now(); + let now = tokio::time::Instant::now(); let dur = Duration::from_millis(if self.interval < 100 { 1000 } else { self.interval }); // avoid too short intervals - let mut interval = tokio2::time::interval_at(now + dur, dur); + let mut interval = tokio::time::interval_at(now + dur, dur); loop { interval.tick().await; - self.count().await; + let snapshot = tokio::task::block_in_place(||{ + self.count() + }); + // update global snapshot + { + let mut prev = STATS_SNAP.write().await; + *prev = snapshot + } } } } diff --git a/src/udp.rs b/src/sync_udp.rs similarity index 80% rename from src/udp.rs rename to src/sync_udp.rs index ac8b931..c90ff6c 100644 --- a/src/udp.rs +++ b/src/sync_udp.rs @@ -9,35 +9,32 @@ use std::sync::Arc; use std::thread; use bytes::{BufMut, BytesMut}; -use futures3::channel::mpsc::Sender; -use futures3::future::pending; +use crossbeam_channel::Sender; use slog::{info, o, warn, Logger}; use socket2::{Domain, Protocol, Socket, Type}; use std::os::unix::io::AsRawFd; -use tokio2::runtime::Builder; use crate::config::System; +use crate::fast_task::FastTask; use crate::stats::STATS; -use crate::task::Task; pub(crate) fn start_sync_udp( log: Logger, listen: SocketAddr, - chans: &[Sender], + chans: &[Sender], config: Arc, n_threads: usize, bufsize: usize, mm_packets: usize, mm_async: bool, mm_timeout: u64, - flush_flags: Arc>, -) { +) -> 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 = Socket::new(Domain::ipv4(), Type::dgram(), Some(Protocol::udp())).expect("creating UDP socket"); + let socket = Socket::new(Domain::IPV4, Type::DGRAM, Some(Protocol::UDP)).expect("creating UDP socket"); socket.set_reuse_address(true).expect("reusing address"); socket.set_reuse_port(true).expect("reusing port"); socket.set_nonblocking(mm_async).expect("setting O_NONBLOCK option"); @@ -45,8 +42,18 @@ pub(crate) fn start_sync_udp( let mm_timeout = if mm_timeout == 0 { config.network.buffer_flush_time } else { mm_timeout }; + // For each out sync thread we create the buffer flush timer, that sets the atomic value to 1 + // every interval + let mut flush_flags = Vec::new(); + + for _ in 0..n_threads { + flush_flags.push(AtomicBool::new(false)); + } + + let flush_flags = Arc::new(flush_flags); + for i in 0..n_threads { - let mut chans = chans.to_owned(); + let chans = chans.to_owned(); let log = log.new(o!("source"=>"mudp_thread")); let sck = socket.try_clone().unwrap(); @@ -203,15 +210,16 @@ pub(crate) fn start_sync_udp( let mut hasher = DefaultHasher::new(); hasher.write(&addr); let ahash = hasher.finish(); + let chan = if config.metrics.consistent_parsing { - &mut chans[ahash as usize % chlen] + &chans[ahash as usize % chlen] } else { next_chan = if next_chan >= (chlen - 1) { 0 } else { next_chan + 1 }; - &mut chans[next_chan] + &chans[next_chan] }; let buf = buf.split(); let buflen = buf.len(); - chan.try_send(Task::Parse(ahash, buf)) + chan.try_send(FastTask::Parse(ahash, buf)) .map_err(|_| { STATS.drops.fetch_add(buflen, Ordering::Relaxed); }) @@ -234,67 +242,5 @@ pub(crate) fn start_sync_udp( }) .expect("starting multimsg thread"); } -} - -pub(crate) fn start_async_udp( - log: Logger, - listen: SocketAddr, - chans: &[Sender], - config: Arc, - n_threads: usize, - greens: usize, - async_sockets: usize, - bufsize: 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 = Socket::new(Domain::ipv4(), Type::dgram(), Some(Protocol::udp())).expect("creating UDP socket"); - socket.set_reuse_address(true).expect("reusing address"); - socket.set_reuse_port(true).expect("reusing port"); - socket.bind(&listen.into()).expect("binding"); - - 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.to_owned(); - let flush_flags = flush_flags.clone(); - let config = config.clone(); - let log = log.clone(); - thread::Builder::new() - .name(format!("bioyino_udp{}", i)) - .spawn(move || { - // each thread runs it's own runtime - let mut runtime = Builder::new().basic_scheduler().enable_all().build().expect("creating runtime for test"); - // Inside each green thread - for _ in 0..greens { - // start a listener for all sockets - for socket in sockets.iter() { - let chans = chans.clone(); - // create UDP listener - let socket = socket.try_clone().expect("cloning socket"); - - runtime.spawn(crate::server::async_statsd_server( - log.clone(), - socket.into(), - chans.clone(), - config.clone(), - bufsize, - i, - flush_flags.clone(), - )); - } - } - - runtime.block_on(pending::<()>()) - }) - .expect("creating UDP reader thread"); - } + flush_flags } diff --git a/src/task.rs b/src/task.rs deleted file mode 100644 index d7de553..0000000 --- a/src/task.rs +++ /dev/null @@ -1,344 +0,0 @@ -use std::collections::hash_map::Entry; -use std::collections::HashMap; -use std::sync::{atomic::Ordering, Arc}; - -use bytes::{BufMut, BytesMut}; -use futures3::channel::mpsc::UnboundedSender; -use futures3::channel::oneshot; -use futures3::SinkExt; -use log::warn as logw; -use slog::{info, warn, Logger}; - -use tokio2::spawn; - -use bioyino_metric::parser::{MetricParser, MetricParsingError, ParseErrorHandler}; -use bioyino_metric::{name::MetricName, Metric, MetricTypeName}; - -use crate::aggregate::{aggregate_task, AggregationData}; -use crate::config::System; - -use crate::{s, Cache, ConsensusKind, Float, IS_LEADER}; - -#[derive(Debug)] -pub enum Task { - Parse(u64, BytesMut), - AddMetric(MetricName, Metric), - AddMetrics(Vec<(MetricName, Metric)>), - AddSnapshot(Vec<(MetricName, Metric)>), - TakeSnapshot(oneshot::Sender), - Rotate(Option>), - Aggregate(AggregationData), -} - -fn update_metric(cache: &mut Cache, name: MetricName, metric: Metric) { - let ename = name.clone(); - let em = MetricTypeName::from_metric(&metric); - match cache.entry(name) { - Entry::Occupied(ref mut entry) => { - let mtype = MetricTypeName::from_metric(&entry.get()); - entry.get_mut().accumulate(metric).unwrap_or_else(|_| { - logw!( - "could not accumulate {:?}: type '{}' into type '{}'", - String::from_utf8_lossy(&ename.name[..]), - em.to_string(), - mtype.to_string(), - ); - s!(agg_errors); - }); - } - Entry::Vacant(entry) => { - entry.insert(metric); - } - }; -} - -#[derive(Debug)] -pub struct TaskRunner { - long: HashMap>, - short: HashMap>, - buffers: HashMap, - names_arena: BytesMut, - 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), - names_arena: BytesMut::new(), - 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 buf = { - let len = buf.len(); - let (_, ref mut prev_buf) = self - .buffers - .entry(addr) - .and_modify(|(times, _)| { - *times = 0; - }) - .or_insert((0, BytesMut::with_capacity(len))); - prev_buf.reserve(buf.len()); - prev_buf.put(buf); - prev_buf - }; - - let parser = MetricParser::new( - buf, - self.config.metrics.max_unparsed_buffer, - self.config.metrics.max_tags_len, - TaskParseErrorHandler(log), - ); - - for (mut name, metric) in parser { - s!(ingress_metrics); - if name.has_tags() && self.config.metrics.create_untagged_copy { - self.names_arena.extend_from_slice(name.name_without_tags()); - let untagged = MetricName::new_untagged(self.names_arena.split()); - update_metric(&mut self.short, untagged, metric.clone()); - } - update_metric(&mut self.short, name, metric); - } - } - 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 - // we also skip sorting tags in this mode, considering them being already sorted by - // other node - list.drain(..).map(|(name, metric)| update_metric(&mut self.long, name, metric)).last(); - } - Task::TakeSnapshot(channel) => { - let is_leader = IS_LEADER.load(Ordering::SeqCst); - let short = if !is_leader && self.config.consensus == ConsensusKind::None { - // there is special case used in agents: when we are not leader and there is - // no consensus, that cannot make us leader, there is no point of aggregating - // long cache at all because it will never be sent anywhere - let mut prev_short = HashMap::with_capacity(self.short.len()); - std::mem::swap(&mut prev_short, &mut self.short); - prev_short - } else { - // clone short cache for further sending - let short = self.short.clone(); - // join short cache to long cache removing data from short - { - let mut long = &mut self.long; // self.long cannot be borrowed in map, so we borrow it earlier - self.short.drain().map(|(name, metric)| update_metric(&mut long, name, metric)).last(); - } - short - }; - // self.short now contains empty hashmap because of draining - // give a copy of snapshot to requestor - channel.send(short).unwrap_or_else(|_| { - s!(queue_errors); - info!(self.log, "task could not send snapshot, receiving thread may be dead"); - }); - } - Task::Rotate(channel) => { - // this was the code before and it probably was not optimal because of copying lots - // of data potentially - //let rotated = self.long.clone(); - //self.long.clear(); - //self.long = HashMap::with_capacity(rotated.len()); - - // we need our long cache to be sent for processing - // in place of it we need a new cache with most probably the same size - // BUT if we use exactly the same size, it may grow infinitely in long term - // so we halve the size so it could be reduced if ingress flow amounts - // become lower - - // TODO: we could avoid this allocation when we don't need it - // but it would be hard to get size decrease until shrink_to method - // is stabilized - // - // if it does, we can - // { long.clear() - // long.shrink_to(prev_len / 2) } - let mut rotated = HashMap::with_capacity(self.long.len() / 2); - std::mem::swap(&mut self.long, &mut rotated); - if let Some(mut c) = channel { - let log = self.log.clone(); - spawn(async move { - c.send(rotated).await.unwrap_or_else(|_| { - s!(queue_errors); - info!(log, "task could not send rotated metric, receiving thread may be dead"); - }); - }); - } - self.buffers.retain(|_, (ref mut times, _)| { - *times += 1; - *times < 5 - }); - } - - Task::Aggregate(data) => aggregate_task(data), - } - } - - // used in tests in peer.rs - pub fn get_long_entry(&self, e: &MetricName) -> Option<&Metric> { - self.long.get(e) - } - pub fn get_short_entry(&self, e: &MetricName) -> Option<&Metric> { - self.short.get(e) - } -} - -struct TaskParseErrorHandler(Option); - -impl ParseErrorHandler for TaskParseErrorHandler { - fn handle(&self, input: &[u8], _pos: usize, e: MetricParsingError) { - s!(parse_errors); - if let Some(ref log) = self.0 { - if let Ok(string) = std::str::from_utf8(input) { - // TODO better error formatting instead of Debug - warn!(log, "parsing error"; "buffer"=> format!("{:?}", string), "position"=>format!("{}", e.position.translate_position(input)), "error"=>format!("{:?}", e)); - } else { - warn!(log, "parsing error (bad unicode)"; "buffer"=> format!("{:?}", input), "position"=>format!("{}",e.position.translate_position(input) ), "error"=>format!("{:?}", e)); - } - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - use bioyino_metric::{name::TagFormat, MetricType}; - - use crate::util::{new_test_graphite_name as new_name, prepare_log}; - - #[test] - fn aggregate_tagged_metrics() { - let mut data = BytesMut::new(); - - // ensure metrics with same tags (going probably in different orders) go to same aggregate - data.extend_from_slice(b"gorets;t2=v2;t1=v1:1000|c"); - data.extend_from_slice(b"\ngorets;t1=v1;t2=v2:1000|c"); - - // ensure metrics with same name but different tags go to different aggregates - data.extend_from_slice(b"\ngorets;t1=v1;t2=v3:1000|c"); - - let mut config = System::default(); - config.metrics.log_parse_errors = true; - let mut runner = TaskRunner::new(prepare_log("aggregate_tagged"), Arc::new(config), 16); - runner.run(Task::Parse(2, data)); - - let mut intermediate = Vec::new(); - intermediate.resize(9000, 0u8); - let mode = TagFormat::Graphite; - - // must be aggregated into two as sum - let key = MetricName::new("gorets;t1=v1;t2=v2".into(), mode, &mut intermediate).unwrap(); - assert!(runner.short.contains_key(&key), "could not find {:?}", key); - let metric = runner.short.get(&key).unwrap().clone(); - assert_eq!(metric.value, 2000f64); - assert_eq!(metric.mtype, MetricType::Counter); - - // must be aggregated into separate - let key = MetricName::new("gorets;t1=v1;t2=v3".into(), mode, &mut intermediate).unwrap(); - assert!(runner.short.contains_key(&key), "could not find {:?}", key); - let metric = runner.short.get(&key).unwrap().clone(); - assert_eq!(metric.value, 1000f64); - assert_eq!(metric.mtype, MetricType::Counter); - } - - #[test] - fn create_untagged_copy() { - let mut data = BytesMut::new(); - data.extend_from_slice(b"tagged.metric;t2=v2;t1=v1:1000|c"); - - let mut config = System::default(); - config.metrics.create_untagged_copy = true; - let mut runner = TaskRunner::new(prepare_log("aggregate_with_copy"), Arc::new(config), 16); - // "send" metric two times - runner.run(Task::Parse(2, data.clone())); - runner.run(Task::Parse(2, data)); - - assert_eq!(runner.short.len(), 2, "additional metrics apepar from nowhere"); - // must be aggregated into two as sum - let key = new_name("tagged.metric;t1=v1;t2=v2"); - assert!(runner.short.contains_key(&key), "could not find {:?}", key); - let metric = runner.short.get(&key).unwrap().clone(); - assert_eq!(metric.value, 2000f64); - assert_eq!(metric.mtype, MetricType::Counter); - assert_eq!(metric.update_counter, 2); - - // ensure "independent" untagged version of tagged metric also exists with same values - let key = new_name("tagged.metric"); - assert!(runner.short.contains_key(&key), "could not find {:?}", key); - let metric = runner.short.get(&key).unwrap().clone(); - assert_eq!(metric.value, 2000f64); - assert_eq!(metric.mtype, MetricType::Counter); - assert_eq!(metric.update_counter, 2); - } - - #[test] - fn parse_trashed_metric_buf() { - let mut data = BytesMut::new(); - data.extend_from_slice(b"trash\ngorets1:+1000|g\nTRASH\ngorets2;tag3=shit;t2=fuck:-1000|g|@0.5\nMORE;tra=sh;|TrasH\nFUUU"); - - let mut config = System::default(); - config.metrics.log_parse_errors = true; - let mut runner = TaskRunner::new(prepare_log("parse_trashed"), Arc::new(config), 16); - runner.run(Task::Parse(2, data)); - - let key = new_name("gorets1"); - 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); - - // expect tags to be sorted after parsing - let key = new_name("gorets2;t2=fuck;tag3=shit"); - 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)); - } - - /* - TODO: e2e for tasks - #[test] - fn parse_then_aggregate() { - let mut data = BytesMut::new(); - - // ensure metrics with same tags (going probably in different orders) go to same aggregate - data.extend_from_slice(b"gorets;t2=v2;t1=v1:1000|c"); - data.extend_from_slice(b"\ngorets;t1=v1;t2=v2:1000|c"); - - // ensure metrics with same name but different tags go to different aggregates - data.extend_from_slice(b"\ngorets;t1=v1;t2=v3:1000|c"); - - let mut config = System::default(); - config.metrics.log_parse_errors = true; - let mut runner = TaskRunner::new(prepare_log("aggregate_tagged"), Arc::new(config), 16); - runner.run(Task::Parse(2, data)); - - dbg!(&runner.short); - // must be aggregated into two as sum - let key = MetricName::new("gorets;t1=v1;t2=v2".into(), None); - assert!(runner.short.contains_key(&key), "could not find {:?}", key); - let metric = runner.short.get(&key).unwrap().clone(); - assert_eq!(metric.value, 2000f64); - assert_eq!(metric.mtype, MetricType::Counter); - - // must be aggregated into separate - let key = MetricName::new("gorets;t1=v1;t2=v3".into(), None); - assert!(runner.short.contains_key(&key), "could not find {:?}", key); - let metric = runner.short.get(&key).unwrap().clone(); - assert_eq!(metric.value, 1000f64); - assert_eq!(metric.mtype, MetricType::Counter); - } - */ -} diff --git a/src/util.rs b/src/util.rs index dbb1e9a..0d855c3 100644 --- a/src/util.rs +++ b/src/util.rs @@ -7,7 +7,7 @@ use std::time::Duration; use thiserror::Error; -use futures3::future::{Future as Future3, TryFutureExt}; +use futures::future::{Future, TryFutureExt}; use resolve::resolver; use slog::{o, warn, Drain, Logger}; use socket2::{Domain, Socket, Type}; @@ -31,12 +31,17 @@ pub enum OtherError { NotFound(String), } +#[cfg(test)] 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 drain = slog_async::Async::new(filter) + .overflow_strategy(slog_async::OverflowStrategy::Block) + .thread_name("bioyino_log".into()) + .chan_size(2048) + .build().fuse(); slog::Logger::root(drain, o!("program"=>"test", "test"=>root)) } @@ -45,11 +50,15 @@ pub(crate) fn setup_logging(daemon: bool, verbosity_console: Verbosity, verbosit ($drain:ident) => { { let values = o!("program"=>"bioyino"); - let drain = slog_async::Async::new($drain).build().fuse(); + let drain = slog_async::Async::new($drain) + .thread_name("bioyino_log".into()) + .chan_size(2048) + .build() + .fuse(); slog::Logger::root(drain, values) } } - }; + } // the complex logic here could be decreased using boxed drains, // but we don't want it yet, for meaningless imaginary performance benefits @@ -104,11 +113,11 @@ pub fn try_resolve(s: &str) -> SocketAddr { } pub fn bound_stream(addr: &SocketAddr) -> Result { - let socket = Socket::new(Domain::ipv4(), Type::stream(), None)?; + let socket = Socket::new(Domain::IPV4, Type::STREAM, None)?; socket.set_reuse_address(true)?; socket.set_reuse_port(true)?; socket.bind(&addr.clone().into())?; - Ok(socket.into_tcp_stream()) + Ok(socket.into()) } //pub fn reusing_listener(addr: &SocketAddr) -> Result { @@ -138,7 +147,7 @@ pub async fn resolve_with_port(host: &str, default_port: u16) -> Result Result { - let resolver = TokioAsyncResolver::tokio_from_system_conf().await?; + let resolver = TokioAsyncResolver::tokio_from_system_conf()?; let response = resolver.lookup_ip(host).await?; @@ -212,7 +221,7 @@ impl Backoff { self.retries -= 1; let delay = self.next_sleep(); - tokio2::time::delay_for(Duration::from_millis(delay)).await; + tokio::time::sleep(Duration::from_millis(delay)).await; Ok(self.retries) } } @@ -230,16 +239,16 @@ impl Backoff { // TODO maybe let caller know it was out of tries, not just the last error pub async fn retry_with_backoff(mut bo: Backoff, mut f: F) -> Result where - I: Future3>, + I: Future>, F: FnMut() -> I, -{ - loop { - match f().await { - r @ Ok(_) => break r, - Err(e) => { - bo.sleep().map_err(|()| e).await?; - continue; + { + loop { + match f().await { + r @ Ok(_) => break r, + Err(e) => { + bo.sleep().map_err(|()| e).await?; + continue; + } } } } -} diff --git a/test/fixtures/full.toml b/test/fixtures/full.toml index b70777f..6e63051 100644 --- a/test/fixtures/full.toml +++ b/test/fixtures/full.toml @@ -5,7 +5,9 @@ verbosity-syslog = "info" verbosity-console = "debug" daemon = true n-threads = 4 +p-threads = 4 w-threads = 4 +a-threads = 4 task-queue-size = 1024 start-as-leader = true stats-interval = 10000 @@ -21,11 +23,9 @@ create-untagged-copy = true [aggregation] round-timestamp = "up" -mode = "separate" -threads = 8 update-count-threshold = 10 -aggregates.timer = [ "count", "updates", "min", "max", "median", "percentile-90", "percentile-999", "percentile-98", "percentile-50", "percentile-999", "percentile-80" ] -aggregates.set = [ "count", "count", "value"] +aggregates.timer = [ "count", "updates", "min", "max", "median", "rate", "percentile-90", "percentile-999", "percentile-98", "percentile-50", "percentile-999", "percentile-80" ] +aggregates.set = [ "count", "count", "value", "rate"] #ms-aggregates = [ "count", "min", "max", "sum", "median", "mean", "percentile-75", "percentile-95", "percentile-98", "percentile-99", "percentile-999" ] [naming.default] @@ -38,7 +38,7 @@ tag-values = { "min" = "lower", "max" = "upper", "percentile-50" = "percentile.5 [naming.timer] prefix = "split.timers" -prefix-overrides = { "updates" = "qewr" } +prefix-overrides = { "updates" = "qewr", "rate" = "rate30" } [naming.set] tag-values = { "count" = "set.counts" } @@ -57,6 +57,7 @@ chunks = 3 listen = "127.0.0.1:8125" peer-listen = "127.0.0.1:8136" #peer-client-bind = "127.0.0.1:8183" +peer-protocol = "2" mgmt-listen = "127.0.0.1:8137" bufsize = 1500 multimessage = true @@ -79,10 +80,3 @@ election-timeout-max = 751 this-node = "some-node" nodes = {"127.0.0.1:8138" = 1} client-bind = "127.0.0.1:8138" - -[consul] -start-as = "disabled" -agent = "127.0.0.1:8500" -session-ttl = 11000 -renew-time = 1000 -key-name = "service/bioyino/lock"